From 9a70e77955cbb954168cc2402ae510559305d194 Mon Sep 17 00:00:00 2001 From: qiaozhanwei Date: Mon, 10 Feb 2020 18:01:56 +0800 Subject: [PATCH 001/221] =?UTF-8?q?1=EF=BC=8Cremove=20dolphinscheduler-rpc?= =?UTF-8?q?=20module=20=202=EF=BC=8Cadd=20dolphinscheduler-remote=20module?= =?UTF-8?q?=203=EF=BC=8Cadd=20dolphinscheduler-service=20module=204?= =?UTF-8?q?=EF=BC=8Crefactor=20LoggerServer=20module=20(#1925)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module * ProcessUtils modify --- dolphinscheduler-api/pom.xml | 8 +- .../dolphinscheduler/api/log/LogClient.java | 137 ---------- .../api/service/LoggerService.java | 6 +- dolphinscheduler-remote/pom.xml | 44 ++++ .../remote/NettyRemotingClient.java | 197 +++++++++++++++ .../remote/NettyRemotingServer.java | 165 ++++++++++++ .../remote/codec/NettyDecoder.java | 92 +++++++ .../remote/codec/NettyEncoder.java | 44 ++++ .../remote/command/Command.java | 102 ++++++++ .../remote/command/CommandHeader.java | 55 ++++ .../remote/command/CommandType.java | 1 + .../command/ExecuteTaskRequestCommand.java | 1 + .../command/ExecuteTaskResponseCommand.java | 1 + .../dolphinscheduler/remote/command/Ping.java | 57 +++++ .../dolphinscheduler/remote/command/Pong.java | 54 ++++ .../log/GetLogBytesRequestCommand.java | 62 +++++ .../log/GetLogBytesResponseCommand.java | 56 +++++ .../log/RollViewLogRequestCommand.java | 80 ++++++ .../log/RollViewLogResponseCommand.java | 55 ++++ .../command/log/ViewLogRequestCommand.java | 58 +++++ .../command/log/ViewLogResponseCommand.java | 55 ++++ .../remote/config/NettyClientConfig.java | 76 ++++++ .../remote/config/NettyServerConfig.java | 95 +++++++ .../remote/exceptions/RemotingException.java | 94 +++++++ .../remote/handler/NettyClientHandler.java | 123 +++++++++ .../remote/handler/NettyServerHandler.java | 121 +++++++++ .../processor/NettyRequestProcessor.java | 28 +++ .../remote/utils/Address.java | 90 +++++++ .../remote/utils/ChannelUtils.java | 41 +++ .../remote/utils/Constants.java | 31 +++ .../remote/utils/FastJsonSerializer.java | 39 +++ .../dolphinscheduler/remote/utils/Pair.java | 47 ++++ .../remote/NettyRemotingClientTest.java | 71 ++++++ dolphinscheduler-rpc/pom.xml | 113 --------- .../src/main/proto/scheduler.proto | 101 -------- dolphinscheduler-server/pom.xml | 2 +- .../server/log/LoggerRequestProcessor.java | 179 +++++++++++++ .../server/log/LoggerServer.java | 91 +++++++ .../server/rpc/LogClient.java | 149 ----------- .../server/rpc/LoggerServer.java | 238 ------------------ .../server/utils/ProcessUtils.java | 4 +- dolphinscheduler-service/pom.xml | 29 +++ .../service/MasterResponseCommand.java | 55 ++++ .../service/WorkerRequestCommand.java | 58 +++++ .../service/log/LogClientService.java | 166 ++++++++++++ .../service/log/LogPromise.java | 81 ++++++ .../service/worker/WorkerClientService.java | 107 ++++++++ pom.xml | 13 +- 48 files changed, 2819 insertions(+), 753 deletions(-) delete mode 100644 dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/log/LogClient.java create mode 100644 dolphinscheduler-remote/pom.xml create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingServer.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/codec/NettyDecoder.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/codec/NettyEncoder.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Command.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandHeader.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandType.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskRequestCommand.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskResponseCommand.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Ping.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Pong.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesRequestCommand.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesResponseCommand.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogRequestCommand.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogResponseCommand.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogRequestCommand.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogResponseCommand.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/config/NettyClientConfig.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/config/NettyServerConfig.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingException.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyServerHandler.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/NettyRequestProcessor.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Address.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/ChannelUtils.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Constants.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/FastJsonSerializer.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Pair.java create mode 100644 dolphinscheduler-remote/src/test/java/org/apache/dolphinscheduler/remote/NettyRemotingClientTest.java delete mode 100644 dolphinscheduler-rpc/pom.xml delete mode 100644 dolphinscheduler-rpc/src/main/proto/scheduler.proto create mode 100644 dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/log/LoggerRequestProcessor.java create mode 100644 dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/log/LoggerServer.java delete mode 100644 dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/rpc/LogClient.java delete mode 100644 dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/rpc/LoggerServer.java create mode 100644 dolphinscheduler-service/pom.xml create mode 100644 dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/MasterResponseCommand.java create mode 100644 dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/WorkerRequestCommand.java create mode 100644 dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java create mode 100644 dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogPromise.java create mode 100644 dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/worker/WorkerClientService.java diff --git a/dolphinscheduler-api/pom.xml b/dolphinscheduler-api/pom.xml index c10f443384..11b23d9fb7 100644 --- a/dolphinscheduler-api/pom.xml +++ b/dolphinscheduler-api/pom.xml @@ -129,13 +129,13 @@ - com.github.xiaoymin - swagger-bootstrap-ui + org.apache.dolphinscheduler + dolphinscheduler-service - org.apache.dolphinscheduler - dolphinscheduler-rpc + com.github.xiaoymin + swagger-bootstrap-ui diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/log/LogClient.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/log/LogClient.java deleted file mode 100644 index 3452060ec9..0000000000 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/log/LogClient.java +++ /dev/null @@ -1,137 +0,0 @@ -/* - * 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.api.log; - -import io.grpc.ManagedChannel; -import io.grpc.ManagedChannelBuilder; -import io.grpc.StatusRuntimeException; -import org.apache.dolphinscheduler.rpc.*; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.concurrent.TimeUnit; - -/** - * log client - */ -public class LogClient { - - private static final Logger logger = LoggerFactory.getLogger(LogClient.class); - - private final ManagedChannel channel; - private final LogViewServiceGrpc.LogViewServiceBlockingStub blockingStub; - - /** - * construct client connecting to HelloWorld server at {@code host:port} - * - * @param host host - * @param port port - */ - public LogClient(String host, int port) { - this(ManagedChannelBuilder.forAddress(host, port) - // Channels are secure by default (via SSL/TLS). For the example we disable TLS to avoid - // needing certificates. - .usePlaintext(true)); - } - - /** - * construct client for accessing RouteGuide server using the existing channel - * - */ - LogClient(ManagedChannelBuilder channelBuilder) { - /** - * set max read size - */ - channelBuilder.maxInboundMessageSize(Integer.MAX_VALUE); - channel = channelBuilder.build(); - blockingStub = LogViewServiceGrpc.newBlockingStub(channel); - } - - /** - * shutdown - * - * @throws InterruptedException InterruptedException - */ - public void shutdown() throws InterruptedException { - channel.shutdown().awaitTermination(5, TimeUnit.SECONDS); - } - - /** - * roll view log - * - * @param path path - * @param skipLineNum skip line number - * @param limit limit - * @return log content - */ - public String rollViewLog(String path,int skipLineNum,int limit) { - logger.info("roll view log : path {},skipLineNum {} ,limit {}", path, skipLineNum, limit); - LogParameter pathParameter = LogParameter - .newBuilder() - .setPath(path) - .setSkipLineNum(skipLineNum) - .setLimit(limit) - .build(); - RetStrInfo retStrInfo; - try { - retStrInfo = blockingStub.rollViewLog(pathParameter); - return retStrInfo.getMsg(); - } catch (StatusRuntimeException e) { - logger.error("roll view log error", e); - return null; - } - } - - /** - * view log - * - * @param path path - * @return log content - */ - public String viewLog(String path) { - logger.info("view log path {}",path); - PathParameter pathParameter = PathParameter.newBuilder().setPath(path).build(); - RetStrInfo retStrInfo; - try { - retStrInfo = blockingStub.viewLog(pathParameter); - return retStrInfo.getMsg(); - } catch (StatusRuntimeException e) { - logger.error("view log error", e); - return null; - } - } - - /** - * get log size - * - * @param path log path - * @return log content bytes - */ - public byte[] getLogBytes(String path) { - logger.info("log path {}",path); - PathParameter pathParameter = PathParameter.newBuilder().setPath(path).build(); - RetByteInfo retByteInfo; - try { - retByteInfo = blockingStub.getLogBytes(pathParameter); - return retByteInfo.getData().toByteArray(); - } catch (StatusRuntimeException e) { - logger.error("log size error", e); - return null; - } - } - -} \ No newline at end of file diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/LoggerService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/LoggerService.java index 61dc1a7193..108d5d4853 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/LoggerService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/LoggerService.java @@ -17,12 +17,12 @@ package org.apache.dolphinscheduler.api.service; import org.apache.dolphinscheduler.api.enums.Status; -import org.apache.dolphinscheduler.api.log.LogClient; import org.apache.dolphinscheduler.api.utils.Result; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.utils.StringUtils; import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.TaskInstance; +import org.apache.dolphinscheduler.service.log.LogClientService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; @@ -65,7 +65,7 @@ public class LoggerService { logger.info("log host : {} , logPath : {} , logServer port : {}",host,taskInstance.getLogPath(),Constants.RPC_PORT); - LogClient logClient = new LogClient(host, Constants.RPC_PORT); + LogClientService logClient = new LogClientService(host, Constants.RPC_PORT); String log = logClient.rollViewLog(taskInstance.getLogPath(),skipLineNum,limit); result.setData(log); logger.info(log); @@ -85,7 +85,7 @@ public class LoggerService { throw new RuntimeException("task instance is null"); } String host = taskInstance.getHost(); - LogClient logClient = new LogClient(host, Constants.RPC_PORT); + LogClientService logClient = new LogClientService(host, Constants.RPC_PORT); return logClient.getLogBytes(taskInstance.getLogPath()); } } diff --git a/dolphinscheduler-remote/pom.xml b/dolphinscheduler-remote/pom.xml new file mode 100644 index 0000000000..b67b033ffa --- /dev/null +++ b/dolphinscheduler-remote/pom.xml @@ -0,0 +1,44 @@ + + + + + dolphinscheduler + org.apache.dolphinscheduler + 1.2.1-SNAPSHOT + + 4.0.0 + + dolphinscheduler-remote + + dolphinscheduler-remote + + http://www.example.com + + + UTF-8 + 1.7 + 1.7 + + + + + io.netty + netty-all + + + org.slf4j + slf4j-api + + + com.alibaba + fastjson + + + junit + junit + test + + + + diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java new file mode 100644 index 0000000000..678fe84f90 --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java @@ -0,0 +1,197 @@ +/* + * 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; + +import io.netty.bootstrap.Bootstrap; +import io.netty.channel.*; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.SocketChannel; +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.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.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.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * remoting netty client + */ +public class NettyRemotingClient { + + private final Logger logger = LoggerFactory.getLogger(NettyRemotingClient.class); + + private final Bootstrap bootstrap = new Bootstrap(); + + private final NettyEncoder encoder = new NettyEncoder(); + + private final ConcurrentHashMap channels = new ConcurrentHashMap(); + + private final ExecutorService defaultExecutor = Executors.newFixedThreadPool(Constants.CPUS); + + private final AtomicBoolean isStarted = new AtomicBoolean(false); + + private final NioEventLoopGroup workerGroup; + + private final NettyClientHandler clientHandler = new NettyClientHandler(this); + + private final NettyClientConfig clientConfig; + + public NettyRemotingClient(final NettyClientConfig clientConfig){ + this.clientConfig = clientConfig; + this.workerGroup = new NioEventLoopGroup(clientConfig.getWorkerThreads(), new ThreadFactory() { + private AtomicInteger threadIndex = new AtomicInteger(0); + + @Override + public Thread newThread(Runnable r) { + return new Thread(r, String.format("NettyClient_%d", this.threadIndex.incrementAndGet())); + } + }); + this.start(); + } + + private void start(){ + + this.bootstrap + .group(this.workerGroup) + .channel(NioSocketChannel.class) + .option(ChannelOption.SO_KEEPALIVE, clientConfig.isSoKeepalive()) + .option(ChannelOption.TCP_NODELAY, clientConfig.isTcpNoDelay()) + .option(ChannelOption.SO_SNDBUF, clientConfig.getSendBufferSize()) + .option(ChannelOption.SO_RCVBUF, clientConfig.getReceiveBufferSize()) + .handler(new ChannelInitializer() { + @Override + public void initChannel(SocketChannel ch) throws Exception { + ch.pipeline().addLast( + new NettyDecoder(), + clientHandler, + encoder); + } + }); + // + isStarted.compareAndSet(false, true); + } + + public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor) { + registerProcessor(commandType, processor, null); + } + + public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor, final ExecutorService executor) { + this.clientHandler.registerProcessor(commandType, processor, executor); + } + + public void send(final Address address, final Command command) throws RemotingException { + final Channel channel = getChannel(address); + if (channel == null) { + throw new RemotingException("network error"); + } + try { + channel.writeAndFlush(command).addListener(new ChannelFutureListener(){ + + @Override + public void operationComplete(ChannelFuture future) throws Exception { + if(future.isSuccess()){ + logger.info("sent command {} to {}", command, address); + } else{ + logger.error("send command {} to {} failed, error {}", command, address, future.cause()); + } + } + }); + } catch (Exception ex) { + String msg = String.format("send command %s to address %s encounter error", command, address); + throw new RemotingException(msg, ex); + } + } + + public Channel getChannel(Address address) { + Channel channel = channels.get(address); + if(channel != null && channel.isActive()){ + return channel; + } + return createChannel(address, true); + } + + public Channel createChannel(Address address, boolean isSync) { + ChannelFuture future; + try { + synchronized (bootstrap){ + future = bootstrap.connect(new InetSocketAddress(address.getHost(), address.getPort())); + } + if(isSync){ + future.sync(); + } + if (future.isSuccess()) { + Channel channel = future.channel(); + channels.put(address, channel); + return channel; + } + } catch (Exception ex) { + logger.info("connect to {} error {}", address, ex); + } + return null; + } + + public ExecutorService getDefaultExecutor() { + return defaultExecutor; + } + + public void close() { + if(isStarted.compareAndSet(true, false)){ + try { + closeChannels(); + if(workerGroup != null){ + this.workerGroup.shutdownGracefully(); + } + if(defaultExecutor != null){ + defaultExecutor.shutdown(); + } + } catch (Exception ex) { + logger.error("netty client close exception", ex); + } + logger.info("netty client closed"); + } + } + + private void closeChannels(){ + for (Channel channel : this.channels.values()) { + channel.close(); + } + this.channels.clear(); + } + + public void removeChannel(Address address){ + Channel channel = this.channels.remove(address); + if(channel != null){ + channel.close(); + } + } +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingServer.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingServer.java new file mode 100644 index 0000000000..5823dbb088 --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingServer.java @@ -0,0 +1,165 @@ +/* + * 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; + +import io.netty.bootstrap.ServerBootstrap; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.ChannelOption; +import io.netty.channel.ChannelPipeline; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.nio.NioServerSocketChannel; +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.CommandType; +import org.apache.dolphinscheduler.remote.config.NettyServerConfig; +import org.apache.dolphinscheduler.remote.handler.NettyServerHandler; +import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; +import org.apache.dolphinscheduler.remote.utils.Constants; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * remoting netty server + */ +public class NettyRemotingServer { + + private final Logger logger = LoggerFactory.getLogger(NettyRemotingServer.class); + + private final ServerBootstrap serverBootstrap = new ServerBootstrap(); + + private final NettyEncoder encoder = new NettyEncoder(); + + private final ExecutorService defaultExecutor = Executors.newFixedThreadPool(Constants.CPUS); + + private final NioEventLoopGroup bossGroup; + + private final NioEventLoopGroup workGroup; + + private final NettyServerConfig serverConfig; + + private final NettyServerHandler serverHandler = new NettyServerHandler(this); + + private final AtomicBoolean isStarted = new AtomicBoolean(false); + + public NettyRemotingServer(final NettyServerConfig serverConfig){ + this.serverConfig = serverConfig; + + this.bossGroup = new NioEventLoopGroup(1, new ThreadFactory() { + private AtomicInteger threadIndex = new AtomicInteger(0); + + public Thread newThread(Runnable r) { + return new Thread(r, String.format("NettyServerBossThread_%d", this.threadIndex.incrementAndGet())); + } + }); + + this.workGroup = new NioEventLoopGroup(serverConfig.getWorkerThread(), new ThreadFactory() { + private AtomicInteger threadIndex = new AtomicInteger(0); + + public Thread newThread(Runnable r) { + return new Thread(r, String.format("NettyServerWorkerThread_%d", this.threadIndex.incrementAndGet())); + } + }); + } + + public void start(){ + + if(this.isStarted.get()){ + return; + } + + this.serverBootstrap + .group(this.bossGroup, this.workGroup) + .channel(NioServerSocketChannel.class) + .option(ChannelOption.SO_REUSEADDR, true) + .option(ChannelOption.SO_BACKLOG, serverConfig.getSoBacklog()) + .childOption(ChannelOption.SO_KEEPALIVE, serverConfig.isSoKeepalive()) + .childOption(ChannelOption.TCP_NODELAY, serverConfig.isTcpNoDelay()) + .childOption(ChannelOption.SO_SNDBUF, serverConfig.getSendBufferSize()) + .childOption(ChannelOption.SO_RCVBUF, serverConfig.getReceiveBufferSize()) + .childHandler(new ChannelInitializer() { + + @Override + protected void initChannel(NioSocketChannel ch) throws Exception { + initNettyChannel(ch); + } + }); + + ChannelFuture future; + try { + future = serverBootstrap.bind(serverConfig.getListenPort()).sync(); + } catch (Exception e) { + logger.error("NettyRemotingServer bind fail {}, exit", e); + throw new RuntimeException(String.format("NettyRemotingServer bind %s fail", serverConfig.getListenPort())); + } + if (future.isSuccess()) { + logger.info("NettyRemotingServer bind success at port : {}", serverConfig.getListenPort()); + } else if (future.cause() != null) { + throw new RuntimeException(String.format("NettyRemotingServer bind %s fail", serverConfig.getListenPort()), future.cause()); + } else { + throw new RuntimeException(String.format("NettyRemotingServer bind %s fail", serverConfig.getListenPort())); + } + // + isStarted.compareAndSet(false, true); + } + + private void initNettyChannel(NioSocketChannel ch) throws Exception{ + ChannelPipeline pipeline = ch.pipeline(); + pipeline.addLast("encoder", encoder); + pipeline.addLast("decoder", new NettyDecoder()); + pipeline.addLast("handler", serverHandler); + } + + public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor) { + this.registerProcessor(commandType, processor, null); + } + + public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor, final ExecutorService executor) { + this.serverHandler.registerProcessor(commandType, processor, executor); + } + + public ExecutorService getDefaultExecutor() { + return defaultExecutor; + } + + public void close() { + if(isStarted.compareAndSet(true, false)){ + try { + if(bossGroup != null){ + this.bossGroup.shutdownGracefully(); + } + if(workGroup != null){ + this.workGroup.shutdownGracefully(); + } + if(defaultExecutor != null){ + defaultExecutor.shutdown(); + } + } catch (Exception ex) { + logger.error("netty server close exception", ex); + } + logger.info("netty server closed"); + } + } +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/codec/NettyDecoder.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/codec/NettyDecoder.java new file mode 100644 index 0000000000..998f4ee7d5 --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/codec/NettyDecoder.java @@ -0,0 +1,92 @@ +/* + * 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.codec; + + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandlerContext; +import io.netty.handler.codec.ReplayingDecoder; +import org.apache.dolphinscheduler.remote.command.Command; +import org.apache.dolphinscheduler.remote.command.CommandHeader; +import org.apache.dolphinscheduler.remote.command.CommandType; + +import java.util.List; + +/** + * netty decoder + */ +public class NettyDecoder extends ReplayingDecoder { + + public NettyDecoder(){ + super(State.MAGIC); + } + + private final CommandHeader commandHeader = new CommandHeader(); + + @Override + protected void decode(ChannelHandlerContext ctx, ByteBuf in, List out) throws Exception { + switch (state()){ + case MAGIC: + checkMagic(in.readByte()); + checkpoint(State.COMMAND); + case COMMAND: + commandHeader.setType(in.readByte()); + checkpoint(State.OPAQUE); + case OPAQUE: + commandHeader.setOpaque(in.readLong()); + checkpoint(State.BODY_LENGTH); + case BODY_LENGTH: + commandHeader.setBodyLength(in.readInt()); + checkpoint(State.BODY); + case BODY: + byte[] body = new byte[commandHeader.getBodyLength()]; + in.readBytes(body); + // + Command packet = new Command(); + packet.setType(commandType(commandHeader.getType())); + packet.setOpaque(commandHeader.getOpaque()); + packet.setBody(body); + out.add(packet); + // + checkpoint(State.MAGIC); + } + } + + private CommandType commandType(byte type){ + for(CommandType ct : CommandType.values()){ + if(ct.ordinal() == type){ + return ct; + } + } + return null; + } + + private void checkMagic(byte magic) { + if (magic != Command.MAGIC) { + throw new IllegalArgumentException("illegal packet [magic]" + magic); + } + } + + enum State{ + MAGIC, + COMMAND, + OPAQUE, + BODY_LENGTH, + BODY; + } +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/codec/NettyEncoder.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/codec/NettyEncoder.java new file mode 100644 index 0000000000..dd4e523a14 --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/codec/NettyEncoder.java @@ -0,0 +1,44 @@ +/* + * 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.codec; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandler.Sharable; +import io.netty.channel.ChannelHandlerContext; +import io.netty.handler.codec.MessageToByteEncoder; +import org.apache.dolphinscheduler.remote.command.Command; + +/** + * netty encoder + */ +@Sharable +public class NettyEncoder extends MessageToByteEncoder { + + @Override + protected void encode(ChannelHandlerContext ctx, Command msg, ByteBuf out) throws Exception { + if(msg == null){ + throw new Exception("encode msg is null"); + } + out.writeByte(Command.MAGIC); + out.writeByte(msg.getType().ordinal()); + out.writeLong(msg.getOpaque()); + out.writeInt(msg.getBody().length); + out.writeBytes(msg.getBody()); + } + +} + diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Command.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Command.java new file mode 100644 index 0000000000..4687db39c0 --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Command.java @@ -0,0 +1,102 @@ +/* + * 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 java.io.Serializable; + +/** + * receive task log request command and content fill + * for netty data serializable transfer + */ +public class Command implements Serializable { + + private static final long serialVersionUID = 1L; + + public static final byte MAGIC = (byte) 0xbabe; + + public Command(){ + } + + public Command(long opaque){ + this.opaque = opaque; + } + + /** + * comman type + */ + private CommandType type; + + /** + * request unique identification + */ + private long opaque; + + private byte[] body; + + public CommandType getType() { + return type; + } + + public void setType(CommandType type) { + this.type = type; + } + + public long getOpaque() { + return opaque; + } + + public void setOpaque(long opaque) { + this.opaque = opaque; + } + + public byte[] getBody() { + return body; + } + + public void setBody(byte[] body) { + this.body = body; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + (int) (opaque ^ (opaque >>> 32)); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + Command other = (Command) obj; + return opaque == other.opaque; + } + + @Override + public String toString() { + return "Command [type=" + type + ", opaque=" + opaque + ", bodyLen=" + (body == null ? 0 : body.length) + "]"; + } + +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandHeader.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandHeader.java new file mode 100644 index 0000000000..92f7ac35dc --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandHeader.java @@ -0,0 +1,55 @@ +/* + * 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 java.io.Serializable; + +/** + * command header + */ +public class CommandHeader implements Serializable { + + private byte type; + + private long opaque; + + private int bodyLength; + + public int getBodyLength() { + return bodyLength; + } + + public void setBodyLength(int bodyLength) { + this.bodyLength = bodyLength; + } + + public byte getType() { + return type; + } + + public void setType(byte type) { + this.type = type; + } + + public long getOpaque() { + return opaque; + } + + public void setOpaque(long opaque) { + this.opaque = opaque; + } +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandType.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandType.java new file mode 100644 index 0000000000..185358a73c --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandType.java @@ -0,0 +1 @@ +/* * 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; public enum CommandType { /** * roll view log request */ ROLL_VIEW_LOG_REQUEST, /** * roll view log response */ ROLL_VIEW_LOG_RESPONSE, /** * view whole log request */ VIEW_WHOLE_LOG_REQUEST, /** * view whole log response */ VIEW_WHOLE_LOG_RESPONSE, /** * get log bytes request */ GET_LOG_BYTES_REQUEST, /** * get log bytes response */ GET_LOG_BYTES_RESPONSE, WORKER_REQUEST, MASTER_RESPONSE, /** * execute task request */ EXECUTE_TASK_REQUEST, /** * execute task response */ EXECUTE_TASK_RESPONSE, PING, PONG; } \ No newline at end of file diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskRequestCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskRequestCommand.java new file mode 100644 index 0000000000..e75c2de379 --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskRequestCommand.java @@ -0,0 +1 @@ +/* * 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; public class ExecuteTaskRequestCommand implements Serializable { private static final AtomicLong REQUEST = new AtomicLong(1); private String taskId; private String attemptId; private String applicationName; private String groupName; private String taskName; private int connectorPort; private String description; private String className; private String methodName; private String params; private List shardItems; public List getShardItems() { return shardItems; } public void setShardItems(List 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; } 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 diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskResponseCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskResponseCommand.java new file mode 100644 index 0000000000..fafb57535b --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskResponseCommand.java @@ -0,0 +1 @@ +/* * 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.concurrent.atomic.AtomicLong; public class ExecuteTaskResponseCommand implements Serializable { private static final AtomicLong REQUEST = new AtomicLong(1); private String taskId; private String attemptId; private Object result; private long receivedTime; private int executeCount; private long executeTime; public String getAttemptId() { return attemptId; } public void setAttemptId(String attemptId) { this.attemptId = attemptId; } public String getTaskId() { return taskId; } public void setTaskId(String taskId) { this.taskId = taskId; } public Object getResult() { return result; } public void setResult(Object result) { this.result = result; } public long getReceivedTime() { return receivedTime; } public void setReceivedTime(long receivedTime) { this.receivedTime = receivedTime; } public int getExecuteCount() { return executeCount; } public void setExecuteCount(int executeCount) { this.executeCount = executeCount; } public long getExecuteTime() { return executeTime; } public void setExecuteTime(long executeTime) { this.executeTime = executeTime; } public Command convert2Command(){ Command command = new Command(REQUEST.getAndIncrement()); command.setType(CommandType.EXECUTE_TASK_RESPONSE); byte[] body = FastJsonSerializer.serialize(this); command.setBody(body); return command; } } \ No newline at end of file diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Ping.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Ping.java new file mode 100644 index 0000000000..365d451564 --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Ping.java @@ -0,0 +1,57 @@ +/* + * 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 io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; + +import java.io.Serializable; +import java.util.concurrent.atomic.AtomicLong; + + +public class Ping implements Serializable { + + private static final AtomicLong ID = new AtomicLong(1); + + protected static ByteBuf EMPTY_BODY = Unpooled.EMPTY_BUFFER; + + private static byte[] EMPTY_BODY_ARRAY = new byte[0]; + + private static final ByteBuf PING_BUF; + + static { + ByteBuf ping = Unpooled.buffer(); + ping.writeByte(Command.MAGIC); + ping.writeByte(CommandType.PING.ordinal()); + ping.writeLong(0); + ping.writeInt(0); + ping.writeBytes(EMPTY_BODY); + PING_BUF = Unpooled.unreleasableBuffer(ping).asReadOnly(); + } + + public static ByteBuf pingContent(){ + return PING_BUF.duplicate(); + } + + public static Command create(){ + Command command = new Command(ID.getAndIncrement()); + command.setType(CommandType.PING); + command.setBody(EMPTY_BODY_ARRAY); + return command; + } +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Pong.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Pong.java new file mode 100644 index 0000000000..bc5abdad79 --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Pong.java @@ -0,0 +1,54 @@ +/* + * 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 io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; + +import java.io.Serializable; + + +public class Pong implements Serializable { + + protected static ByteBuf EMPTY_BODY = Unpooled.EMPTY_BUFFER; + + private static byte[] EMPTY_BODY_ARRAY = new byte[0]; + + private static final ByteBuf PONG_BUF; + + static { + ByteBuf ping = Unpooled.buffer(); + ping.writeByte(Command.MAGIC); + ping.writeByte(CommandType.PONG.ordinal()); + ping.writeLong(0); + ping.writeInt(0); + ping.writeBytes(EMPTY_BODY); + PONG_BUF = Unpooled.unreleasableBuffer(ping).asReadOnly(); + } + + public static ByteBuf pingContent(){ + return PONG_BUF.duplicate(); + } + + public static Command create(long opaque){ + Command command = new Command(opaque); + command.setType(CommandType.PONG); + command.setBody(EMPTY_BODY_ARRAY); + return command; + } +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesRequestCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesRequestCommand.java new file mode 100644 index 0000000000..1a2e6e4dd1 --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesRequestCommand.java @@ -0,0 +1,62 @@ +/* + * 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.log; + +import org.apache.dolphinscheduler.remote.command.Command; +import org.apache.dolphinscheduler.remote.command.CommandType; +import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer; + +import java.io.Serializable; +import java.util.concurrent.atomic.AtomicLong; + +/** + * get log bytes request command + */ +public class GetLogBytesRequestCommand implements Serializable { + + private static final AtomicLong REQUEST = new AtomicLong(1); + + private String path; + + public GetLogBytesRequestCommand() { + } + + public GetLogBytesRequestCommand(String path) { + this.path = path; + } + + public String getPath() { + return path; + } + + public void setPath(String path) { + this.path = path; + } + + /** + * + * @return + */ + public Command convert2Command(){ + Command command = new Command(REQUEST.getAndIncrement()); + command.setType(CommandType.GET_LOG_BYTES_REQUEST); + byte[] body = FastJsonSerializer.serialize(this); + command.setBody(body); + return command; + } +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesResponseCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesResponseCommand.java new file mode 100644 index 0000000000..05692fb5c9 --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesResponseCommand.java @@ -0,0 +1,56 @@ +/* + * 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.log; + +import org.apache.dolphinscheduler.remote.command.Command; +import org.apache.dolphinscheduler.remote.command.CommandType; +import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer; + +import java.io.Serializable; + +/** + * get log bytes response command + */ +public class GetLogBytesResponseCommand implements Serializable { + + private byte[] data; + + public GetLogBytesResponseCommand() { + } + + public GetLogBytesResponseCommand(byte[] data) { + this.data = data; + } + + public byte[] getData() { + return data; + } + + public void setData(byte[] data) { + this.data = data; + } + + public Command convert2Command(long opaque){ + Command command = new Command(opaque); + command.setType(CommandType.GET_LOG_BYTES_RESPONSE); + byte[] body = FastJsonSerializer.serialize(this); + command.setBody(body); + return command; + } + +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogRequestCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogRequestCommand.java new file mode 100644 index 0000000000..49d19aa1f2 --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogRequestCommand.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.dolphinscheduler.remote.command.log; + +import org.apache.dolphinscheduler.remote.command.Command; +import org.apache.dolphinscheduler.remote.command.CommandType; +import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer; + +import java.io.Serializable; +import java.util.concurrent.atomic.AtomicLong; + +/** + * roll view log request command + */ +public class RollViewLogRequestCommand implements Serializable { + + private static final AtomicLong REQUEST = new AtomicLong(1); + + private String path; + + private int skipLineNum; + + private int limit; + + public RollViewLogRequestCommand() { + } + + public RollViewLogRequestCommand(String path, int skipLineNum, int limit) { + this.path = path; + this.skipLineNum = skipLineNum; + this.limit = limit; + } + + public String getPath() { + return path; + } + + public void setPath(String path) { + this.path = path; + } + + public int getSkipLineNum() { + return skipLineNum; + } + + public void setSkipLineNum(int skipLineNum) { + this.skipLineNum = skipLineNum; + } + + public int getLimit() { + return limit; + } + + public void setLimit(int limit) { + this.limit = limit; + } + + public Command convert2Command(){ + Command command = new Command(REQUEST.getAndIncrement()); + command.setType(CommandType.ROLL_VIEW_LOG_REQUEST); + byte[] body = FastJsonSerializer.serialize(this); + command.setBody(body); + return command; + } +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogResponseCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogResponseCommand.java new file mode 100644 index 0000000000..def3257073 --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogResponseCommand.java @@ -0,0 +1,55 @@ +/* + * 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.log; + +import org.apache.dolphinscheduler.remote.command.Command; +import org.apache.dolphinscheduler.remote.command.CommandType; +import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer; + +import java.io.Serializable; + +/** + * roll view log response command + */ +public class RollViewLogResponseCommand implements Serializable { + + private String msg; + + public RollViewLogResponseCommand() { + } + + public RollViewLogResponseCommand(String msg) { + this.msg = msg; + } + + public String getMsg() { + return msg; + } + + public void setMsg(String msg) { + this.msg = msg; + } + + public Command convert2Command(long opaque){ + Command command = new Command(opaque); + command.setType(CommandType.ROLL_VIEW_LOG_RESPONSE); + byte[] body = FastJsonSerializer.serialize(this); + command.setBody(body); + return command; + } +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogRequestCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogRequestCommand.java new file mode 100644 index 0000000000..9ba9cd3c23 --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogRequestCommand.java @@ -0,0 +1,58 @@ +/* + * 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.log; + +import org.apache.dolphinscheduler.remote.command.Command; +import org.apache.dolphinscheduler.remote.command.CommandType; +import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer; + +import java.io.Serializable; +import java.util.concurrent.atomic.AtomicLong; + +/** + * view log request command + */ +public class ViewLogRequestCommand implements Serializable { + + private static final AtomicLong REQUEST = new AtomicLong(1); + + private String path; + + public ViewLogRequestCommand() { + } + + public ViewLogRequestCommand(String path) { + this.path = path; + } + + public String getPath() { + return path; + } + + public void setPath(String path) { + this.path = path; + } + + public Command convert2Command(){ + Command command = new Command(REQUEST.getAndIncrement()); + command.setType(CommandType.VIEW_WHOLE_LOG_REQUEST); + byte[] body = FastJsonSerializer.serialize(this); + command.setBody(body); + return command; + } +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogResponseCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogResponseCommand.java new file mode 100644 index 0000000000..6e3c799a3d --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogResponseCommand.java @@ -0,0 +1,55 @@ +/* + * 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.log; + +import org.apache.dolphinscheduler.remote.command.Command; +import org.apache.dolphinscheduler.remote.command.CommandType; +import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer; + +import java.io.Serializable; + +/** + * view log response command + */ +public class ViewLogResponseCommand implements Serializable { + + private String msg; + + public ViewLogResponseCommand() { + } + + public ViewLogResponseCommand(String msg) { + this.msg = msg; + } + + public String getMsg() { + return msg; + } + + public void setMsg(String msg) { + this.msg = msg; + } + + public Command convert2Command(long opaque){ + Command command = new Command(opaque); + command.setType(CommandType.VIEW_WHOLE_LOG_RESPONSE); + byte[] body = FastJsonSerializer.serialize(this); + command.setBody(body); + return command; + } +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/config/NettyClientConfig.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/config/NettyClientConfig.java new file mode 100644 index 0000000000..56d2643a67 --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/config/NettyClientConfig.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.remote.config; + +import org.apache.dolphinscheduler.remote.utils.Constants; + +/** + * netty client config + */ +public class NettyClientConfig { + + private int workerThreads = Constants.CPUS; + + private boolean tcpNoDelay = true; + + private boolean soKeepalive = true; + + private int sendBufferSize = 65535; + + private int receiveBufferSize = 65535; + + public int getWorkerThreads() { + return workerThreads; + } + + public void setWorkerThreads(int workerThreads) { + this.workerThreads = workerThreads; + } + + public boolean isTcpNoDelay() { + return tcpNoDelay; + } + + public void setTcpNoDelay(boolean tcpNoDelay) { + this.tcpNoDelay = tcpNoDelay; + } + + public boolean isSoKeepalive() { + return soKeepalive; + } + + public void setSoKeepalive(boolean soKeepalive) { + this.soKeepalive = soKeepalive; + } + + public int getSendBufferSize() { + return sendBufferSize; + } + + public void setSendBufferSize(int sendBufferSize) { + this.sendBufferSize = sendBufferSize; + } + + public int getReceiveBufferSize() { + return receiveBufferSize; + } + + public void setReceiveBufferSize(int receiveBufferSize) { + this.receiveBufferSize = receiveBufferSize; + } + +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/config/NettyServerConfig.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/config/NettyServerConfig.java new file mode 100644 index 0000000000..847f316089 --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/config/NettyServerConfig.java @@ -0,0 +1,95 @@ +/* + * 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.config; + +import org.apache.dolphinscheduler.remote.utils.Constants; + +/** + * netty server config + */ +public class NettyServerConfig { + + private int soBacklog = 1024; + + private boolean tcpNoDelay = true; + + private boolean soKeepalive = true; + + private int sendBufferSize = 65535; + + private int receiveBufferSize = 65535; + + private int workerThread = Constants.CPUS; + + private int listenPort = 12346; + + public int getListenPort() { + return listenPort; + } + + public void setListenPort(int listenPort) { + this.listenPort = listenPort; + } + + public int getSoBacklog() { + return soBacklog; + } + + public void setSoBacklog(int soBacklog) { + this.soBacklog = soBacklog; + } + + public boolean isTcpNoDelay() { + return tcpNoDelay; + } + + public void setTcpNoDelay(boolean tcpNoDelay) { + this.tcpNoDelay = tcpNoDelay; + } + + public boolean isSoKeepalive() { + return soKeepalive; + } + + public void setSoKeepalive(boolean soKeepalive) { + this.soKeepalive = soKeepalive; + } + + public int getSendBufferSize() { + return sendBufferSize; + } + + public void setSendBufferSize(int sendBufferSize) { + this.sendBufferSize = sendBufferSize; + } + + public int getReceiveBufferSize() { + return receiveBufferSize; + } + + public void setReceiveBufferSize(int receiveBufferSize) { + this.receiveBufferSize = receiveBufferSize; + } + + public int getWorkerThread() { + return workerThread; + } + + public void setWorkerThread(int workerThread) { + this.workerThread = workerThread; + } +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingException.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingException.java new file mode 100644 index 0000000000..29d48db8f8 --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingException.java @@ -0,0 +1,94 @@ +/* + * 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.exceptions; + +/** + * remote exception + */ +public class RemotingException extends Exception { + + public RemotingException() { + super(); + } + + /** Constructs a new runtime exception with the specified detail message. + * The cause is not initialized, and may subsequently be initialized by a + * call to {@link #initCause}. + * + * @param message the detail message. The detail message is saved for + * later retrieval by the {@link #getMessage()} method. + */ + public RemotingException(String message) { + super(message); + } + + /** + * Constructs a new runtime exception with the specified detail message and + * cause.

Note that the detail message associated with + * {@code cause} is not automatically incorporated in + * this runtime exception's detail message. + * + * @param message the detail message (which is saved for later retrieval + * by the {@link #getMessage()} method). + * @param cause the cause (which is saved for later retrieval by the + * {@link #getCause()} method). (A null value is + * permitted, and indicates that the cause is nonexistent or + * unknown.) + * @since 1.4 + */ + public RemotingException(String message, Throwable cause) { + super(message, cause); + } + + /** Constructs a new runtime exception with the specified cause and a + * detail message of (cause==null ? null : cause.toString()) + * (which typically contains the class and detail message of + * cause). This constructor is useful for runtime exceptions + * that are little more than wrappers for other throwables. + * + * @param cause the cause (which is saved for later retrieval by the + * {@link #getCause()} method). (A null value is + * permitted, and indicates that the cause is nonexistent or + * unknown.) + * @since 1.4 + */ + public RemotingException(Throwable cause) { + super(cause); + } + + /** + * Constructs a new runtime exception with the specified detail + * message, cause, suppression enabled or disabled, and writable + * stack trace enabled or disabled. + * + * @param message the detail message. + * @param cause the cause. (A {@code null} value is permitted, + * and indicates that the cause is nonexistent or unknown.) + * @param enableSuppression whether or not suppression is enabled + * or disabled + * @param writableStackTrace whether or not the stack trace should + * be writable + * + * @since 1.7 + */ + protected RemotingException(String message, Throwable cause, + boolean enableSuppression, + boolean writableStackTrace) { + super(message, cause, enableSuppression, writableStackTrace); + } +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java new file mode 100644 index 0000000000..b06308090f --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java @@ -0,0 +1,123 @@ +/* + * 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.handler; + +import io.netty.channel.*; +import org.apache.dolphinscheduler.remote.NettyRemotingClient; +import org.apache.dolphinscheduler.remote.command.Command; +import org.apache.dolphinscheduler.remote.command.CommandType; +import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; +import org.apache.dolphinscheduler.remote.utils.ChannelUtils; +import org.apache.dolphinscheduler.remote.utils.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.RejectedExecutionException; + +/** + * netty client request handler + */ +@ChannelHandler.Sharable +public class NettyClientHandler extends ChannelInboundHandlerAdapter { + + private final Logger logger = LoggerFactory.getLogger(NettyClientHandler.class); + + private final NettyRemotingClient nettyRemotingClient; + + private final ConcurrentHashMap> processors = new ConcurrentHashMap(); + + public NettyClientHandler(NettyRemotingClient nettyRemotingClient){ + this.nettyRemotingClient = nettyRemotingClient; + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + nettyRemotingClient.removeChannel(ChannelUtils.toAddress(ctx.channel())); + ctx.channel().close(); + } + + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + processReceived(ctx.channel(), (Command)msg); + } + + public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor) { + this.registerProcessor(commandType, processor, nettyRemotingClient.getDefaultExecutor()); + } + + public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor, final ExecutorService executor) { + ExecutorService executorRef = executor; + if(executorRef == null){ + executorRef = nettyRemotingClient.getDefaultExecutor(); + } + this.processors.putIfAbsent(commandType, new Pair(processor, executorRef)); + } + + private void processReceived(final Channel channel, final Command msg) { + final CommandType commandType = msg.getType(); + final Pair pair = processors.get(commandType); + if (pair != null) { + Runnable r = new Runnable() { + @Override + public void run() { + try { + pair.getLeft().process(channel, msg); + } catch (Throwable ex) { + logger.error("process msg {} error : {}", msg, ex); + } + } + }; + try { + pair.getRight().submit(r); + } catch (RejectedExecutionException e) { + logger.warn("thread pool is full, discard msg {} from {}", msg, ChannelUtils.getRemoteAddress(channel)); + } + } else { + logger.warn("commandType {} not support", commandType); + } + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { + logger.error("exceptionCaught : {}", cause); + nettyRemotingClient.removeChannel(ChannelUtils.toAddress(ctx.channel())); + ctx.channel().close(); + } + + @Override + public void channelWritabilityChanged(ChannelHandlerContext ctx) throws Exception { + Channel ch = ctx.channel(); + ChannelConfig config = ch.config(); + + if (!ch.isWritable()) { + if (logger.isWarnEnabled()) { + logger.warn("{} is not writable, over high water level : {}", + new Object[]{ch, config.getWriteBufferHighWaterMark()}); + } + + config.setAutoRead(false); + } else { + if (logger.isWarnEnabled()) { + logger.warn("{} is writable, to low water : {}", + new Object[]{ch, config.getWriteBufferLowWaterMark()}); + } + config.setAutoRead(true); + } + } +} \ No newline at end of file diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyServerHandler.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyServerHandler.java new file mode 100644 index 0000000000..8a7ee39a77 --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyServerHandler.java @@ -0,0 +1,121 @@ +/* + * 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.handler; + +import io.netty.channel.*; +import org.apache.dolphinscheduler.remote.NettyRemotingServer; +import org.apache.dolphinscheduler.remote.command.Command; +import org.apache.dolphinscheduler.remote.command.CommandType; +import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; +import org.apache.dolphinscheduler.remote.utils.ChannelUtils; +import org.apache.dolphinscheduler.remote.utils.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.RejectedExecutionException; + +/** + * netty server request handler + */ +@ChannelHandler.Sharable +public class NettyServerHandler extends ChannelInboundHandlerAdapter { + + private final Logger logger = LoggerFactory.getLogger(NettyServerHandler.class); + + private final NettyRemotingServer nettyRemotingServer; + + private final ConcurrentHashMap> processors = new ConcurrentHashMap(); + + public NettyServerHandler(NettyRemotingServer nettyRemotingServer){ + this.nettyRemotingServer = nettyRemotingServer; + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + ctx.channel().close(); + } + + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + processReceived(ctx.channel(), (Command)msg); + } + + public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor) { + this.registerProcessor(commandType, processor, null); + } + + public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor, final ExecutorService executor) { + ExecutorService executorRef = executor; + if(executorRef == null){ + executorRef = nettyRemotingServer.getDefaultExecutor(); + } + this.processors.putIfAbsent(commandType, new Pair(processor, executorRef)); + } + + private void processReceived(final Channel channel, final Command msg) { + final CommandType commandType = msg.getType(); + final Pair pair = processors.get(commandType); + if (pair != null) { + Runnable r = new Runnable() { + public void run() { + try { + pair.getLeft().process(channel, msg); + } catch (Throwable ex) { + logger.error("process msg {} error : {}", msg, ex); + } + } + }; + try { + pair.getRight().submit(r); + } catch (RejectedExecutionException e) { + logger.warn("thread pool is full, discard msg {} from {}", msg, ChannelUtils.getRemoteAddress(channel)); + } + } else { + logger.warn("commandType {} not support", commandType); + } + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { + logger.error("exceptionCaught : {}", cause); + ctx.channel().close(); + } + + @Override + public void channelWritabilityChanged(ChannelHandlerContext ctx) throws Exception { + Channel ch = ctx.channel(); + ChannelConfig config = ch.config(); + + if (!ch.isWritable()) { + if (logger.isWarnEnabled()) { + logger.warn("{} is not writable, over high water level : {}", + new Object[]{ch, config.getWriteBufferHighWaterMark()}); + } + + config.setAutoRead(false); + } else { + if (logger.isWarnEnabled()) { + logger.warn("{} is writable, to low water : {}", + new Object[]{ch, config.getWriteBufferLowWaterMark()}); + } + config.setAutoRead(true); + } + } +} \ No newline at end of file diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/NettyRequestProcessor.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/NettyRequestProcessor.java new file mode 100644 index 0000000000..10a8195710 --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/NettyRequestProcessor.java @@ -0,0 +1,28 @@ +/* + * 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.processor; + +import io.netty.channel.Channel; +import org.apache.dolphinscheduler.remote.command.Command; + +/** + * netty request processor + */ +public interface NettyRequestProcessor { + + void process(final Channel channel, final Command command); +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Address.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Address.java new file mode 100644 index 0000000000..221b895cb9 --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Address.java @@ -0,0 +1,90 @@ +/* + * 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.utils; + +import java.io.Serializable; + +/** + * server address + */ +public class Address implements Serializable { + + private String host; + + private int port; + + public Address(){ + //NOP + } + + public Address(String host, int port){ + this.host = host; + this.port = port; + } + + public String getHost() { + return host; + } + + public void setHost(String host) { + this.host = host; + } + + public int getPort() { + return port; + } + + public void setPort(int port) { + this.port = port; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((host == null) ? 0 : host.hashCode()); + result = prime * result + port; + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + Address other = (Address) obj; + if (host == null) { + if (other.host != null) { + return false; + } + } else if (!host.equals(other.host)) { + return false; + } + return port == other.port; + } + + @Override + public String toString() { + return "Address [host=" + host + ", port=" + port + "]"; + } +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/ChannelUtils.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/ChannelUtils.java new file mode 100644 index 0000000000..e9d93da41d --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/ChannelUtils.java @@ -0,0 +1,41 @@ +/* + * 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.utils; + +import io.netty.channel.Channel; + +import java.net.InetSocketAddress; + +/** + * channel utils + */ +public class ChannelUtils { + + public static String getLocalAddress(Channel channel){ + return ((InetSocketAddress)channel.localAddress()).getAddress().getHostAddress(); + } + + public static String getRemoteAddress(Channel channel){ + return ((InetSocketAddress)channel.remoteAddress()).getAddress().getHostAddress(); + } + + public static Address toAddress(Channel channel){ + InetSocketAddress socketAddress = ((InetSocketAddress)channel.remoteAddress()); + return new Address(socketAddress.getAddress().getHostAddress(), socketAddress.getPort()); + } + +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Constants.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Constants.java new file mode 100644 index 0000000000..c0a930ca41 --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Constants.java @@ -0,0 +1,31 @@ +/* + * 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.utils; + +import java.nio.charset.Charset; + +public class Constants { + + public static final String COMMA = ","; + + public static final String SLASH = "/"; + + public static final Charset UTF8 = Charset.forName("UTF-8"); + + public static final int CPUS = Runtime.getRuntime().availableProcessors(); + +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/FastJsonSerializer.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/FastJsonSerializer.java new file mode 100644 index 0000000000..a9b85461ff --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/FastJsonSerializer.java @@ -0,0 +1,39 @@ +/* + * 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.utils; + +import com.alibaba.fastjson.JSON; + +/** + * json serialize or deserialize + */ +public class FastJsonSerializer { + + public static byte[] serialize(T obj) { + String json = JSON.toJSONString(obj); + return json.getBytes(Constants.UTF8); + } + + public static String serializeToString(T obj) { + return JSON.toJSONString(obj); + } + + public static T deserialize(byte[] src, Class clazz) { + return JSON.parseObject(new String(src, Constants.UTF8), clazz); + } + +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Pair.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Pair.java new file mode 100644 index 0000000000..a79a3748cd --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Pair.java @@ -0,0 +1,47 @@ +/* + * 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.utils; + + +public class Pair { + + private L left; + + private R right; + + public Pair(L left, R right) { + this.left = left; + this.right = right; + } + + public L getLeft() { + return left; + } + + public void setLeft(L left) { + this.left = left; + } + + public R getRight() { + return right; + } + + public void setRight(R right) { + this.right = right; + } +} diff --git a/dolphinscheduler-remote/src/test/java/org/apache/dolphinscheduler/remote/NettyRemotingClientTest.java b/dolphinscheduler-remote/src/test/java/org/apache/dolphinscheduler/remote/NettyRemotingClientTest.java new file mode 100644 index 0000000000..19fd564bd5 --- /dev/null +++ b/dolphinscheduler-remote/src/test/java/org/apache/dolphinscheduler/remote/NettyRemotingClientTest.java @@ -0,0 +1,71 @@ +package org.apache.dolphinscheduler.remote;/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import io.netty.channel.Channel; +import org.apache.dolphinscheduler.remote.NettyRemotingClient; +import org.apache.dolphinscheduler.remote.NettyRemotingServer; +import org.apache.dolphinscheduler.remote.command.Command; +import org.apache.dolphinscheduler.remote.command.CommandType; +import org.apache.dolphinscheduler.remote.command.Ping; +import org.apache.dolphinscheduler.remote.command.Pong; +import org.apache.dolphinscheduler.remote.config.NettyClientConfig; +import org.apache.dolphinscheduler.remote.config.NettyServerConfig; +import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; +import org.apache.dolphinscheduler.remote.utils.Address; +import org.junit.Assert; +import org.junit.Test; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicLong; + +public class NettyRemotingClientTest { + + + @Test + public void testSend(){ + NettyServerConfig serverConfig = new NettyServerConfig(); + + NettyRemotingServer server = new NettyRemotingServer(serverConfig); + server.registerProcessor(CommandType.PING, new NettyRequestProcessor() { + @Override + public void process(Channel channel, Command command) { + channel.writeAndFlush(Pong.create(command.getOpaque())); + } + }); + server.start(); + // + CountDownLatch latch = new CountDownLatch(1); + AtomicLong opaque = new AtomicLong(1); + final NettyClientConfig clientConfig = new NettyClientConfig(); + NettyRemotingClient client = new NettyRemotingClient(clientConfig); + client.registerProcessor(CommandType.PONG, new NettyRequestProcessor() { + @Override + public void process(Channel channel, Command command) { + opaque.set(command.getOpaque()); + latch.countDown(); + } + }); + Command commandPing = Ping.create(); + try { + client.send(new Address("127.0.0.1", serverConfig.getListenPort()), commandPing); + latch.await(); + } catch (Exception e) { + e.printStackTrace(); + } + Assert.assertEquals(opaque.get(), commandPing.getOpaque()); + } +} diff --git a/dolphinscheduler-rpc/pom.xml b/dolphinscheduler-rpc/pom.xml deleted file mode 100644 index 680a4a24c0..0000000000 --- a/dolphinscheduler-rpc/pom.xml +++ /dev/null @@ -1,113 +0,0 @@ - - - - - - org.apache.dolphinscheduler - dolphinscheduler - 1.2.1-SNAPSHOT - - 4.0.0 - - dolphinscheduler-rpc - - dolphinscheduler-rpc - https://github.com/apache/incubator-dolphinscheduler - - - UTF-8 - 1.8 - 1.8 - - 3.5.1 - 1.9.0 - - - - - com.google.protobuf - protobuf-java - ${protobuf.version} - - - io.grpc - grpc-netty - ${grpc.version} - - - io.grpc - grpc-protobuf - ${grpc.version} - - - io.grpc - grpc-stub - ${grpc.version} - - - - com.google.guava - guava - - - - - - - kr.motd.maven - os-maven-plugin - 1.5.0.Final - - - - - org.xolstice.maven.plugins - protobuf-maven-plugin - 0.5.0 - - com.google.protobuf:protoc:3.5.1-1:exe:${os.detected.classifier} - grpc-java - io.grpc:protoc-gen-grpc-java:${grpc.version}:exe:${os.detected.classifier} - - - - compile - - compile - - - - compile-custom - - compile-custom - - - - - - org.apache.maven.plugins - maven-compiler-plugin - - ${java.version} - ${java.version} - ${project.build.sourceEncoding} - - - - - diff --git a/dolphinscheduler-rpc/src/main/proto/scheduler.proto b/dolphinscheduler-rpc/src/main/proto/scheduler.proto deleted file mode 100644 index b8b595cb2a..0000000000 --- a/dolphinscheduler-rpc/src/main/proto/scheduler.proto +++ /dev/null @@ -1,101 +0,0 @@ -/* - * 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. - * - */ - -syntax = "proto3"; - -package schduler; - -option java_multiple_files = true; -option java_package = "org.apache.dolphinscheduler.rpc"; -option java_outer_classname = "SchdulerProto"; - - -/** - * return str info - */ -message RetStrInfo { - /** - * str msg info - */ - string msg = 1 ; -} - -/** - * return byte info - */ -message RetByteInfo { - /** - * byte data info - */ - bytes data = 1; -} - -/** - * log parameter - */ -message LogParameter { - - /** - * path - */ - string path = 1 ; - - /** - * skip line num - */ - int32 skipLineNum = 2 ; - - /** - * display limt num - */ - int32 limit = 3 ; -} - - -/** - * path parameter - */ -message PathParameter { - - /** - * path - */ - string path = 1 ; -} - -/** - * log view service - */ -service LogViewService { - - /** - * roll view log - */ - rpc rollViewLog(LogParameter) returns (RetStrInfo) {}; - - /** - * view all log - */ - rpc viewLog(PathParameter) returns (RetStrInfo) {}; - - /** - * get log bytes - */ - rpc getLogBytes(PathParameter) returns (RetByteInfo) {}; -} - diff --git a/dolphinscheduler-server/pom.xml b/dolphinscheduler-server/pom.xml index 751fd919a8..080b87ebaa 100644 --- a/dolphinscheduler-server/pom.xml +++ b/dolphinscheduler-server/pom.xml @@ -71,7 +71,7 @@ org.apache.dolphinscheduler - dolphinscheduler-rpc + dolphinscheduler-service org.apache.curator diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/log/LoggerRequestProcessor.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/log/LoggerRequestProcessor.java new file mode 100644 index 0000000000..4e4404ea1c --- /dev/null +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/log/LoggerRequestProcessor.java @@ -0,0 +1,179 @@ +/* + * 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.server.log; + +import io.netty.channel.Channel; +import org.apache.dolphinscheduler.remote.command.Command; +import org.apache.dolphinscheduler.remote.command.CommandType; +import org.apache.dolphinscheduler.remote.command.log.*; +import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; +import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.*; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * logger request process logic + */ +public class LoggerRequestProcessor implements NettyRequestProcessor { + + private final Logger logger = LoggerFactory.getLogger(LoggerRequestProcessor.class); + + private final ThreadPoolExecutor executor; + + public LoggerRequestProcessor(){ + this.executor = new ThreadPoolExecutor(4, 4, 10, TimeUnit.SECONDS, new LinkedBlockingQueue<>(100)); + } + + @Override + public void process(Channel channel, Command command) { + logger.info("received command : {}", command); + + /** + * reuqest task log command type + */ + final CommandType commandType = command.getType(); + switch (commandType){ + case GET_LOG_BYTES_REQUEST: + GetLogBytesRequestCommand getLogRequest = FastJsonSerializer.deserialize( + command.getBody(), GetLogBytesRequestCommand.class); + byte[] bytes = getFileContentBytes(getLogRequest.getPath()); + GetLogBytesResponseCommand getLogResponse = new GetLogBytesResponseCommand(bytes); + channel.writeAndFlush(getLogResponse.convert2Command(command.getOpaque())); + break; + case VIEW_WHOLE_LOG_REQUEST: + ViewLogRequestCommand viewLogRequest = FastJsonSerializer.deserialize( + command.getBody(), ViewLogRequestCommand.class); + String msg = readWholeFileContent(viewLogRequest.getPath()); + ViewLogResponseCommand viewLogResponse = new ViewLogResponseCommand(msg); + channel.writeAndFlush(viewLogResponse.convert2Command(command.getOpaque())); + break; + case ROLL_VIEW_LOG_REQUEST: + RollViewLogRequestCommand rollViewLogRequest = FastJsonSerializer.deserialize( + command.getBody(), RollViewLogRequestCommand.class); + List lines = readPartFileContent(rollViewLogRequest.getPath(), + rollViewLogRequest.getSkipLineNum(), rollViewLogRequest.getLimit()); + StringBuilder builder = new StringBuilder(); + for (String line : lines){ + builder.append(line + "\r\n"); + } + RollViewLogResponseCommand rollViewLogRequestResponse = new RollViewLogResponseCommand(builder.toString()); + channel.writeAndFlush(rollViewLogRequestResponse.convert2Command(command.getOpaque())); + break; + default: + throw new IllegalArgumentException("unknown commandType"); + } + } + + public ExecutorService getExecutor(){ + return this.executor; + } + + /** + * get files content bytes,for down load file + * + * @param filePath file path + * @return byte array of file + * @throws Exception exception + */ + private byte[] getFileContentBytes(String filePath){ + InputStream in = null; + ByteArrayOutputStream bos = null; + try { + in = new FileInputStream(filePath); + bos = new ByteArrayOutputStream(); + byte[] buf = new byte[1024]; + int len; + while ((len = in.read(buf)) != -1) { + bos.write(buf, 0, len); + } + return bos.toByteArray(); + }catch (IOException e){ + logger.error("get file bytes error",e); + }finally { + if (bos != null){ + try { + bos.close(); + } catch (IOException ignore) {} + } + if (in != null){ + try { + in.close(); + } catch (IOException ignore) {} + } + } + return new byte[0]; + } + + /** + * read part file content,can skip any line and read some lines + * + * @param filePath file path + * @param skipLine skip line + * @param limit read lines limit + * @return part file content + */ + private List readPartFileContent(String filePath, + int skipLine, + int limit){ + try (Stream stream = Files.lines(Paths.get(filePath))) { + return stream.skip(skipLine).limit(limit).collect(Collectors.toList()); + } catch (IOException e) { + logger.error("read file error",e); + } + return Collections.EMPTY_LIST; + } + + /** + * read whole file content + * + * @param filePath file path + * @return whole file content + */ + private String readWholeFileContent(String filePath){ + BufferedReader br = null; + String line; + StringBuilder sb = new StringBuilder(); + try { + br = new BufferedReader(new InputStreamReader(new FileInputStream(filePath))); + while ((line = br.readLine()) != null){ + sb.append(line + "\r\n"); + } + return sb.toString(); + }catch (IOException e){ + logger.error("read file error",e); + }finally { + try { + if (br != null){ + br.close(); + } + } catch (IOException ignore) {} + } + return ""; + } +} diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/log/LoggerServer.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/log/LoggerServer.java new file mode 100644 index 0000000000..3520fb09ec --- /dev/null +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/log/LoggerServer.java @@ -0,0 +1,91 @@ +/* + * 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.server.log; + + +import org.apache.dolphinscheduler.common.Constants; +import org.apache.dolphinscheduler.remote.NettyRemotingServer; +import org.apache.dolphinscheduler.remote.command.CommandType; +import org.apache.dolphinscheduler.remote.config.NettyServerConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * logger server + */ +public class LoggerServer { + + private static final Logger logger = LoggerFactory.getLogger(LoggerServer.class); + + /** + * netty server + */ + private final NettyRemotingServer server; + + /** + * netty server config + */ + private final NettyServerConfig serverConfig; + + /** + * loggger request processor + */ + private final LoggerRequestProcessor requestProcessor; + + public LoggerServer(){ + this.serverConfig = new NettyServerConfig(); + this.serverConfig.setListenPort(Constants.RPC_PORT); + this.server = new NettyRemotingServer(serverConfig); + this.requestProcessor = new LoggerRequestProcessor(); + this.server.registerProcessor(CommandType.GET_LOG_BYTES_REQUEST, requestProcessor, requestProcessor.getExecutor()); + this.server.registerProcessor(CommandType.ROLL_VIEW_LOG_REQUEST, requestProcessor, requestProcessor.getExecutor()); + this.server.registerProcessor(CommandType.VIEW_WHOLE_LOG_REQUEST, requestProcessor, requestProcessor.getExecutor()); + } + + /** + * main launches the server from the command line. + * @param args arguments + */ + public static void main(String[] args) { + final LoggerServer server = new LoggerServer(); + server.start(); + } + + /** + * server start + */ + public void start() { + this.server.start(); + logger.info("logger server started, listening on port : {}" , Constants.RPC_PORT); + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + LoggerServer.this.stop(); + } + }); + } + + /** + * stop + */ + public void stop() { + this.server.close(); + logger.info("logger server shut down"); + } + +} diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/rpc/LogClient.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/rpc/LogClient.java deleted file mode 100644 index 1c6c97b88f..0000000000 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/rpc/LogClient.java +++ /dev/null @@ -1,149 +0,0 @@ -/* - * 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.server.rpc; - -import io.grpc.ManagedChannel; -import io.grpc.ManagedChannelBuilder; -import io.grpc.StatusRuntimeException; -import org.apache.dolphinscheduler.rpc.*; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.concurrent.TimeUnit; - -/** - * log client - */ -public class LogClient { - - /** - * logger of LogClient - */ - private static final Logger logger = LoggerFactory.getLogger(LogClient.class); - - /** - * managed channel - */ - private final ManagedChannel channel; - - /** - * blocking stub - */ - private final LogViewServiceGrpc.LogViewServiceBlockingStub blockingStub; - - /** - * Construct client connecting to HelloWorld server at host:port. - * - * @param host host - * @param port port - */ - public LogClient(String host, int port) { - this(ManagedChannelBuilder.forAddress(host, port) - // Channels are secure by default (via SSL/TLS). For the example we disable TLS to avoid - // needing certificates. - .usePlaintext(true)); - } - - /** - * Construct client for accessing RouteGuide server using the existing channel. - * - * @param channelBuilder channel builder - */ - LogClient(ManagedChannelBuilder channelBuilder) { - /** - * set max message read size - */ - channelBuilder.maxInboundMessageSize(Integer.MAX_VALUE); - channel = channelBuilder.build(); - blockingStub = LogViewServiceGrpc.newBlockingStub(channel); - } - - /** - * shut down channel - * - * @throws InterruptedException interrupted exception - */ - public void shutdown() throws InterruptedException { - channel.shutdown().awaitTermination(5, TimeUnit.SECONDS); - } - - /** - * roll view log - * - * @param path log path - * @param skipLineNum skip line num - * @param limit limit - * @return log content - */ - public String rollViewLog(String path,int skipLineNum,int limit) { - logger.info("roll view log , path : {},skipLineNum : {} ,limit :{}", path, skipLineNum, limit); - LogParameter pathParameter = LogParameter - .newBuilder() - .setPath(path) - .setSkipLineNum(skipLineNum) - .setLimit(limit) - .build(); - RetStrInfo retStrInfo; - try { - retStrInfo = blockingStub.rollViewLog(pathParameter); - return retStrInfo.getMsg(); - } catch (StatusRuntimeException e) { - logger.error("roll view log failed", e); - return null; - } - } - - /** - * view all log - * - * @param path log path - * @return log content - */ - public String viewLog(String path) { - logger.info("view log path : {}",path); - - PathParameter pathParameter = PathParameter.newBuilder().setPath(path).build(); - RetStrInfo retStrInfo; - try { - retStrInfo = blockingStub.viewLog(pathParameter); - return retStrInfo.getMsg(); - } catch (StatusRuntimeException e) { - logger.error("view log failed", e); - return null; - } - } - - /** - * get log bytes - * - * @param path log path - * @return log content - */ - public byte[] getLogBytes(String path) { - logger.info("get log bytes {}",path); - - PathParameter pathParameter = PathParameter.newBuilder().setPath(path).build(); - RetByteInfo retByteInfo; - try { - retByteInfo = blockingStub.getLogBytes(pathParameter); - return retByteInfo.getData().toByteArray(); - } catch (StatusRuntimeException e) { - logger.error("get log bytes failed ", e); - return null; - } - } -} \ No newline at end of file diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/rpc/LoggerServer.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/rpc/LoggerServer.java deleted file mode 100644 index 5ec5df92fc..0000000000 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/rpc/LoggerServer.java +++ /dev/null @@ -1,238 +0,0 @@ -/* - * 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.server.rpc; - -import io.grpc.stub.StreamObserver; -import org.apache.dolphinscheduler.common.Constants; -import com.google.protobuf.ByteString; -import io.grpc.Server; -import io.grpc.ServerBuilder; -import org.apache.dolphinscheduler.rpc.*; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.*; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.util.List; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -/** - * logger server - */ -public class LoggerServer { - - private static final Logger logger = LoggerFactory.getLogger(LoggerServer.class); - - /** - * server - */ - private Server server; - - /** - * server start - * @throws IOException io exception - */ - public void start() throws IOException { - /* The port on which the server should run */ - int port = Constants.RPC_PORT; - server = ServerBuilder.forPort(port) - .addService(new LogViewServiceGrpcImpl()) - .build() - .start(); - logger.info("server started, listening on port : {}" , port); - Runtime.getRuntime().addShutdownHook(new Thread() { - @Override - public void run() { - // Use stderr here since the logger may have been reset by its JVM shutdown hook. - logger.info("shutting down gRPC server since JVM is shutting down"); - LoggerServer.this.stop(); - logger.info("server shut down"); - } - }); - } - - /** - * stop - */ - private void stop() { - if (server != null) { - server.shutdown(); - } - } - - /** - * await termination on the main thread since the grpc library uses daemon threads. - */ - private void blockUntilShutdown() throws InterruptedException { - if (server != null) { - server.awaitTermination(); - } - } - - /** - * main launches the server from the command line. - */ - - /** - * main launches the server from the command line. - * @param args arguments - * @throws IOException io exception - * @throws InterruptedException interrupted exception - */ - public static void main(String[] args) throws IOException, InterruptedException { - final LoggerServer server = new LoggerServer(); - server.start(); - server.blockUntilShutdown(); - } - - /** - * Log View Service Grpc Implementation - */ - static class LogViewServiceGrpcImpl extends LogViewServiceGrpc.LogViewServiceImplBase { - @Override - public void rollViewLog(LogParameter request, StreamObserver responseObserver) { - - logger.info("log parameter path : {} ,skip line : {}, limit : {}", - request.getPath(), - request.getSkipLineNum(), - request.getLimit()); - List list = readFile(request.getPath(), request.getSkipLineNum(), request.getLimit()); - StringBuilder sb = new StringBuilder(); - boolean errorLineFlag = false; - for (String line : list){ - sb.append(line + "\r\n"); - } - RetStrInfo retInfoBuild = RetStrInfo.newBuilder().setMsg(sb.toString()).build(); - responseObserver.onNext(retInfoBuild); - responseObserver.onCompleted(); - } - - @Override - public void viewLog(PathParameter request, StreamObserver responseObserver) { - logger.info("task path is : {} " , request.getPath()); - RetStrInfo retInfoBuild = RetStrInfo.newBuilder().setMsg(readFile(request.getPath())).build(); - responseObserver.onNext(retInfoBuild); - responseObserver.onCompleted(); - } - - @Override - public void getLogBytes(PathParameter request, StreamObserver responseObserver) { - try { - ByteString bytes = ByteString.copyFrom(getFileBytes(request.getPath())); - RetByteInfo.Builder builder = RetByteInfo.newBuilder(); - builder.setData(bytes); - responseObserver.onNext(builder.build()); - responseObserver.onCompleted(); - }catch (Exception e){ - logger.error("get log bytes failed",e); - } - } - } - - /** - * get files bytes - * - * @param path path - * @return byte array of file - * @throws Exception exception - */ - private static byte[] getFileBytes(String path){ - InputStream in = null; - ByteArrayOutputStream bos = null; - try { - in = new FileInputStream(path); - bos = new ByteArrayOutputStream(); - byte[] buf = new byte[1024]; - int len = 0; - while ((len = in.read(buf)) != -1) { - bos.write(buf, 0, len); - } - return bos.toByteArray(); - }catch (IOException e){ - logger.error("get file bytes error",e); - }finally { - if (bos != null){ - try { - bos.close(); - } catch (IOException e) { - e.printStackTrace(); - } - } - if (in != null){ - try { - in.close(); - } catch (IOException e) { - e.printStackTrace(); - } - } - } - return null; - } - - /** - * read file content - * - * @param path - * @param skipLine - * @param limit - * @return - */ - private static List readFile(String path,int skipLine,int limit){ - try (Stream stream = Files.lines(Paths.get(path))) { - return stream.skip(skipLine).limit(limit).collect(Collectors.toList()); - } catch (IOException e) { - logger.error("read file failed",e); - } - return null; - } - - /** - * read file content - * - * @param path path - * @return string of file content - * @throws Exception exception - */ - private static String readFile(String path){ - BufferedReader br = null; - String line = null; - StringBuilder sb = new StringBuilder(); - try { - br = new BufferedReader(new InputStreamReader(new FileInputStream(path))); - boolean errorLineFlag = false; - while ((line = br.readLine()) != null){ - sb.append(line + "\r\n"); - } - - return sb.toString(); - }catch (IOException e){ - logger.error("read file failed",e); - }finally { - try { - if (br != null){ - br.close(); - } - } catch (IOException e) { - logger.error(e.getMessage(),e); - } - } - return null; - } - -} \ No newline at end of file diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ProcessUtils.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ProcessUtils.java index fd0a08cd8e..69284ee69c 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ProcessUtils.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ProcessUtils.java @@ -22,8 +22,8 @@ import org.apache.dolphinscheduler.common.utils.LoggerUtils; import org.apache.dolphinscheduler.common.utils.OSUtils; import org.apache.dolphinscheduler.common.utils.StringUtils; import org.apache.dolphinscheduler.dao.entity.TaskInstance; -import org.apache.dolphinscheduler.server.rpc.LogClient; import org.apache.commons.io.FileUtils; +import org.apache.dolphinscheduler.service.log.LogClientService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -375,7 +375,7 @@ public class ProcessUtils { public static void killYarnJob(TaskInstance taskInstance) { try { Thread.sleep(Constants.SLEEP_TIME_MILLIS); - LogClient logClient = new LogClient(taskInstance.getHost(), Constants.RPC_PORT); + LogClientService logClient = new LogClientService(taskInstance.getHost(), Constants.RPC_PORT); String log = logClient.viewLog(taskInstance.getLogPath()); if (StringUtils.isNotEmpty(log)) { diff --git a/dolphinscheduler-service/pom.xml b/dolphinscheduler-service/pom.xml new file mode 100644 index 0000000000..31a2837254 --- /dev/null +++ b/dolphinscheduler-service/pom.xml @@ -0,0 +1,29 @@ + + + + + dolphinscheduler + org.apache.dolphinscheduler + 1.2.1-SNAPSHOT + + 4.0.0 + + dolphinscheduler-service + + dolphinscheduler-service + http://www.example.com + + + UTF-8 + 1.7 + 1.7 + + + + + org.apache.dolphinscheduler + dolphinscheduler-remote + + + diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/MasterResponseCommand.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/MasterResponseCommand.java new file mode 100644 index 0000000000..7607159c7c --- /dev/null +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/MasterResponseCommand.java @@ -0,0 +1,55 @@ +/* + * 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.service; + +import org.apache.dolphinscheduler.remote.command.Command; +import org.apache.dolphinscheduler.remote.command.CommandType; +import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer; + +import java.io.Serializable; + +/** + * view log response command + */ +public class MasterResponseCommand implements Serializable { + + private String msg; + + public MasterResponseCommand() { + } + + public MasterResponseCommand(String msg) { + this.msg = msg; + } + + public String getMsg() { + return msg; + } + + public void setMsg(String msg) { + this.msg = msg; + } + + public Command convert2Command(long opaque){ + Command command = new Command(opaque); + command.setType(CommandType.MASTER_RESPONSE); + byte[] body = FastJsonSerializer.serialize(this); + command.setBody(body); + return command; + } +} diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/WorkerRequestCommand.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/WorkerRequestCommand.java new file mode 100644 index 0000000000..419add4d7c --- /dev/null +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/WorkerRequestCommand.java @@ -0,0 +1,58 @@ +/* + * 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.service; + +import org.apache.dolphinscheduler.remote.command.Command; +import org.apache.dolphinscheduler.remote.command.CommandType; +import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer; + +import java.io.Serializable; +import java.util.concurrent.atomic.AtomicLong; + +/** + * view log request command + */ +public class WorkerRequestCommand implements Serializable { + + private static final AtomicLong REQUEST = new AtomicLong(1); + + private String path; + + public WorkerRequestCommand() { + } + + public WorkerRequestCommand(String path) { + this.path = path; + } + + public String getPath() { + return path; + } + + public void setPath(String path) { + this.path = path; + } + + public Command convert2Command(){ + Command command = new Command(REQUEST.getAndIncrement()); + command.setType(CommandType.WORKER_REQUEST); + byte[] body = FastJsonSerializer.serialize(this); + command.setBody(body); + return command; + } +} diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java new file mode 100644 index 0000000000..a316c7046d --- /dev/null +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java @@ -0,0 +1,166 @@ +/* + * 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.service.log; + +import io.netty.channel.Channel; +import org.apache.dolphinscheduler.remote.NettyRemotingClient; +import org.apache.dolphinscheduler.remote.command.Command; +import org.apache.dolphinscheduler.remote.command.CommandType; +import org.apache.dolphinscheduler.remote.command.log.*; +import org.apache.dolphinscheduler.remote.config.NettyClientConfig; +import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; +import org.apache.dolphinscheduler.remote.utils.Address; +import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * log client + */ +public class LogClientService implements NettyRequestProcessor { + + private static final Logger logger = LoggerFactory.getLogger(LogClientService.class); + + private final NettyClientConfig clientConfig; + + private final NettyRemotingClient client; + + private final Address address; + + /** + * request time out + */ + private final long logRequestTimeout = 10 * 1000; + + /** + * construct client + * @param host host + * @param port port + */ + public LogClientService(String host, int port) { + this.address = new Address(host, port); + this.clientConfig = new NettyClientConfig(); + this.clientConfig.setWorkerThreads(1); + this.client = new NettyRemotingClient(clientConfig); + this.client.registerProcessor(CommandType.ROLL_VIEW_LOG_RESPONSE,this); + this.client.registerProcessor(CommandType.VIEW_WHOLE_LOG_RESPONSE, this); + this.client.registerProcessor(CommandType.GET_LOG_BYTES_RESPONSE, this); + + } + + /** + * shutdown + */ + public void shutdown() { + this.client.close(); + logger.info("logger client shutdown"); + } + + /** + * roll view log + * @param path path + * @param skipLineNum skip line number + * @param limit limit + * @return log content + */ + public String rollViewLog(String path,int skipLineNum,int limit) { + logger.info("roll view log, path {}, skipLineNum {} ,limit {}", path, skipLineNum, limit); + RollViewLogRequestCommand request = new RollViewLogRequestCommand(path, skipLineNum, limit); + String result = ""; + try { + Command command = request.convert2Command(); + this.client.send(address, command); + LogPromise promise = new LogPromise(command.getOpaque(), logRequestTimeout); + result = ((String)promise.getResult()); + } catch (Exception e) { + logger.error("roll view log error", e); + } + return result; + } + + /** + * view log + * @param path path + * @return log content + */ + public String viewLog(String path) { + logger.info("view log path {}", path); + ViewLogRequestCommand request = new ViewLogRequestCommand(path); + String result = ""; + try { + Command command = request.convert2Command(); + this.client.send(address, command); + LogPromise promise = new LogPromise(command.getOpaque(), logRequestTimeout); + result = ((String)promise.getResult()); + } catch (Exception e) { + logger.error("view log error", e); + } + return result; + } + + /** + * get log size + * @param path log path + * @return log content bytes + */ + public byte[] getLogBytes(String path) { + logger.info("log path {}", path); + GetLogBytesRequestCommand request = new GetLogBytesRequestCommand(path); + byte[] result = null; + try { + Command command = request.convert2Command(); + this.client.send(address, command); + LogPromise promise = new LogPromise(command.getOpaque(), logRequestTimeout); + result = (byte[])promise.getResult(); + } catch (Exception e) { + logger.error("get log size error", e); + } + return result; + } + + @Override + public void process(Channel channel, Command command) { + logger.info("received log response : {}", command); + switch (command.getType()){ + case ROLL_VIEW_LOG_RESPONSE: + RollViewLogResponseCommand rollReviewLog = FastJsonSerializer.deserialize( + command.getBody(), RollViewLogResponseCommand.class); + LogPromise.notify(command.getOpaque(), rollReviewLog.getMsg()); + break; + case VIEW_WHOLE_LOG_RESPONSE: + ViewLogResponseCommand viewLog = FastJsonSerializer.deserialize( + command.getBody(), ViewLogResponseCommand.class); + LogPromise.notify(command.getOpaque(), viewLog.getMsg()); + break; + case GET_LOG_BYTES_RESPONSE: + GetLogBytesResponseCommand getLog = FastJsonSerializer.deserialize( + command.getBody(), GetLogBytesResponseCommand.class); + LogPromise.notify(command.getOpaque(), getLog.getData()); + break; + default: + throw new UnsupportedOperationException(String.format("command type : %s is not supported ", command.getType())); + } + } + + public static void main(String[] args) throws Exception{ + LogClientService logClient = new LogClientService("192.168.220.247", 50051); + byte[] logBytes = logClient.getLogBytes("/opt/program/incubator-dolphinscheduler/logs/1/463/540.log"); + System.out.println(new String(logBytes)); + } + +} \ No newline at end of file diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogPromise.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogPromise.java new file mode 100644 index 0000000000..8920b8a527 --- /dev/null +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogPromise.java @@ -0,0 +1,81 @@ +/* + * 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.service.log; + +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +/** + * log asyc callback + */ +public class LogPromise { + + private static final ConcurrentHashMap PROMISES = new ConcurrentHashMap<>(); + + private long opaque; + + private final long start; + + private final long timeout; + + private final CountDownLatch latch; + + private Object result; + + public LogPromise(long opaque, long timeout){ + this.opaque = opaque; + this.timeout = timeout; + this.start = System.currentTimeMillis(); + this.latch = new CountDownLatch(1); + PROMISES.put(opaque, this); + } + + + /** + * notify client finish + * @param opaque unique identification + * @param result result + */ + public static void notify(long opaque, Object result){ + LogPromise promise = PROMISES.remove(opaque); + if(promise != null){ + promise.doCountDown(result); + } + } + + private void doCountDown(Object result){ + this.result = result; + this.latch.countDown(); + } + + public boolean isTimeout(){ + return System.currentTimeMillis() - start > timeout; + } + + public Object getResult(){ + try { + latch.await(timeout, TimeUnit.MILLISECONDS); + } catch (InterruptedException ignore) { + } + PROMISES.remove(opaque); + return this.result; + } + + +} diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/worker/WorkerClientService.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/worker/WorkerClientService.java new file mode 100644 index 0000000000..c1071226a8 --- /dev/null +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/worker/WorkerClientService.java @@ -0,0 +1,107 @@ +/* + * 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.service.worker; + +import io.netty.channel.Channel; +import org.apache.dolphinscheduler.remote.NettyRemotingClient; +import org.apache.dolphinscheduler.remote.command.Command; +import org.apache.dolphinscheduler.remote.command.CommandType; +import org.apache.dolphinscheduler.remote.command.log.*; +import org.apache.dolphinscheduler.remote.config.NettyClientConfig; +import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; +import org.apache.dolphinscheduler.remote.utils.Address; +import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer; +import org.apache.dolphinscheduler.service.MasterResponseCommand; +import org.apache.dolphinscheduler.service.WorkerRequestCommand; +import org.apache.dolphinscheduler.service.log.LogPromise; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * log client + */ +public class WorkerClientService implements NettyRequestProcessor { + + private static final Logger logger = LoggerFactory.getLogger(WorkerClientService.class); + + private final NettyClientConfig clientConfig; + + private final NettyRemotingClient client; + + private final Address address; + + /** + * request time out + */ + private final long logRequestTimeout = 10 * 1000; + + /** + * construct client + * @param host host + * @param port port + */ + public WorkerClientService(String host, int port) { + this.address = new Address(host, port); + this.clientConfig = new NettyClientConfig(); + this.clientConfig.setWorkerThreads(1); + this.client = new NettyRemotingClient(clientConfig); + this.client.registerProcessor(CommandType.MASTER_RESPONSE, this); + + } + + /** + * shutdown + */ + public void shutdown() { + this.client.close(); + logger.info("logger client shutdown"); + } + + + public String reportResult() { + WorkerRequestCommand request = new WorkerRequestCommand(); + String result = ""; + try { + Command command = request.convert2Command(); + this.client.send(address, command); + LogPromise promise = new LogPromise(command.getOpaque(), logRequestTimeout); + result = ((String)promise.getResult()); + } catch (Exception e) { + e.printStackTrace(); + logger.error("roll view log error", e); + } + return result; + } + + + @Override + public void process(Channel channel, Command command) { + logger.info("received log response : {}", command); + MasterResponseCommand masterResponseCommand = FastJsonSerializer.deserialize( + command.getBody(), MasterResponseCommand.class); + LogPromise.notify(command.getOpaque(), masterResponseCommand.getMsg()); + } + + public static void main(String[] args) throws Exception{ + WorkerClientService workerClientService = new WorkerClientService("192.168.220.247", 1128); + String result = workerClientService.reportResult(); + System.out.println(result); + + } + +} \ No newline at end of file diff --git a/pom.xml b/pom.xml index 875577c672..f6009dc689 100644 --- a/pom.xml +++ b/pom.xml @@ -229,7 +229,12 @@ org.apache.dolphinscheduler - dolphinscheduler-rpc + dolphinscheduler-remote + ${project.version} + + + org.apache.dolphinscheduler + dolphinscheduler-service ${project.version} @@ -770,9 +775,6 @@ **/dolphinscheduler-ui/src/view/common/outro.inc **/dolphinscheduler-ui/src/view/common/meta.inc **/dolphinscheduler-ui/src/combo/1.0.0/3rd.css - - **/dolphinscheduler-rpc/src/main/java/org/apache/dolphinscheduler/rpc/LogViewServiceGrpc.java - true @@ -859,8 +861,9 @@ dolphinscheduler-api dolphinscheduler-dao dolphinscheduler-alert - dolphinscheduler-rpc dolphinscheduler-dist + dolphinscheduler-remote + dolphinscheduler-service From 29f2b41ed0444e85a336b5988ab17b8b1f20dd28 Mon Sep 17 00:00:00 2001 From: Tboy Date: Mon, 10 Feb 2020 21:00:39 +0800 Subject: [PATCH 002/221] 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: khadgarmage Co-authored-by: zhukai Co-authored-by: Yelli --- .../alert/utils/MailUtils.java | 8 +- dolphinscheduler-api/pom.xml | 6 - .../ProcessDefinitionController.java | 16 +-- .../controller/ProcessInstanceController.java | 14 +- .../api/service/DataAnalysisService.java | 10 +- .../api/service/ExecutorService.java | 34 ++--- .../api/service/LoggerService.java | 8 +- .../api/service/ProcessDefinitionService.java | 10 +- .../api/service/ProcessInstanceService.java | 44 +++--- .../api/service/ResourcesService.java | 2 +- .../api/service/SchedulerService.java | 22 +-- .../api/service/TaskInstanceService.java | 4 +- .../api/utils/ZookeeperMonitor.java | 4 +- .../api/service/ExecutorService2Test.java | 28 ++-- .../service/ProcessDefinitionServiceTest.java | 27 ++++ dolphinscheduler-common/pom.xml | 16 +-- .../common/utils/ScriptRunner.java | 4 +- .../common/utils/StringUtils.java | 125 +----------------- .../common/utils/ScriptRunnerTest.java | 4 +- .../common/utils/StringUtilsTest.java | 66 +++++++++ dolphinscheduler-dao/pom.xml | 15 --- .../dao/entity/ProcessDefinition.java | 48 ++++--- .../server/master/MasterServer.java | 14 +- .../runner/MasterBaseTaskExecThread.java | 16 +-- .../master/runner/MasterExecThread.java | 54 ++++---- .../master/runner/MasterSchedulerThread.java | 20 +-- .../master/runner/MasterTaskExecThread.java | 10 +- .../runner/SubProcessTaskExecThread.java | 16 +-- .../server/monitor/ZKMonitorImpl.java | 2 +- .../server/utils/RemoveZKNode.java | 2 +- .../server/worker/WorkerServer.java | 20 +-- .../server/worker/runner/FetchTaskThread.java | 28 ++-- .../worker/runner/TaskScheduleThread.java | 26 ++-- .../worker/task/AbstractCommandExecutor.java | 20 +-- .../server/worker/task/AbstractYarnTask.java | 10 +- .../server/worker/task/datax/DataxTask.java | 16 +-- .../task/dependent/DependentExecute.java | 14 +- .../worker/task/dependent/DependentTask.java | 10 +- .../server/worker/task/flink/FlinkTask.java | 2 +- .../server/worker/task/http/HttpTask.java | 10 +- .../task/processdure/ProcedureTask.java | 10 +- .../server/worker/task/python/PythonTask.java | 10 +- .../server/worker/task/shell/ShellTask.java | 8 +- .../server/worker/task/sql/SqlTask.java | 24 ++-- .../server/zk/ZKMasterClient.java | 18 +-- .../server/zk/ZKWorkerClient.java | 2 +- .../server/master/MasterExecThreadTest.java | 16 +-- .../shell/ShellCommandExecutorTest.java | 8 +- .../server/worker/sql/SqlExecutorTest.java | 8 +- .../worker/task/datax/DataxTaskTest.java | 16 +-- dolphinscheduler-service/pom.xml | 41 +++++- .../service/MasterResponseCommand.java | 55 -------- .../service/WorkerRequestCommand.java | 58 -------- .../service}/permission/PermissionCheck.java | 36 ++--- .../service/process/ProcessService.java | 14 +- .../quartz/DruidConnectionProvider.java | 3 +- .../service}/quartz/ProcessScheduleJob.java | 27 ++-- .../service}/quartz/QuartzExecutors.java | 4 +- .../service/quartz}/cron/AbstractCycle.java | 4 +- .../service/quartz}/cron/CronUtils.java | 10 +- .../service/quartz}/cron/CycleFactory.java | 4 +- .../service/quartz}/cron/CycleLinks.java | 4 +- .../service}/queue/ITaskQueue.java | 2 +- .../service}/queue/TaskQueueFactory.java | 4 +- .../service}/queue/TaskQueueZkImpl.java | 4 +- .../service/worker/WorkerClientService.java | 107 --------------- .../service}/zk/AbstractZKClient.java | 5 +- .../service}/zk/DefaultEnsembleProvider.java | 2 +- .../service}/zk/ZookeeperCachedOperator.java | 7 +- .../service}/zk/ZookeeperConfig.java | 2 +- .../service}/zk/ZookeeperOperator.java | 4 +- .../src/main/resources/quartz.properties | 2 +- .../src/test/java}/cron/CronUtilsTest.java | 8 +- .../test/java}/queue/BaseTaskQueueTest.java | 5 +- .../test/java}/queue/TaskQueueZKImplTest.java | 2 +- .../src/test/java/queue}/ZKServer.java | 2 +- .../pages/dag/_source/formModel/formModel.vue | 20 ++- .../dag/_source/formModel/tasks/dependent.vue | 17 ++- .../dag/_source/formModel/tasks/flink.vue | 2 +- .../pages/dag/_source/formModel/tasks/mr.vue | 29 ++++ .../dag/_source/formModel/tasks/procedure.vue | 25 +++- .../dag/_source/formModel/tasks/python.vue | 32 ++++- .../dag/_source/formModel/tasks/shell.vue | 34 ++++- .../dag/_source/formModel/tasks/spark.vue | 2 +- .../_source/formModel/tasks/sub_process.vue | 10 +- .../definition/pages/list/_source/list.vue | 7 + .../src/js/module/i18n/locale/en_US.js | 1 + .../src/js/module/i18n/locale/zh_CN.js | 1 + pom.xml | 1 + sql/dolphinscheduler-postgre.sql | 1 + sql/dolphinscheduler_mysql.sql | 1 + .../mysql/dolphinscheduler_ddl.sql | 37 ++++++ .../mysql/dolphinscheduler_dml.sql | 29 +--- .../postgresql/dolphinscheduler_ddl.sql | 34 +++++ .../postgresql/dolphinscheduler_dml.sql | 29 +--- 95 files changed, 788 insertions(+), 865 deletions(-) create mode 100644 dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/StringUtilsTest.java delete mode 100644 dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/MasterResponseCommand.java delete mode 100644 dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/WorkerRequestCommand.java rename {dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service}/permission/PermissionCheck.java (80%) rename dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/ProcessDao.java => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java (99%) rename {dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service}/quartz/DruidConnectionProvider.java (99%) rename {dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service}/quartz/ProcessScheduleJob.java (83%) rename {dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service}/quartz/QuartzExecutors.java (99%) rename {dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz}/cron/AbstractCycle.java (99%) rename {dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz}/cron/CronUtils.java (98%) rename {dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz}/cron/CycleFactory.java (99%) rename {dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz}/cron/CycleLinks.java (97%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service}/queue/ITaskQueue.java (97%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service}/queue/TaskQueueFactory.java (97%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service}/queue/TaskQueueZkImpl.java (99%) delete mode 100644 dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/worker/WorkerClientService.java rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service}/zk/AbstractZKClient.java (99%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service}/zk/DefaultEnsembleProvider.java (96%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service}/zk/ZookeeperCachedOperator.java (90%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service}/zk/ZookeeperConfig.java (98%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common => dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service}/zk/ZookeeperOperator.java (98%) rename {dolphinscheduler-common => dolphinscheduler-service}/src/main/resources/quartz.properties (96%) rename {dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao => dolphinscheduler-service/src/test/java}/cron/CronUtilsTest.java (98%) rename {dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common => dolphinscheduler-service/src/test/java}/queue/BaseTaskQueueTest.java (90%) rename {dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common => dolphinscheduler-service/src/test/java}/queue/TaskQueueZKImplTest.java (99%) rename {dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/zk => dolphinscheduler-service/src/test/java/queue}/ZKServer.java (99%) create mode 100644 sql/upgrade/1.2.2_schema/mysql/dolphinscheduler_ddl.sql rename dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/zk/TestZk.java => sql/upgrade/1.2.2_schema/mysql/dolphinscheduler_dml.sql (64%) create mode 100644 sql/upgrade/1.2.2_schema/postgresql/dolphinscheduler_ddl.sql rename dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/zk/TestZkServer.java => sql/upgrade/1.2.2_schema/postgresql/dolphinscheduler_dml.sql (64%) diff --git a/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/MailUtils.java b/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/MailUtils.java index 7ebe6a7863..99efdc8a6a 100644 --- a/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/MailUtils.java +++ b/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/MailUtils.java @@ -90,14 +90,14 @@ public class MailUtils { public static Map sendMails(Collection receivers, Collection receiversCc, String title, String content, ShowType showType) { Map retMap = new HashMap<>(); retMap.put(Constants.STATUS, false); - + // if there is no receivers && no receiversCc, no need to process if (CollectionUtils.isEmpty(receivers) && CollectionUtils.isEmpty(receiversCc)) { return retMap; } receivers.removeIf(StringUtils::isEmpty); - + if (showType == ShowType.TABLE || showType == ShowType.TEXT){ // send email HtmlEmail email = new HtmlEmail(); @@ -335,7 +335,7 @@ public class MailUtils { */ private static void handleException(Collection receivers, Map retMap, Exception e) { logger.error("Send email to {} failed {}", receivers, e); - retMap.put(Constants.MESSAGE, "Send email to {" + StringUtils.join(receivers, ",") + "} failed," + e.toString()); + retMap.put(Constants.MESSAGE, "Send email to {" + String.join(",", receivers) + "} failed," + e.toString()); } -} \ No newline at end of file +} diff --git a/dolphinscheduler-api/pom.xml b/dolphinscheduler-api/pom.xml index 11b23d9fb7..6440805af7 100644 --- a/dolphinscheduler-api/pom.xml +++ b/dolphinscheduler-api/pom.xml @@ -31,12 +31,6 @@ org.apache.dolphinscheduler dolphinscheduler-alert - - - org.apache.dolphinscheduler - dolphinscheduler-dao - - diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ProcessDefinitionController.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ProcessDefinitionController.java index de9cc12a36..c07ecf9ca7 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ProcessDefinitionController.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ProcessDefinitionController.java @@ -53,7 +53,7 @@ public class ProcessDefinitionController extends BaseController{ /** * create process definition - * + * * @param loginUser login user * @param projectName project name * @param name process definition name @@ -96,7 +96,7 @@ public class ProcessDefinitionController extends BaseController{ /** * verify process definition name unique - * + * * @param loginUser login user * @param projectName project name * @param name name @@ -328,9 +328,9 @@ public class ProcessDefinitionController extends BaseController{ /** - * + * * get tasks list by process definition id - * + * * * @param loginUser login user * @param projectName project name @@ -442,7 +442,7 @@ public class ProcessDefinitionController extends BaseController{ loginUser.getUserName(), projectName, processDefinitionIds); Map result = new HashMap<>(5); - List deleteFailedIdList = new ArrayList(); + List deleteFailedIdList = new ArrayList<>(); if(StringUtils.isNotEmpty(processDefinitionIds)){ String[] processDefinitionIdArray = processDefinitionIds.split(","); @@ -451,17 +451,17 @@ public class ProcessDefinitionController extends BaseController{ try { Map deleteResult = processDefinitionService.deleteProcessDefinitionById(loginUser, projectName, processDefinitionId); if(!Status.SUCCESS.equals(deleteResult.get(Constants.STATUS))){ - deleteFailedIdList.add(processDefinitionId); + deleteFailedIdList.add(strProcessDefinitionId); logger.error((String)deleteResult.get(Constants.MSG)); } } catch (Exception e) { - deleteFailedIdList.add(processDefinitionId); + deleteFailedIdList.add(strProcessDefinitionId); } } } if(!deleteFailedIdList.isEmpty()){ - putMsg(result, Status.BATCH_DELETE_PROCESS_DEFINE_BY_IDS_ERROR,StringUtils.join(deleteFailedIdList,",")); + putMsg(result, Status.BATCH_DELETE_PROCESS_DEFINE_BY_IDS_ERROR, String.join(",", deleteFailedIdList)); }else{ putMsg(result, Status.SUCCESS); } diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ProcessInstanceController.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ProcessInstanceController.java index 542aad5c33..80db6c86af 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ProcessInstanceController.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ProcessInstanceController.java @@ -22,12 +22,12 @@ import org.apache.dolphinscheduler.api.utils.Result; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.enums.Flag; -import org.apache.dolphinscheduler.common.queue.ITaskQueue; -import org.apache.dolphinscheduler.common.queue.TaskQueueFactory; import org.apache.dolphinscheduler.common.utils.ParameterUtils; import org.apache.dolphinscheduler.common.utils.StringUtils; import org.apache.dolphinscheduler.dao.entity.User; import io.swagger.annotations.*; +import org.apache.dolphinscheduler.service.queue.ITaskQueue; +import org.apache.dolphinscheduler.service.queue.TaskQueueFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; @@ -58,7 +58,7 @@ public class ProcessInstanceController extends BaseController{ /** * query process instance list paging - * + * * @param loginUser login user * @param projectName project name * @param pageNo page number @@ -372,7 +372,7 @@ public class ProcessInstanceController extends BaseController{ // task queue ITaskQueue tasksQueue = TaskQueueFactory.getTaskQueueInstance(); Map result = new HashMap<>(5); - List deleteFailedIdList = new ArrayList(); + List deleteFailedIdList = new ArrayList<>(); if(StringUtils.isNotEmpty(processInstanceIds)){ String[] processInstanceIdArray = processInstanceIds.split(","); @@ -381,16 +381,16 @@ public class ProcessInstanceController extends BaseController{ try { Map deleteResult = processInstanceService.deleteProcessInstanceById(loginUser, projectName, processInstanceId,tasksQueue); if(!Status.SUCCESS.equals(deleteResult.get(Constants.STATUS))){ - deleteFailedIdList.add(processInstanceId); + deleteFailedIdList.add(strProcessInstanceId); logger.error((String)deleteResult.get(Constants.MSG)); } } catch (Exception e) { - deleteFailedIdList.add(processInstanceId); + deleteFailedIdList.add(strProcessInstanceId); } } } if(deleteFailedIdList.size() > 0){ - putMsg(result, Status.BATCH_DELETE_PROCESS_INSTANCE_BY_IDS_ERROR,StringUtils.join(deleteFailedIdList,",")); + putMsg(result, Status.BATCH_DELETE_PROCESS_INSTANCE_BY_IDS_ERROR, String.join(",", deleteFailedIdList)); }else{ putMsg(result, Status.SUCCESS); } diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataAnalysisService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataAnalysisService.java index f4becbe36d..0743ef9294 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataAnalysisService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataAnalysisService.java @@ -24,13 +24,13 @@ import org.apache.dolphinscheduler.api.enums.Status; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.CommandType; import org.apache.dolphinscheduler.common.enums.UserType; -import org.apache.dolphinscheduler.common.queue.ITaskQueue; -import org.apache.dolphinscheduler.common.queue.TaskQueueFactory; import org.apache.dolphinscheduler.common.utils.DateUtils; import org.apache.dolphinscheduler.common.utils.StringUtils; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.*; import org.apache.dolphinscheduler.dao.mapper.*; +import org.apache.dolphinscheduler.service.process.ProcessService; +import org.apache.dolphinscheduler.service.queue.ITaskQueue; +import org.apache.dolphinscheduler.service.queue.TaskQueueFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; @@ -69,7 +69,7 @@ public class DataAnalysisService extends BaseService{ TaskInstanceMapper taskInstanceMapper; @Autowired - ProcessDao processDao; + ProcessService processService; /** * statistical task instance status data @@ -296,7 +296,7 @@ public class DataAnalysisService extends BaseService{ if(projectId !=0){ projectIds.add(projectId); }else if(loginUser.getUserType() == UserType.GENERAL_USER){ - projectIds = processDao.getProjectIdListHavePerm(loginUser.getId()); + projectIds = processService.getProjectIdListHavePerm(loginUser.getId()); if(projectIds.size() ==0 ){ projectIds.add(0); } diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java index 257f15d580..6edd48d499 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java @@ -25,12 +25,12 @@ import org.apache.dolphinscheduler.common.utils.CollectionUtils; import org.apache.dolphinscheduler.common.utils.DateUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.StringUtils; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.*; import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionMapper; import org.apache.dolphinscheduler.dao.mapper.ProcessInstanceMapper; import org.apache.dolphinscheduler.dao.mapper.ProjectMapper; -import org.apache.dolphinscheduler.dao.utils.cron.CronUtils; +import org.apache.dolphinscheduler.service.process.ProcessService; +import org.apache.dolphinscheduler.service.quartz.cron.CronUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; @@ -67,7 +67,7 @@ public class ExecutorService extends BaseService{ @Autowired - private ProcessDao processDao; + private ProcessService processService; /** * execute process instance @@ -186,13 +186,13 @@ public class ExecutorService extends BaseService{ return checkResult; } - ProcessInstance processInstance = processDao.findProcessInstanceDetailById(processInstanceId); + ProcessInstance processInstance = processService.findProcessInstanceDetailById(processInstanceId); if (processInstance == null) { putMsg(result, Status.PROCESS_INSTANCE_NOT_EXIST, processInstanceId); return result; } - ProcessDefinition processDefinition = processDao.findProcessDefineById(processInstance.getProcessDefinitionId()); + ProcessDefinition processDefinition = processService.findProcessDefineById(processInstance.getProcessDefinitionId()); if(executeType != ExecuteType.STOP && executeType != ExecuteType.PAUSE){ result = checkProcessDefinitionValid(processDefinition, processInstance.getProcessDefinitionId()); if (result.get(Constants.STATUS) != Status.SUCCESS) { @@ -227,7 +227,7 @@ public class ExecutorService extends BaseService{ } else { processInstance.setCommandType(CommandType.STOP); processInstance.addHistoryCmd(CommandType.STOP); - processDao.updateProcessInstance(processInstance); + processService.updateProcessInstance(processInstance); result = updateProcessInstanceState(processInstanceId, ExecutionStatus.READY_STOP); } break; @@ -237,7 +237,7 @@ public class ExecutorService extends BaseService{ } else { processInstance.setCommandType(CommandType.PAUSE); processInstance.addHistoryCmd(CommandType.PAUSE); - processDao.updateProcessInstance(processInstance); + processService.updateProcessInstance(processInstance); result = updateProcessInstanceState(processInstanceId, ExecutionStatus.READY_PAUSE); } break; @@ -257,7 +257,7 @@ public class ExecutorService extends BaseService{ */ private boolean checkTenantSuitable(ProcessDefinition processDefinition) { // checkTenantExists(); - Tenant tenant = processDao.getTenantForProcess(processDefinition.getTenantId(), + Tenant tenant = processService.getTenantForProcess(processDefinition.getTenantId(), processDefinition.getUserId()); if(tenant == null){ return false; @@ -319,7 +319,7 @@ public class ExecutorService extends BaseService{ private Map updateProcessInstanceState(Integer processInstanceId, ExecutionStatus executionStatus) { Map result = new HashMap<>(5); - int update = processDao.updateProcessInstanceState(processInstanceId, executionStatus); + int update = processService.updateProcessInstanceState(processInstanceId, executionStatus); if (update > 0) { putMsg(result, Status.SUCCESS); } else { @@ -347,12 +347,12 @@ public class ExecutorService extends BaseService{ CMDPARAM_RECOVER_PROCESS_ID_STRING, instanceId)); command.setExecutorId(loginUser.getId()); - if(!processDao.verifyIsNeedCreateCommand(command)){ + if(!processService.verifyIsNeedCreateCommand(command)){ putMsg(result, Status.PROCESS_INSTANCE_EXECUTING_COMMAND,processDefinitionId); return result; } - int create = processDao.createCommand(command); + int create = processService.createCommand(command); if (create > 0) { putMsg(result, Status.SUCCESS); @@ -376,7 +376,7 @@ public class ExecutorService extends BaseService{ putMsg(result,Status.REQUEST_PARAMS_NOT_VALID_ERROR,"process definition id"); } List ids = new ArrayList<>(); - processDao.recurseFindSubProcessId(processDefineId, ids); + processService.recurseFindSubProcessId(processDefineId, ids); Integer[] idArray = ids.toArray(new Integer[ids.size()]); if (ids.size() > 0){ List processDefinitionList; @@ -506,9 +506,9 @@ public class ExecutorService extends BaseService{ cmdParam.put(CMDPARAM_COMPLEMENT_DATA_START_DATE, DateUtils.dateToString(start)); cmdParam.put(CMDPARAM_COMPLEMENT_DATA_END_DATE, DateUtils.dateToString(end)); command.setCommandParam(JSONUtils.toJson(cmdParam)); - return processDao.createCommand(command); + return processService.createCommand(command); }else if (runMode == RunMode.RUN_MODE_PARALLEL){ - List schedules = processDao.queryReleaseSchedulerListByProcessDefinitionId(processDefineId); + List schedules = processService.queryReleaseSchedulerListByProcessDefinitionId(processDefineId); List listDate = new LinkedList<>(); if(!CollectionUtils.isEmpty(schedules)){ for (Schedule item : schedules) { @@ -521,7 +521,7 @@ public class ExecutorService extends BaseService{ cmdParam.put(CMDPARAM_COMPLEMENT_DATA_START_DATE, DateUtils.dateToString(date)); cmdParam.put(CMDPARAM_COMPLEMENT_DATA_END_DATE, DateUtils.dateToString(date)); command.setCommandParam(JSONUtils.toJson(cmdParam)); - processDao.createCommand(command); + processService.createCommand(command); } return listDate.size(); }else{ @@ -532,7 +532,7 @@ public class ExecutorService extends BaseService{ cmdParam.put(CMDPARAM_COMPLEMENT_DATA_START_DATE, DateUtils.dateToString(start)); cmdParam.put(CMDPARAM_COMPLEMENT_DATA_END_DATE, DateUtils.dateToString(start)); command.setCommandParam(JSONUtils.toJson(cmdParam)); - processDao.createCommand(command); + processService.createCommand(command); start = DateUtils.getSomeDay(start, 1); } return runCunt; @@ -544,7 +544,7 @@ public class ExecutorService extends BaseService{ } }else{ command.setCommandParam(JSONUtils.toJson(cmdParam)); - return processDao.createCommand(command); + return processService.createCommand(command); } return 0; diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/LoggerService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/LoggerService.java index 108d5d4853..3a9ff11fc3 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/LoggerService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/LoggerService.java @@ -20,9 +20,9 @@ import org.apache.dolphinscheduler.api.enums.Status; import org.apache.dolphinscheduler.api.utils.Result; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.utils.StringUtils; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.service.log.LogClientService; +import org.apache.dolphinscheduler.service.process.ProcessService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; @@ -37,7 +37,7 @@ public class LoggerService { private static final Logger logger = LoggerFactory.getLogger(LoggerService.class); @Autowired - private ProcessDao processDao; + private ProcessService processService; /** * view log @@ -49,7 +49,7 @@ public class LoggerService { */ public Result queryLog(int taskInstId, int skipLineNum, int limit) { - TaskInstance taskInstance = processDao.findTaskInstanceById(taskInstId); + TaskInstance taskInstance = processService.findTaskInstanceById(taskInstId); if (taskInstance == null){ return new Result(Status.TASK_INSTANCE_NOT_FOUND.getCode(), Status.TASK_INSTANCE_NOT_FOUND.getMsg()); @@ -80,7 +80,7 @@ public class LoggerService { * @return log byte array */ public byte[] getLogBytes(int taskInstId) { - TaskInstance taskInstance = processDao.findTaskInstanceById(taskInstId); + TaskInstance taskInstance = processService.findTaskInstanceById(taskInstId); if (taskInstance == null){ throw new RuntimeException("task instance is null"); } diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionService.java index 5fe708ce8c..844902b9d4 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionService.java @@ -43,9 +43,9 @@ import org.apache.dolphinscheduler.common.utils.CollectionUtils; import org.apache.dolphinscheduler.common.utils.DateUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.StringUtils; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.*; import org.apache.dolphinscheduler.dao.mapper.*; +import org.apache.dolphinscheduler.service.process.ProcessService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; @@ -94,7 +94,7 @@ public class ProcessDefinitionService extends BaseDAGService { private ScheduleMapper scheduleMapper; @Autowired - private ProcessDao processDao; + private ProcessService processService; @Autowired private WorkerGroupMapper workerGroupMapper; @@ -143,6 +143,7 @@ public class ProcessDefinitionService extends BaseDAGService { processDefine.setConnects(connects); processDefine.setTimeout(processData.getTimeout()); processDefine.setTenantId(processData.getTenantId()); + processDefine.setModifyBy(loginUser.getUserName()); //custom global params List globalParamsList = processData.getGlobalParams(); @@ -282,7 +283,7 @@ public class ProcessDefinitionService extends BaseDAGService { if ((checkProcessJson.get(Constants.STATUS) != Status.SUCCESS)) { return checkProcessJson; } - ProcessDefinition processDefinition = processDao.findProcessDefineById(id); + ProcessDefinition processDefinition = processService.findProcessDefineById(id); if (processDefinition == null) { // check process definition exists putMsg(result, Status.PROCESS_DEFINE_NOT_EXIST, id); @@ -295,7 +296,7 @@ public class ProcessDefinitionService extends BaseDAGService { putMsg(result, Status.SUCCESS); } - ProcessDefinition processDefine = processDao.findProcessDefineById(id); + ProcessDefinition processDefine = processService.findProcessDefineById(id); Date now = new Date(); processDefine.setId(id); @@ -308,6 +309,7 @@ public class ProcessDefinitionService extends BaseDAGService { processDefine.setConnects(connects); processDefine.setTimeout(processData.getTimeout()); processDefine.setTenantId(processData.getTenantId()); + processDefine.setModifyBy(loginUser.getUserName()); //custom global params List globalParamsList = new ArrayList<>(); diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessInstanceService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessInstanceService.java index 87e1a0ede1..2b1f04e6ce 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessInstanceService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessInstanceService.java @@ -30,15 +30,15 @@ import org.apache.dolphinscheduler.common.graph.DAG; import org.apache.dolphinscheduler.common.model.TaskNode; import org.apache.dolphinscheduler.common.model.TaskNodeRelation; import org.apache.dolphinscheduler.common.process.Property; -import org.apache.dolphinscheduler.common.queue.ITaskQueue; import org.apache.dolphinscheduler.common.utils.*; import org.apache.dolphinscheduler.common.utils.placeholder.BusinessTimeUtils; -import org.apache.dolphinscheduler.dao.ProcessDao; import com.alibaba.fastjson.JSON; import com.baomidou.mybatisplus.core.metadata.IPage; import com.baomidou.mybatisplus.extension.plugins.pagination.Page; import org.apache.dolphinscheduler.dao.entity.*; import org.apache.dolphinscheduler.dao.mapper.*; +import org.apache.dolphinscheduler.service.process.ProcessService; +import org.apache.dolphinscheduler.service.queue.ITaskQueue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; @@ -72,7 +72,7 @@ public class ProcessInstanceService extends BaseDAGService { ProjectService projectService; @Autowired - ProcessDao processDao; + ProcessService processService; @Autowired ProcessInstanceMapper processInstanceMapper; @@ -112,7 +112,7 @@ public class ProcessInstanceService extends BaseDAGService { if (resultEnum != Status.SUCCESS) { return checkResult; } - ProcessInstance processInstance = processDao.findProcessInstanceDetailById(processId); + ProcessInstance processInstance = processService.findProcessInstanceDetailById(processId); String workerGroupName = ""; if(processInstance.getWorkerGroupId() == -1){ workerGroupName = DEFAULT; @@ -125,7 +125,7 @@ public class ProcessInstanceService extends BaseDAGService { } } processInstance.setWorkerGroupName(workerGroupName); - ProcessDefinition processDefinition = processDao.findProcessDefineById(processInstance.getProcessDefinitionId()); + ProcessDefinition processDefinition = processService.findProcessDefineById(processInstance.getProcessDefinitionId()); processInstance.setReceivers(processDefinition.getReceivers()); processInstance.setReceiversCc(processDefinition.getReceiversCc()); result.put(Constants.DATA_LIST, processInstance); @@ -228,8 +228,8 @@ public class ProcessInstanceService extends BaseDAGService { if (resultEnum != Status.SUCCESS) { return checkResult; } - ProcessInstance processInstance = processDao.findProcessInstanceDetailById(processId); - List taskInstanceList = processDao.findValidTaskListByProcessId(processId); + ProcessInstance processInstance = processService.findProcessInstanceDetailById(processId); + List taskInstanceList = processService.findValidTaskListByProcessId(processId); AddDependResultForTaskList(taskInstanceList); Map resultMap = new HashMap<>(); resultMap.put(PROCESS_INSTANCE_STATE, processInstance.getState().toString()); @@ -304,7 +304,7 @@ public class ProcessInstanceService extends BaseDAGService { return checkResult; } - TaskInstance taskInstance = processDao.findTaskInstanceById(taskId); + TaskInstance taskInstance = processService.findTaskInstanceById(taskId); if (taskInstance == null) { putMsg(result, Status.TASK_INSTANCE_NOT_EXISTS, taskId); return result; @@ -314,7 +314,7 @@ public class ProcessInstanceService extends BaseDAGService { return result; } - ProcessInstance subWorkflowInstance = processDao.findSubProcessInstance( + ProcessInstance subWorkflowInstance = processService.findSubProcessInstance( taskInstance.getProcessInstanceId(), taskInstance.getId()); if (subWorkflowInstance == null) { putMsg(result, Status.SUB_PROCESS_INSTANCE_NOT_EXIST, taskId); @@ -356,7 +356,7 @@ public class ProcessInstanceService extends BaseDAGService { } //check process instance exists - ProcessInstance processInstance = processDao.findProcessInstanceDetailById(processInstanceId); + ProcessInstance processInstance = processService.findProcessInstanceDetailById(processInstanceId); if (processInstance == null) { putMsg(result, Status.PROCESS_INSTANCE_NOT_EXIST, processInstanceId); return result; @@ -380,7 +380,7 @@ public class ProcessInstanceService extends BaseDAGService { String globalParams = null; String originDefParams = null; int timeout = processInstance.getTimeout(); - ProcessDefinition processDefinition = processDao.findProcessDefineById(processInstance.getProcessDefinitionId()); + ProcessDefinition processDefinition = processService.findProcessDefineById(processInstance.getProcessDefinitionId()); if (StringUtils.isNotEmpty(processInstanceJson)) { ProcessData processData = JSONUtils.parseObject(processInstanceJson, ProcessData.class); //check workflow json is valid @@ -396,7 +396,7 @@ public class ProcessInstanceService extends BaseDAGService { processInstance.getCmdTypeIfComplement(), schedule); timeout = processData.getTimeout(); processInstance.setTimeout(timeout); - Tenant tenant = processDao.getTenantForProcess(processData.getTenantId(), + Tenant tenant = processService.getTenantForProcess(processData.getTenantId(), processDefinition.getUserId()); if(tenant != null){ processInstance.setTenantCode(tenant.getTenantCode()); @@ -406,7 +406,7 @@ public class ProcessInstanceService extends BaseDAGService { } // int update = processDao.updateProcessInstance(processInstanceId, processInstanceJson, // globalParams, schedule, flag, locations, connects); - int update = processDao.updateProcessInstance(processInstance); + int update = processService.updateProcessInstance(processInstance); int updateDefine = 1; if (syncDefine && StringUtils.isNotEmpty(processInstanceJson)) { processDefinition.setProcessDefinitionJson(processInstanceJson); @@ -445,7 +445,7 @@ public class ProcessInstanceService extends BaseDAGService { return checkResult; } - ProcessInstance subInstance = processDao.findProcessInstanceDetailById(subId); + ProcessInstance subInstance = processService.findProcessInstanceDetailById(subId); if (subInstance == null) { putMsg(result, Status.PROCESS_INSTANCE_NOT_EXIST, subId); return result; @@ -455,7 +455,7 @@ public class ProcessInstanceService extends BaseDAGService { return result; } - ProcessInstance parentWorkflowInstance = processDao.findParentProcessInstance(subId); + ProcessInstance parentWorkflowInstance = processService.findParentProcessInstance(subId); if (parentWorkflowInstance == null) { putMsg(result, Status.SUB_PROCESS_INSTANCE_NOT_EXIST); return result; @@ -476,7 +476,7 @@ public class ProcessInstanceService extends BaseDAGService { * @return delete result code */ @Transactional(rollbackFor = Exception.class) - public Map deleteProcessInstanceById(User loginUser, String projectName, Integer processInstanceId,ITaskQueue tasksQueue) { + public Map deleteProcessInstanceById(User loginUser, String projectName, Integer processInstanceId, ITaskQueue tasksQueue) { Map result = new HashMap<>(5); Project project = projectMapper.queryByName(projectName); @@ -486,8 +486,8 @@ public class ProcessInstanceService extends BaseDAGService { if (resultEnum != Status.SUCCESS) { return checkResult; } - ProcessInstance processInstance = processDao.findProcessInstanceDetailById(processInstanceId); - List taskInstanceList = processDao.findValidTaskListByProcessId(processInstanceId); + ProcessInstance processInstance = processService.findProcessInstanceDetailById(processInstanceId); + List taskInstanceList = processService.findValidTaskListByProcessId(processInstanceId); if (null == processInstance) { putMsg(result, Status.PROCESS_INSTANCE_NOT_EXIST, processInstanceId); @@ -512,7 +512,7 @@ public class ProcessInstanceService extends BaseDAGService { .append(taskInstance.getId()) .append(UNDERLINE); - int taskWorkerGroupId = processDao.getTaskWorkerGroupId(taskInstance); + int taskWorkerGroupId = processService.getTaskWorkerGroupId(taskInstance); WorkerGroup workerGroup = workerGroupMapper.selectById(taskWorkerGroupId); if(workerGroup == null){ @@ -541,9 +541,9 @@ public class ProcessInstanceService extends BaseDAGService { } // delete database cascade - int delete = processDao.deleteWorkProcessInstanceById(processInstanceId); - processDao.deleteAllSubWorkProcessByParentId(processInstanceId); - processDao.deleteWorkProcessMapByParentId(processInstanceId); + int delete = processService.deleteWorkProcessInstanceById(processInstanceId); + processService.deleteAllSubWorkProcessByParentId(processInstanceId); + processService.deleteWorkProcessMapByParentId(processInstanceId); if (delete > 0) { putMsg(result, Status.SUCCESS); diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ResourcesService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ResourcesService.java index 09b1d31151..29a16447e1 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ResourcesService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ResourcesService.java @@ -539,7 +539,7 @@ public class ResourcesService extends BaseService { putMsg(result, Status.SUCCESS); Map map = new HashMap<>(); map.put(ALIAS, resource.getAlias()); - map.put(CONTENT, StringUtils.join(content, "\n")); + map.put(CONTENT, String.join("\n", content)); result.setData(map); }else{ logger.error("read file {} not exist in hdfs", hdfsFileName); diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/SchedulerService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/SchedulerService.java index bdce9470ca..72122100a1 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/SchedulerService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/SchedulerService.java @@ -26,7 +26,6 @@ import org.apache.dolphinscheduler.common.model.Server; import org.apache.dolphinscheduler.common.utils.DateUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.StringUtils; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; import org.apache.dolphinscheduler.dao.entity.Project; import org.apache.dolphinscheduler.dao.entity.Schedule; @@ -34,11 +33,12 @@ import org.apache.dolphinscheduler.dao.entity.User; import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionMapper; import org.apache.dolphinscheduler.dao.mapper.ProjectMapper; import org.apache.dolphinscheduler.dao.mapper.ScheduleMapper; -import org.apache.dolphinscheduler.dao.utils.cron.CronUtils; -import org.apache.dolphinscheduler.dao.quartz.ProcessScheduleJob; -import org.apache.dolphinscheduler.dao.quartz.QuartzExecutors; import com.baomidou.mybatisplus.core.metadata.IPage; import com.baomidou.mybatisplus.extension.plugins.pagination.Page; +import org.apache.dolphinscheduler.service.process.ProcessService; +import org.apache.dolphinscheduler.service.quartz.ProcessScheduleJob; +import org.apache.dolphinscheduler.service.quartz.QuartzExecutors; +import org.apache.dolphinscheduler.service.quartz.cron.CronUtils; import org.quartz.CronExpression; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -68,7 +68,7 @@ public class SchedulerService extends BaseService { private MonitorService monitorService; @Autowired - private ProcessDao processDao; + private ProcessService processService; @Autowired private ScheduleMapper scheduleMapper; @@ -119,7 +119,7 @@ public class SchedulerService extends BaseService { } // check work flow define release state - ProcessDefinition processDefinition = processDao.findProcessDefineById(processDefineId); + ProcessDefinition processDefinition = processService.findProcessDefineById(processDefineId); result = executorService.checkProcessDefinitionValid(processDefinition, processDefineId); if (result.get(Constants.STATUS) != Status.SUCCESS) { return result; @@ -221,7 +221,7 @@ public class SchedulerService extends BaseService { return result; } - ProcessDefinition processDefinition = processDao.findProcessDefineById(schedule.getProcessDefinitionId()); + ProcessDefinition processDefinition = processService.findProcessDefineById(schedule.getProcessDefinitionId()); if (processDefinition == null) { putMsg(result, Status.PROCESS_DEFINE_NOT_EXIST, schedule.getProcessDefinitionId()); return result; @@ -321,7 +321,7 @@ public class SchedulerService extends BaseService { putMsg(result, Status.SCHEDULE_CRON_REALEASE_NEED_NOT_CHANGE, scheduleStatus); return result; } - ProcessDefinition processDefinition = processDao.findProcessDefineById(scheduleObj.getProcessDefinitionId()); + ProcessDefinition processDefinition = processService.findProcessDefineById(scheduleObj.getProcessDefinitionId()); if (processDefinition == null) { putMsg(result, Status.PROCESS_DEFINE_NOT_EXIST, scheduleObj.getProcessDefinitionId()); return result; @@ -338,7 +338,7 @@ public class SchedulerService extends BaseService { } // check sub process definition release state List subProcessDefineIds = new ArrayList<>(); - processDao.recurseFindSubProcessId(scheduleObj.getProcessDefinitionId(), subProcessDefineIds); + processService.recurseFindSubProcessId(scheduleObj.getProcessDefinitionId(), subProcessDefineIds); Integer[] idArray = subProcessDefineIds.toArray(new Integer[subProcessDefineIds.size()]); if (subProcessDefineIds.size() > 0){ List subProcessDefinitionList = @@ -423,7 +423,7 @@ public class SchedulerService extends BaseService { return result; } - ProcessDefinition processDefinition = processDao.findProcessDefineById(processDefineId); + ProcessDefinition processDefinition = processService.findProcessDefineById(processDefineId); if (processDefinition == null) { putMsg(result, Status.PROCESS_DEFINE_NOT_EXIST, processDefineId); return result; @@ -472,7 +472,7 @@ public class SchedulerService extends BaseService { logger.info("set schedule, project id: {}, scheduleId: {}", projectId, scheduleId); - Schedule schedule = processDao.querySchedule(scheduleId); + Schedule schedule = processService.querySchedule(scheduleId); if (schedule == null) { logger.warn("process schedule info not exists"); return; diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskInstanceService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskInstanceService.java index 74afa2a44e..9690f5c69f 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskInstanceService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskInstanceService.java @@ -24,7 +24,6 @@ import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.utils.CollectionUtils; import org.apache.dolphinscheduler.common.utils.DateUtils; import org.apache.dolphinscheduler.common.utils.StringUtils; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.Project; import org.apache.dolphinscheduler.dao.entity.TaskInstance; @@ -33,6 +32,7 @@ import org.apache.dolphinscheduler.dao.mapper.ProjectMapper; import org.apache.dolphinscheduler.dao.mapper.TaskInstanceMapper; import com.baomidou.mybatisplus.core.metadata.IPage; import com.baomidou.mybatisplus.extension.plugins.pagination.Page; +import org.apache.dolphinscheduler.service.process.ProcessService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; @@ -56,7 +56,7 @@ public class TaskInstanceService extends BaseService { ProjectService projectService; @Autowired - ProcessDao processDao; + ProcessService processService; @Autowired TaskInstanceMapper taskInstanceMapper; diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/ZookeeperMonitor.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/ZookeeperMonitor.java index 66f57f6a11..f91d3bc68c 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/ZookeeperMonitor.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/ZookeeperMonitor.java @@ -18,9 +18,9 @@ package org.apache.dolphinscheduler.api.utils; import org.apache.dolphinscheduler.common.enums.ZKNodeType; import org.apache.dolphinscheduler.common.utils.StringUtils; -import org.apache.dolphinscheduler.common.zk.AbstractZKClient; import org.apache.dolphinscheduler.common.model.Server; import org.apache.dolphinscheduler.dao.entity.ZookeeperRecord; +import org.apache.dolphinscheduler.service.zk.AbstractZKClient; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.stereotype.Component; @@ -34,7 +34,7 @@ import java.util.List; * monitor zookeeper info */ @Component -public class ZookeeperMonitor extends AbstractZKClient{ +public class ZookeeperMonitor extends AbstractZKClient { private static final Logger LOG = LoggerFactory.getLogger(ZookeeperMonitor.class); diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ExecutorService2Test.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ExecutorService2Test.java index 66c7a3ebab..07d7477930 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ExecutorService2Test.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ExecutorService2Test.java @@ -22,10 +22,10 @@ import org.apache.dolphinscheduler.common.enums.CommandType; import org.apache.dolphinscheduler.common.enums.Priority; import org.apache.dolphinscheduler.common.enums.ReleaseState; import org.apache.dolphinscheduler.common.enums.RunMode; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.*; import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionMapper; import org.apache.dolphinscheduler.dao.mapper.ProjectMapper; +import org.apache.dolphinscheduler.service.process.ProcessService; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -52,7 +52,7 @@ public class ExecutorService2Test { private ExecutorService executorService; @Mock - private ProcessDao processDao; + private ProcessService processService; @Mock private ProcessDefinitionMapper processDefinitionMapper; @@ -100,8 +100,8 @@ public class ExecutorService2Test { Mockito.when(projectMapper.queryByName(projectName)).thenReturn(project); Mockito.when(projectService.checkProjectAndAuth(loginUser, project, projectName)).thenReturn(checkProjectAndAuth()); Mockito.when(processDefinitionMapper.selectById(processDefinitionId)).thenReturn(processDefinition); - Mockito.when(processDao.getTenantForProcess(tenantId, userId)).thenReturn(new Tenant()); - Mockito.when(processDao.createCommand(any(Command.class))).thenReturn(1); + Mockito.when(processService.getTenantForProcess(tenantId, userId)).thenReturn(new Tenant()); + Mockito.when(processService.createCommand(any(Command.class))).thenReturn(1); } /** @@ -111,7 +111,7 @@ public class ExecutorService2Test { @Test public void testNoComplement() throws ParseException { try { - Mockito.when(processDao.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(zeroSchedulerList()); + Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(zeroSchedulerList()); Map result = executorService.execProcessInstance(loginUser, projectName, processDefinitionId, cronTime, CommandType.START_PROCESS, null, null, @@ -119,7 +119,7 @@ public class ExecutorService2Test { "", "", RunMode.RUN_MODE_SERIAL, Priority.LOW, 0, 110); Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS)); - verify(processDao, times(1)).createCommand(any(Command.class)); + verify(processService, times(1)).createCommand(any(Command.class)); }catch (Exception e){ Assert.assertTrue(false); } @@ -132,7 +132,7 @@ public class ExecutorService2Test { @Test public void testDateError() throws ParseException { try { - Mockito.when(processDao.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(zeroSchedulerList()); + Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(zeroSchedulerList()); Map result = executorService.execProcessInstance(loginUser, projectName, processDefinitionId, "2020-01-31 23:00:00,2020-01-01 00:00:00", CommandType.COMPLEMENT_DATA, null, null, @@ -140,7 +140,7 @@ public class ExecutorService2Test { "", "", RunMode.RUN_MODE_SERIAL, Priority.LOW, 0, 110); Assert.assertEquals(Status.START_PROCESS_INSTANCE_ERROR, result.get(Constants.STATUS)); - verify(processDao, times(0)).createCommand(any(Command.class)); + verify(processService, times(0)).createCommand(any(Command.class)); }catch (Exception e){ Assert.assertTrue(false); } @@ -153,7 +153,7 @@ public class ExecutorService2Test { @Test public void testSerial() throws ParseException { try { - Mockito.when(processDao.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(zeroSchedulerList()); + Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(zeroSchedulerList()); Map result = executorService.execProcessInstance(loginUser, projectName, processDefinitionId, cronTime, CommandType.COMPLEMENT_DATA, null, null, @@ -161,7 +161,7 @@ public class ExecutorService2Test { "", "", RunMode.RUN_MODE_SERIAL, Priority.LOW, 0, 110); Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS)); - verify(processDao, times(1)).createCommand(any(Command.class)); + verify(processService, times(1)).createCommand(any(Command.class)); }catch (Exception e){ Assert.assertTrue(false); } @@ -174,7 +174,7 @@ public class ExecutorService2Test { @Test public void testParallelWithOutSchedule() throws ParseException { try{ - Mockito.when(processDao.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(zeroSchedulerList()); + Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(zeroSchedulerList()); Map result = executorService.execProcessInstance(loginUser, projectName, processDefinitionId, cronTime, CommandType.COMPLEMENT_DATA, null, null, @@ -182,7 +182,7 @@ public class ExecutorService2Test { "", "", RunMode.RUN_MODE_PARALLEL, Priority.LOW, 0, 110); Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS)); - verify(processDao, times(31)).createCommand(any(Command.class)); + verify(processService, times(31)).createCommand(any(Command.class)); }catch (Exception e){ Assert.assertTrue(false); } @@ -195,7 +195,7 @@ public class ExecutorService2Test { @Test public void testParallelWithSchedule() throws ParseException { try{ - Mockito.when(processDao.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(oneSchedulerList()); + Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(oneSchedulerList()); Map result = executorService.execProcessInstance(loginUser, projectName, processDefinitionId, cronTime, CommandType.COMPLEMENT_DATA, null, null, @@ -203,7 +203,7 @@ public class ExecutorService2Test { "", "", RunMode.RUN_MODE_PARALLEL, Priority.LOW, 0, 110); Assert.assertEquals(Status.SUCCESS, result.get(Constants.STATUS)); - verify(processDao, times(15)).createCommand(any(Command.class)); + verify(processService, times(15)).createCommand(any(Command.class)); }catch (Exception e){ Assert.assertTrue(false); } diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionServiceTest.java index b30a919598..ccbbc36ea4 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionServiceTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionServiceTest.java @@ -29,6 +29,7 @@ import org.apache.dolphinscheduler.common.utils.FileUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.dao.entity.*; import org.apache.dolphinscheduler.dao.mapper.*; +import org.apache.dolphinscheduler.service.process.ProcessService; import org.apache.http.entity.ContentType; import org.json.JSONException; import org.junit.Assert; @@ -77,6 +78,9 @@ public class ProcessDefinitionServiceTest { @Mock private WorkerGroupMapper workerGroupMapper; + @Mock + private ProcessService processService; + private String sqlDependentJson = "{\"globalParams\":[]," + "\"tasks\":[{\"type\":\"SQL\",\"id\":\"tasks-27297\",\"name\":\"sql\"," + "\"params\":{\"type\":\"MYSQL\",\"datasource\":1,\"sql\":\"select * from test\"," + @@ -422,6 +426,27 @@ public class ProcessDefinitionServiceTest { Assert.assertTrue(deleteFlag); } + @Test + public void testUpdateProcessDefinition () { + User loginUser = new User(); + loginUser.setId(1); + loginUser.setUserType(UserType.ADMIN_USER); + + Map result = new HashMap<>(5); + putMsg(result, Status.SUCCESS); + + String projectName = "project_test1"; + Project project = getProject(projectName); + + Mockito.when(projectMapper.queryByName(projectName)).thenReturn(getProject(projectName)); + Mockito.when(projectService.checkProjectAndAuth(loginUser, project, projectName)).thenReturn(result); + Mockito.when(processService.findProcessDefineById(1)).thenReturn(getProcessDefinition()); + + Map updateResult = processDefinitionService.updateProcessDefinition(loginUser, projectName, 1, "test", + sqlDependentJson, "", "", ""); + + Assert.assertEquals(Status.UPDATE_PROCESS_DEFINITION_ERROR, updateResult.get(Constants.STATUS)); + } /** * get mock datasource @@ -443,6 +468,8 @@ public class ProcessDefinitionServiceTest { processDefinition.setId(46); processDefinition.setName("testProject"); processDefinition.setProjectId(2); + processDefinition.setTenantId(1); + processDefinition.setDescription(""); return processDefinition; } diff --git a/dolphinscheduler-common/pom.xml b/dolphinscheduler-common/pom.xml index bd2448eee7..9eb4bf7d04 100644 --- a/dolphinscheduler-common/pom.xml +++ b/dolphinscheduler-common/pom.xml @@ -85,21 +85,7 @@ com.fasterxml.jackson.core jackson-databind - - org.apache.curator - curator-client - ${curator.version} - - - log4j-1.2-api - org.apache.logging.log4j - - - io.netty - netty - - - + org.apache.commons commons-collections4 diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/ScriptRunner.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/ScriptRunner.java index bbc937c89f..f92839bfe5 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/ScriptRunner.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/ScriptRunner.java @@ -169,13 +169,13 @@ public class ScriptRunner { if (stopOnError && rs != null) { ResultSetMetaData md = rs.getMetaData(); int cols = md.getColumnCount(); - for (int i = 0; i < cols; i++) { + for (int i = 1; i < cols; i++) { String name = md.getColumnLabel(i); logger.info("{} \t", name); } logger.info(""); while (rs.next()) { - for (int i = 0; i < cols; i++) { + for (int i = 1; i < cols; i++) { String value = rs.getString(i); logger.info("{} \t", value); } diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/StringUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/StringUtils.java index 12b75fb0e5..af2817a8d7 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/StringUtils.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/StringUtils.java @@ -16,17 +16,7 @@ */ package org.apache.dolphinscheduler.common.utils; - -import java.nio.charset.StandardCharsets; -import java.util.Iterator; -import java.util.Objects; -import java.util.regex.Pattern; - - public class StringUtils { - - public static final int INDEX_NOT_FOUND = -1; - public static final String EMPTY = ""; public static boolean isEmpty(final CharSequence cs) { @@ -37,119 +27,14 @@ public class StringUtils { return !isEmpty(cs); } - public static boolean isBlank(CharSequence cs){ - int strLen; - if (cs == null || (strLen = cs.length()) == 0) { + public static boolean isBlank(String s){ + if (isEmpty(s)) { return true; } - for (int i = 0; i < strLen; i++) { - if (Character.isWhitespace(cs.charAt(i)) == false) { - return false; - } - } - return true; - } - - public static boolean isNotBlank(CharSequence str){ - return !isBlank(str); - } - - public static String substringBefore(final String str, final String separator) { - if (isBlank(str) || separator == null) { - return str; - } - if (separator.isEmpty()) { - return EMPTY; - } - final int pos = str.indexOf(separator); - if (pos == INDEX_NOT_FOUND) { - return str; - } - return str.substring(0, pos); - } - - public static String substringAfter(final String str, final String separator) { - if (isBlank(str)) { - return str; - } - if (separator == null) { - return EMPTY; - } - final int pos = str.indexOf(separator); - if (pos == INDEX_NOT_FOUND) { - return EMPTY; - } - return str.substring(pos + separator.length()); + return s.trim().length() == 0; } - public static String substringAfterLast(final String str, final String separator) { - if (isEmpty(str)) { - return str; - } - if (isEmpty(separator)) { - return EMPTY; - } - final int pos = str.lastIndexOf(separator); - if (pos == INDEX_NOT_FOUND || pos == str.length() - separator.length()) { - return EMPTY; - } - return str.substring(pos + separator.length()); - } - - public static String getUtf8String(byte[] bytes){ - return new String(bytes, StandardCharsets.UTF_8); - } - - public static byte[] getUtf8Bytes(String str){ - return str.getBytes(StandardCharsets.UTF_8); - } - - public static boolean hasChinese(String str) { - if (str == null) { - return false; - } - Pattern pattern = Pattern.compile("[\\u4E00-\\u9FBF]+"); - return pattern.matcher(str).find(); - } - - public static boolean hasSpace(String str) { - if (str == null) { - return false; - } - int len = str.length(); - for (int i = 0; i < len; i++) { - if (str.charAt(i) == ' ') { - return true; - } - } - return false; - } - - public static String join(final Iterable iterable, final String separator){ - Iterator iterator = iterable.iterator(); - if (iterator == null) { - return null; - } - if (!iterator.hasNext()) { - return EMPTY; - } - final Object first = iterator.next(); - if (!iterable.iterator().hasNext()) { - return Objects.toString(first, ""); - } - final StringBuilder buf = new StringBuilder(64); - if (first != null) { - buf.append(first); - } - while (iterator.hasNext()) { - if (separator != null) { - buf.append(separator); - } - final Object obj = iterator.next(); - if (obj != null) { - buf.append(obj); - } - } - return buf.toString(); + public static boolean isNotBlank(String s){ + return !isBlank(s); } } diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/ScriptRunnerTest.java b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/ScriptRunnerTest.java index 0eb1cce950..155d52ab75 100644 --- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/ScriptRunnerTest.java +++ b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/ScriptRunnerTest.java @@ -48,7 +48,7 @@ public class ScriptRunnerTest { Mockito.when(st.getResultSet()).thenReturn(rs); ResultSetMetaData md = Mockito.mock(ResultSetMetaData.class); Mockito.when(rs.getMetaData()).thenReturn(md); - Mockito.when(md.getColumnCount()).thenReturn(1); + Mockito.when(md.getColumnCount()).thenReturn(2); Mockito.when(rs.next()).thenReturn(true, false); ScriptRunner s = new ScriptRunner(conn, true, true); if (dbName.isEmpty()) { @@ -56,7 +56,7 @@ public class ScriptRunnerTest { } else { s.runScript(new StringReader("select 1;"), dbName); } - Mockito.verify(md).getColumnLabel(0); + Mockito.verify(md).getColumnLabel(1); } catch(Exception e) { Assert.assertNotNull(e); } diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/StringUtilsTest.java b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/StringUtilsTest.java new file mode 100644 index 0000000000..947e7310db --- /dev/null +++ b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/StringUtilsTest.java @@ -0,0 +1,66 @@ +/* + * 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.common.utils; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; + +public class StringUtilsTest { + @Test + public void testIsNotEmpty() { + //null string + boolean b = StringUtils.isNotEmpty(null); + Assert.assertFalse(b); + + //"" string + b = StringUtils.isNotEmpty(""); + Assert.assertFalse(b); + + //" " string + b = StringUtils.isNotEmpty(" "); + Assert.assertTrue(b); + + //"test" string + b = StringUtils.isNotEmpty("test"); + Assert.assertTrue(b); + } + + @Test + public void testIsNotBlank() { + //null string + boolean b = StringUtils.isNotBlank(null); + Assert.assertFalse(b); + + //"" string + b = StringUtils.isNotBlank(""); + Assert.assertFalse(b); + + //" " string + b = StringUtils.isNotBlank(" "); + Assert.assertFalse(b); + + //" test " string + b = StringUtils.isNotBlank(" test "); + Assert.assertTrue(b); + + //"test" string + b = StringUtils.isNotBlank("test"); + Assert.assertTrue(b); + } +} diff --git a/dolphinscheduler-dao/pom.xml b/dolphinscheduler-dao/pom.xml index b3b22c128d..20d19410e2 100644 --- a/dolphinscheduler-dao/pom.xml +++ b/dolphinscheduler-dao/pom.xml @@ -116,21 +116,6 @@ cron-utils - - org.quartz-scheduler - quartz - - - c3p0 - c3p0 - - - - - - org.quartz-scheduler - quartz-jobs - commons-configuration commons-configuration diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessDefinition.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessDefinition.java index cd0494ecc6..6e7ea7d64f 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessDefinition.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessDefinition.java @@ -29,6 +29,7 @@ import com.baomidou.mybatisplus.core.toolkit.StringUtils; import java.util.Date; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.stream.Collectors; @@ -158,6 +159,11 @@ public class ProcessDefinition { */ private int tenantId; + /** + * modify user name + */ + private String modifyBy; + public String getName() { return name; @@ -337,6 +343,30 @@ public class ProcessDefinition { this.timeout = timeout; } + public int getTenantId() { + return tenantId; + } + + public void setTenantId(int tenantId) { + this.tenantId = tenantId; + } + + public String getDescription() { + return description; + } + + public void setDescription(String description) { + this.description = description; + } + + public String getModifyBy() { + return modifyBy; + } + + public void setModifyBy(String modifyBy) { + this.modifyBy = modifyBy; + } + @Override public String toString() { return "ProcessDefinition{" + @@ -346,6 +376,7 @@ public class ProcessDefinition { ", releaseState=" + releaseState + ", projectId=" + projectId + ", processDefinitionJson='" + processDefinitionJson + '\'' + + ", description='" + description + '\'' + ", globalParams='" + globalParams + '\'' + ", globalParamList=" + globalParamList + ", globalParamMap=" + globalParamMap + @@ -362,22 +393,7 @@ public class ProcessDefinition { ", scheduleReleaseState=" + scheduleReleaseState + ", timeout=" + timeout + ", tenantId=" + tenantId + + ", modifyBy='" + modifyBy + '\'' + '}'; } - - public int getTenantId() { - return tenantId; - } - - public void setTenantId(int tenantId) { - this.tenantId = tenantId; - } - - public String getDescription() { - return description; - } - - public void setDescription(String description) { - this.description = description; - } } diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java index 0647b9450b..9ce59a5aae 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java @@ -24,12 +24,12 @@ import org.apache.dolphinscheduler.common.thread.ThreadUtils; import org.apache.dolphinscheduler.common.utils.OSUtils; import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; import org.apache.dolphinscheduler.common.utils.StringUtils; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.runner.MasterSchedulerThread; -import org.apache.dolphinscheduler.dao.quartz.ProcessScheduleJob; -import org.apache.dolphinscheduler.dao.quartz.QuartzExecutors; import org.apache.dolphinscheduler.server.zk.ZKMasterClient; +import org.apache.dolphinscheduler.service.process.ProcessService; +import org.apache.dolphinscheduler.service.quartz.ProcessScheduleJob; +import org.apache.dolphinscheduler.service.quartz.QuartzExecutors; import org.quartz.SchedulerException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,10 +66,10 @@ public class MasterServer implements IStoppable { private ScheduledExecutorService heartbeatMasterService; /** - * dolphinscheduler database interface + * process service */ @Autowired - protected ProcessDao processDao; + protected ProcessService processService; /** * master exec thread pool @@ -126,7 +126,7 @@ public class MasterServer implements IStoppable { // master scheduler thread MasterSchedulerThread masterSchedulerThread = new MasterSchedulerThread( zkMasterClient, - processDao, + processService, masterConfig.getMasterExecThreads()); // submit master scheduler thread @@ -136,7 +136,7 @@ public class MasterServer implements IStoppable { // what system should do if exception try { logger.info("start Quartz server..."); - ProcessScheduleJob.init(processDao); + ProcessScheduleJob.init(processService); QuartzExecutors.getInstance().start(); } catch (Exception e) { try { diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterBaseTaskExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterBaseTaskExecThread.java index c1552c4621..46541f6ce8 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterBaseTaskExecThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterBaseTaskExecThread.java @@ -16,15 +16,15 @@ */ package org.apache.dolphinscheduler.server.master.runner; -import org.apache.dolphinscheduler.common.queue.ITaskQueue; -import org.apache.dolphinscheduler.common.queue.TaskQueueFactory; import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; import org.apache.dolphinscheduler.dao.AlertDao; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.dao.utils.BeanContext; import org.apache.dolphinscheduler.server.master.config.MasterConfig; +import org.apache.dolphinscheduler.service.process.ProcessService; +import org.apache.dolphinscheduler.service.queue.ITaskQueue; +import org.apache.dolphinscheduler.service.queue.TaskQueueFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,9 +41,9 @@ public class MasterBaseTaskExecThread implements Callable { private static final Logger logger = LoggerFactory.getLogger(MasterBaseTaskExecThread.class); /** - * process dao + * process service */ - protected ProcessDao processDao; + protected ProcessService processService; /** * alert database access @@ -81,7 +81,7 @@ public class MasterBaseTaskExecThread implements Callable { * @param processInstance process instance */ public MasterBaseTaskExecThread(TaskInstance taskInstance, ProcessInstance processInstance){ - this.processDao = BeanContext.getBean(ProcessDao.class); + this.processService = BeanContext.getBean(ProcessService.class); this.alertDao = BeanContext.getBean(AlertDao.class); this.processInstance = processInstance; this.taskQueue = TaskQueueFactory.getTaskQueueInstance(); @@ -121,14 +121,14 @@ public class MasterBaseTaskExecThread implements Callable { try { if(!submitDB){ // submit task to db - task = processDao.submitTask(taskInstance, processInstance); + task = processService.submitTask(taskInstance, processInstance); if(task != null && task.getId() != 0){ submitDB = true; } } if(submitDB && !submitQueue){ // submit task to queue - submitQueue = processDao.submitTaskToQueue(task); + submitQueue = processService.submitTaskToQueue(task); } if(submitDB && submitQueue){ return task; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java index 2b1ff4d23f..b9d21f60c2 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java @@ -28,14 +28,14 @@ import org.apache.dolphinscheduler.common.process.ProcessDag; import org.apache.dolphinscheduler.common.thread.Stopper; import org.apache.dolphinscheduler.common.thread.ThreadUtils; import org.apache.dolphinscheduler.common.utils.*; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.Schedule; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.dao.utils.DagHelper; -import org.apache.dolphinscheduler.dao.utils.cron.CronUtils; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.utils.AlertManager; +import org.apache.dolphinscheduler.service.process.ProcessService; +import org.apache.dolphinscheduler.service.quartz.cron.CronUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -124,9 +124,9 @@ public class MasterExecThread implements Runnable { private DAG dag; /** - * process dao + * process service */ - private ProcessDao processDao; + private ProcessService processService; /** * master config @@ -136,10 +136,10 @@ public class MasterExecThread implements Runnable { /** * constructor of MasterExecThread * @param processInstance process instance - * @param processDao process dao + * @param processService process dao */ - public MasterExecThread(ProcessInstance processInstance,ProcessDao processDao){ - this.processDao = processDao; + public MasterExecThread(ProcessInstance processInstance, ProcessService processService){ + this.processService = processService; this.processInstance = processInstance; this.masterConfig = SpringApplicationContext.getBean(MasterConfig.class); @@ -177,7 +177,7 @@ public class MasterExecThread implements Runnable { logger.error("process execute failed, process id:{}", processInstance.getId()); processInstance.setState(ExecutionStatus.FAILURE); processInstance.setEndTime(new Date()); - processDao.updateProcessInstance(processInstance); + processService.updateProcessInstance(processInstance); }finally { taskExecService.shutdown(); // post handle @@ -205,11 +205,11 @@ public class MasterExecThread implements Runnable { Date startDate = DateUtils.getScheduleDate(cmdParam.get(CMDPARAM_COMPLEMENT_DATA_START_DATE)); Date endDate = DateUtils.getScheduleDate(cmdParam.get(CMDPARAM_COMPLEMENT_DATA_END_DATE)); - processDao.saveProcessInstance(processInstance); + processService.saveProcessInstance(processInstance); // get schedules int processDefinitionId = processInstance.getProcessDefinitionId(); - List schedules = processDao.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId); + List schedules = processService.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId); List listDate = Lists.newLinkedList(); if(!CollectionUtils.isEmpty(schedules)){ for (Schedule schedule : schedules) { @@ -223,7 +223,7 @@ public class MasterExecThread implements Runnable { iterator = listDate.iterator(); scheduleDate = iterator.next(); processInstance.setScheduleTime(scheduleDate); - processDao.updateProcessInstance(processInstance); + processService.updateProcessInstance(processInstance); }else{ scheduleDate = processInstance.getScheduleTime(); if(scheduleDate == null){ @@ -239,7 +239,7 @@ public class MasterExecThread implements Runnable { logger.error("process {} dag is null, please check out parameters", processInstance.getId()); processInstance.setState(ExecutionStatus.SUCCESS); - processDao.updateProcessInstance(processInstance); + processService.updateProcessInstance(processInstance); return; } @@ -281,10 +281,10 @@ public class MasterExecThread implements Runnable { processInstance.setCommandParam(JSONUtils.toJson(cmdParam)); } - List taskInstanceList = processDao.findValidTaskListByProcessId(processInstance.getId()); + List taskInstanceList = processService.findValidTaskListByProcessId(processInstance.getId()); for(TaskInstance taskInstance : taskInstanceList){ taskInstance.setFlag(Flag.NO); - processDao.updateTaskInstance(taskInstance); + processService.updateTaskInstance(taskInstance); } processInstance.setState(ExecutionStatus.RUNNING_EXEUTION); processInstance.setGlobalParams(ParameterUtils.curingGlobalParams( @@ -292,7 +292,7 @@ public class MasterExecThread implements Runnable { processInstance.getProcessDefinition().getGlobalParamList(), CommandType.COMPLEMENT_DATA, processInstance.getScheduleTime())); - processDao.saveProcessInstance(processInstance); + processService.saveProcessInstance(processInstance); } // flow end @@ -320,11 +320,11 @@ public class MasterExecThread implements Runnable { */ private void endProcess() { processInstance.setEndTime(new Date()); - processDao.updateProcessInstance(processInstance); + processService.updateProcessInstance(processInstance); if(processInstance.getState().typeIsWaittingThread()){ - processDao.createRecoveryWaitingThreadCommand(null, processInstance); + processService.createRecoveryWaitingThreadCommand(null, processInstance); } - List taskInstances = processDao.findValidTaskListByProcessId(processInstance.getId()); + List taskInstances = processService.findValidTaskListByProcessId(processInstance.getId()); alertManager.sendAlertProcessInstance(processInstance, taskInstances); } @@ -361,7 +361,7 @@ public class MasterExecThread implements Runnable { dependFailedTask.clear(); completeTaskList.clear(); errorTaskList.clear(); - List taskInstanceList = processDao.findValidTaskListByProcessId(processInstance.getId()); + List taskInstanceList = processService.findValidTaskListByProcessId(processInstance.getId()); for(TaskInstance task : taskInstanceList){ if(task.isTaskComplete()){ completeTaskList.put(task.getName(), task); @@ -417,7 +417,7 @@ public class MasterExecThread implements Runnable { * @return TaskInstance */ private TaskInstance findTaskIfExists(String taskName){ - List taskInstanceList = processDao.findValidTaskListByProcessId(this.processInstance.getId()); + List taskInstanceList = processService.findValidTaskListByProcessId(this.processInstance.getId()); for(TaskInstance taskInstance : taskInstanceList){ if(taskInstance.getName().equals(taskName)){ return taskInstance; @@ -706,7 +706,7 @@ public class MasterExecThread implements Runnable { * @return process instance execution status */ private ExecutionStatus getProcessInstanceState(){ - ProcessInstance instance = processDao.findProcessInstanceById(processInstance.getId()); + ProcessInstance instance = processService.findProcessInstanceById(processInstance.getId()); ExecutionStatus state = instance.getState(); if(activeTaskNode.size() > 0){ @@ -784,10 +784,10 @@ public class MasterExecThread implements Runnable { processInstance.getState().toString(), state.toString(), processInstance.getCommandType().toString()); processInstance.setState(state); - ProcessInstance instance = processDao.findProcessInstanceById(processInstance.getId()); + ProcessInstance instance = processService.findProcessInstanceById(processInstance.getId()); instance.setState(state); instance.setProcessDefinition(processInstance.getProcessDefinition()); - processDao.updateProcessInstance(instance); + processService.updateProcessInstance(instance); processInstance = instance; } } @@ -845,7 +845,7 @@ public class MasterExecThread implements Runnable { // send warning email if process time out. if( !sendTimeWarning && checkProcessTimeOut(processInstance) ){ alertManager.sendProcessTimeoutAlert(processInstance, - processDao.findProcessDefineById(processInstance.getProcessDefinitionId())); + processService.findProcessDefineById(processInstance.getProcessDefinitionId())); sendTimeWarning = true; } for(Map.Entry> entry: activeTaskNode.entrySet()) { @@ -903,7 +903,7 @@ public class MasterExecThread implements Runnable { if(completeTask.getState()== ExecutionStatus.PAUSE){ completeTask.setState(ExecutionStatus.KILL); completeTaskList.put(entry.getKey(), completeTask); - processDao.updateTaskInstance(completeTask); + processService.updateTaskInstance(completeTask); } } } @@ -961,7 +961,7 @@ public class MasterExecThread implements Runnable { Future future = entry.getValue(); TaskInstance taskInstance = taskExecThread.getTaskInstance(); - taskInstance = processDao.findTaskInstanceById(taskInstance.getId()); + taskInstance = processService.findTaskInstanceById(taskInstance.getId()); if(taskInstance.getState().typeIsFinished()){ continue; } @@ -1031,7 +1031,7 @@ public class MasterExecThread implements Runnable { } try { Integer intId = Integer.valueOf(taskId); - TaskInstance task = processDao.findTaskInstanceById(intId); + TaskInstance task = processService.findTaskInstanceById(intId); if(task == null){ logger.error("start node id cannot be found: {}", taskId); }else { diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerThread.java index a873fb786d..84b1432b0d 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerThread.java @@ -23,12 +23,12 @@ import org.apache.dolphinscheduler.common.thread.Stopper; import org.apache.dolphinscheduler.common.thread.ThreadUtils; import org.apache.dolphinscheduler.common.utils.OSUtils; import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; -import org.apache.dolphinscheduler.common.zk.AbstractZKClient; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.Command; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.zk.ZKMasterClient; +import org.apache.dolphinscheduler.service.process.ProcessService; +import org.apache.dolphinscheduler.service.zk.AbstractZKClient; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,7 +53,7 @@ public class MasterSchedulerThread implements Runnable { /** * dolphinscheduler database interface */ - private final ProcessDao processDao; + private final ProcessService processService; /** * zookeeper master client @@ -74,11 +74,11 @@ public class MasterSchedulerThread implements Runnable { /** * constructor of MasterSchedulerThread * @param zkClient zookeeper master client - * @param processDao process dao + * @param processService process service * @param masterExecThreadNum master exec thread num */ - public MasterSchedulerThread(ZKMasterClient zkClient, ProcessDao processDao, int masterExecThreadNum){ - this.processDao = processDao; + public MasterSchedulerThread(ZKMasterClient zkClient, ProcessService processService, int masterExecThreadNum){ + this.processService = processService; this.zkMasterClient = zkClient; this.masterExecThreadNum = masterExecThreadNum; this.masterExecService = ThreadUtils.newDaemonFixedThreadExecutor("Master-Exec-Thread",masterExecThreadNum); @@ -115,19 +115,19 @@ public class MasterSchedulerThread implements Runnable { ThreadPoolExecutor poolExecutor = (ThreadPoolExecutor) masterExecService; int activeCount = poolExecutor.getActiveCount(); // make sure to scan and delete command table in one transaction - Command command = processDao.findOneCommand(); + Command command = processService.findOneCommand(); if (command != null) { logger.info(String.format("find one command: id: %d, type: %s", command.getId(),command.getCommandType().toString())); try{ - processInstance = processDao.handleCommand(logger, OSUtils.getHost(), this.masterExecThreadNum - activeCount, command); + processInstance = processService.handleCommand(logger, OSUtils.getHost(), this.masterExecThreadNum - activeCount, command); if (processInstance != null) { logger.info("start master exec thread , split DAG ..."); - masterExecService.execute(new MasterExecThread(processInstance,processDao)); + masterExecService.execute(new MasterExecThread(processInstance, processService)); } }catch (Exception e){ logger.error("scan command error ", e); - processDao.moveToErrorCommand(command, e.toString()); + processService.moveToErrorCommand(command, e.toString()); } } else{ //indicate that no command ,sleep for 1s diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecThread.java index f2ee66b64a..66d1a3f4c2 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecThread.java @@ -82,7 +82,7 @@ public class MasterTaskExecThread extends MasterBaseTaskExecThread { result = waitTaskQuit(); } taskInstance.setEndTime(new Date()); - processDao.updateTaskInstance(taskInstance); + processService.updateTaskInstance(taskInstance); logger.info("task :{} id:{}, process id:{}, exec thread completed ", this.taskInstance.getName(),taskInstance.getId(), processInstance.getId() ); return result; @@ -94,7 +94,7 @@ public class MasterTaskExecThread extends MasterBaseTaskExecThread { */ public Boolean waitTaskQuit(){ // query new state - taskInstance = processDao.findTaskInstanceById(taskInstance.getId()); + taskInstance = processService.findTaskInstanceById(taskInstance.getId()); logger.info("wait task: process id: {}, task id:{}, task name:{} complete", this.taskInstance.getProcessInstanceId(), this.taskInstance.getId(), this.taskInstance.getName()); // task time out @@ -126,15 +126,15 @@ public class MasterTaskExecThread extends MasterBaseTaskExecThread { if (remainTime < 0) { logger.warn("task id: {} execution time out",taskInstance.getId()); // process define - ProcessDefinition processDefine = processDao.findProcessDefineById(processInstance.getProcessDefinitionId()); + ProcessDefinition processDefine = processService.findProcessDefineById(processInstance.getProcessDefinitionId()); // send warn mail alertDao.sendTaskTimeoutAlert(processInstance.getWarningGroupId(),processDefine.getReceivers(),processDefine.getReceiversCc(),taskInstance.getId(),taskInstance.getName()); checkTimeout = false; } } // updateProcessInstance task instance - taskInstance = processDao.findTaskInstanceById(taskInstance.getId()); - processInstance = processDao.findProcessInstanceById(processInstance.getId()); + taskInstance = processService.findTaskInstanceById(taskInstance.getId()); + processInstance = processService.findProcessInstanceById(processInstance.getId()); Thread.sleep(Constants.SLEEP_TIME_MILLIS); } catch (Exception e) { logger.error("exception",e); diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/SubProcessTaskExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/SubProcessTaskExecThread.java index 0026de7c25..fc16b5112b 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/SubProcessTaskExecThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/SubProcessTaskExecThread.java @@ -64,7 +64,7 @@ public class SubProcessTaskExecThread extends MasterBaseTaskExecThread { } setTaskInstanceState(); waitTaskQuit(); - subProcessInstance = processDao.findSubProcessInstance(processInstance.getId(), taskInstance.getId()); + subProcessInstance = processService.findSubProcessInstance(processInstance.getId(), taskInstance.getId()); // at the end of the subflow , the task state is changed to the subflow state if(subProcessInstance != null){ @@ -75,7 +75,7 @@ public class SubProcessTaskExecThread extends MasterBaseTaskExecThread { } } taskInstance.setEndTime(new Date()); - processDao.updateTaskInstance(taskInstance); + processService.updateTaskInstance(taskInstance); logger.info("subflow task :{} id:{}, process id:{}, exec thread completed ", this.taskInstance.getName(),taskInstance.getId(), processInstance.getId() ); result = true; @@ -96,14 +96,14 @@ public class SubProcessTaskExecThread extends MasterBaseTaskExecThread { * @return */ private Boolean setTaskInstanceState(){ - subProcessInstance = processDao.findSubProcessInstance(processInstance.getId(), taskInstance.getId()); + subProcessInstance = processService.findSubProcessInstance(processInstance.getId(), taskInstance.getId()); if(subProcessInstance == null || taskInstance.getState().typeIsFinished()){ return false; } taskInstance.setState(ExecutionStatus.RUNNING_EXEUTION); taskInstance.setStartTime(new Date()); - processDao.updateTaskInstance(taskInstance); + processService.updateTaskInstance(taskInstance); return true; } @@ -111,7 +111,7 @@ public class SubProcessTaskExecThread extends MasterBaseTaskExecThread { * updateProcessInstance parent state */ private void updateParentProcessState(){ - ProcessInstance parentProcessInstance = processDao.findProcessInstanceById(this.processInstance.getId()); + ProcessInstance parentProcessInstance = processService.findProcessInstanceById(this.processInstance.getId()); if(parentProcessInstance == null){ logger.error("parent work flow instance is null , please check it! work flow id {}", processInstance.getId()); @@ -145,7 +145,7 @@ public class SubProcessTaskExecThread extends MasterBaseTaskExecThread { continue; } } - subProcessInstance = processDao.findProcessInstanceById(subProcessInstance.getId()); + subProcessInstance = processService.findProcessInstanceById(subProcessInstance.getId()); updateParentProcessState(); if (subProcessInstance.getState().typeIsFinished()){ break; @@ -171,7 +171,7 @@ public class SubProcessTaskExecThread extends MasterBaseTaskExecThread { return; } subProcessInstance.setState(ExecutionStatus.READY_STOP); - processDao.updateProcessInstance(subProcessInstance); + processService.updateProcessInstance(subProcessInstance); } /** @@ -183,6 +183,6 @@ public class SubProcessTaskExecThread extends MasterBaseTaskExecThread { return; } subProcessInstance.setState(ExecutionStatus.READY_PAUSE); - processDao.updateProcessInstance(subProcessInstance); + processService.updateProcessInstance(subProcessInstance); } } diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/monitor/ZKMonitorImpl.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/monitor/ZKMonitorImpl.java index 927074012d..5acc8fd931 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/monitor/ZKMonitorImpl.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/monitor/ZKMonitorImpl.java @@ -16,7 +16,7 @@ */ package org.apache.dolphinscheduler.server.monitor; -import org.apache.dolphinscheduler.common.zk.ZookeeperOperator; +import org.apache.dolphinscheduler.service.zk.ZookeeperOperator; import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Component; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/RemoveZKNode.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/RemoveZKNode.java index 7264c2f59d..5550e750b5 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/RemoveZKNode.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/RemoveZKNode.java @@ -16,7 +16,7 @@ */ package org.apache.dolphinscheduler.server.utils; -import org.apache.dolphinscheduler.common.zk.ZookeeperOperator; +import org.apache.dolphinscheduler.service.zk.ZookeeperOperator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/WorkerServer.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/WorkerServer.java index d270880408..92ce585da5 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/WorkerServer.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/WorkerServer.java @@ -22,22 +22,22 @@ import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.IStoppable; import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.enums.TaskType; -import org.apache.dolphinscheduler.common.queue.ITaskQueue; -import org.apache.dolphinscheduler.common.queue.TaskQueueFactory; import org.apache.dolphinscheduler.common.thread.Stopper; import org.apache.dolphinscheduler.common.thread.ThreadPoolExecutors; import org.apache.dolphinscheduler.common.thread.ThreadUtils; import org.apache.dolphinscheduler.common.utils.CollectionUtils; import org.apache.dolphinscheduler.common.utils.OSUtils; import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; -import org.apache.dolphinscheduler.common.zk.AbstractZKClient; import org.apache.dolphinscheduler.dao.AlertDao; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.server.utils.ProcessUtils; import org.apache.dolphinscheduler.server.worker.config.WorkerConfig; import org.apache.dolphinscheduler.server.worker.runner.FetchTaskThread; import org.apache.dolphinscheduler.server.zk.ZKWorkerClient; +import org.apache.dolphinscheduler.service.process.ProcessService; +import org.apache.dolphinscheduler.service.queue.ITaskQueue; +import org.apache.dolphinscheduler.service.queue.TaskQueueFactory; +import org.apache.dolphinscheduler.service.zk.AbstractZKClient; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; @@ -73,10 +73,10 @@ public class WorkerServer implements IStoppable { /** - * process database access + * process service */ @Autowired - private ProcessDao processDao; + private ProcessService processService; /** * alert database access @@ -167,7 +167,7 @@ public class WorkerServer implements IStoppable { killExecutorService.execute(killProcessThread); // new fetch task thread - FetchTaskThread fetchTaskThread = new FetchTaskThread(zkWorkerClient, processDao, taskQueue); + FetchTaskThread fetchTaskThread = new FetchTaskThread(zkWorkerClient, processService, taskQueue); // submit fetch task thread fetchTaskExecutorService.execute(fetchTaskThread); @@ -297,7 +297,7 @@ public class WorkerServer implements IStoppable { Set taskInfoSet = taskQueue.smembers(Constants.DOLPHINSCHEDULER_TASKS_KILL); if (CollectionUtils.isNotEmpty(taskInfoSet)){ for (String taskInfo : taskInfoSet){ - killTask(taskInfo, processDao); + killTask(taskInfo, processService); removeKillInfoFromQueue(taskInfo); } } @@ -319,7 +319,7 @@ public class WorkerServer implements IStoppable { * @param taskInfo task info * @param pd process dao */ - private void killTask(String taskInfo, ProcessDao pd) { + private void killTask(String taskInfo, ProcessService pd) { logger.info("get one kill command from tasks kill queue: " + taskInfo); String[] taskInfoArray = taskInfo.split("-"); if(taskInfoArray.length != 2){ @@ -357,7 +357,7 @@ public class WorkerServer implements IStoppable { * @param taskInstance * @param pd process dao */ - private void deleteTaskFromQueue(TaskInstance taskInstance, ProcessDao pd){ + private void deleteTaskFromQueue(TaskInstance taskInstance, ProcessService pd){ // creating distributed locks, lock path /dolphinscheduler/lock/worker InterProcessMutex mutex = null; logger.info("delete task from tasks queue: " + taskInstance.getId()); diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/FetchTaskThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/FetchTaskThread.java index 221ad069bb..357ac9d92a 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/FetchTaskThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/FetchTaskThread.java @@ -19,17 +19,17 @@ package org.apache.dolphinscheduler.server.worker.runner; import org.apache.curator.framework.recipes.locks.InterProcessMutex; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.ExecutionStatus; -import org.apache.dolphinscheduler.common.queue.ITaskQueue; import org.apache.dolphinscheduler.common.thread.Stopper; import org.apache.dolphinscheduler.common.thread.ThreadUtils; import org.apache.dolphinscheduler.common.utils.*; -import org.apache.dolphinscheduler.common.zk.AbstractZKClient; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.dao.entity.Tenant; import org.apache.dolphinscheduler.dao.entity.WorkerGroup; import org.apache.dolphinscheduler.server.worker.config.WorkerConfig; import org.apache.dolphinscheduler.server.zk.ZKWorkerClient; +import org.apache.dolphinscheduler.service.process.ProcessService; +import org.apache.dolphinscheduler.service.queue.ITaskQueue; +import org.apache.dolphinscheduler.service.zk.AbstractZKClient; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -63,7 +63,7 @@ public class FetchTaskThread implements Runnable{ /** * process database access */ - private final ProcessDao processDao; + private final ProcessService processService; /** * worker thread pool executor @@ -91,10 +91,10 @@ public class FetchTaskThread implements Runnable{ private WorkerConfig workerConfig; public FetchTaskThread(ZKWorkerClient zkWorkerClient, - ProcessDao processDao, + ProcessService processService, ITaskQueue taskQueue){ this.zkWorkerClient = zkWorkerClient; - this.processDao = processDao; + this.processService = processService; this.taskQueue = taskQueue; this.workerConfig = SpringApplicationContext.getBean(WorkerConfig.class); this.taskNum = workerConfig.getWorkerFetchTaskNum(); @@ -112,12 +112,12 @@ public class FetchTaskThread implements Runnable{ */ private boolean checkWorkerGroup(TaskInstance taskInstance, String host){ - int taskWorkerGroupId = processDao.getTaskWorkerGroupId(taskInstance); + int taskWorkerGroupId = processService.getTaskWorkerGroupId(taskInstance); if(taskWorkerGroupId <= 0){ return true; } - WorkerGroup workerGroup = processDao.queryWorkerGroupById(taskWorkerGroupId); + WorkerGroup workerGroup = processService.queryWorkerGroupById(taskWorkerGroupId); if(workerGroup == null ){ logger.info("task {} cannot find the worker group, use all worker instead.", taskInstance.getId()); return true; @@ -184,7 +184,7 @@ public class FetchTaskThread implements Runnable{ // mainly to wait for the master insert task to succeed waitForTaskInstance(); - taskInstance = processDao.getTaskInstanceDetailByTaskId(taskInstId); + taskInstance = processService.getTaskInstanceDetailByTaskId(taskInstId); // verify task instance is null if (verifyTaskInstanceIsNull(taskInstance)) { @@ -200,7 +200,7 @@ public class FetchTaskThread implements Runnable{ // if process definition is null ,process definition already deleted int userId = taskInstance.getProcessDefine() == null ? 0 : taskInstance.getProcessDefine().getUserId(); - Tenant tenant = processDao.getTenantForProcess( + Tenant tenant = processService.getTenantForProcess( taskInstance.getProcessInstance().getTenantId(), userId); @@ -212,7 +212,7 @@ public class FetchTaskThread implements Runnable{ } // set queue for process instance, user-specified queue takes precedence over tenant queue - String userQueue = processDao.queryUserQueueByProcessInstanceId(taskInstance.getProcessInstanceId()); + String userQueue = processService.queryUserQueueByProcessInstanceId(taskInstance.getProcessInstanceId()); taskInstance.getProcessInstance().setQueue(StringUtils.isEmpty(userQueue) ? tenant.getQueue() : userQueue); taskInstance.getProcessInstance().setTenantCode(tenant.getTenantCode()); @@ -234,7 +234,7 @@ public class FetchTaskThread implements Runnable{ logger.info("task : {} ready to submit to task scheduler thread",taskInstId); // submit task - workerExecService.submit(new TaskScheduleThread(taskInstance, processDao)); + workerExecService.submit(new TaskScheduleThread(taskInstance, processService)); // remove node from zk removeNodeFromTaskQueue(taskQueueStr); @@ -259,7 +259,7 @@ public class FetchTaskThread implements Runnable{ removeNodeFromTaskQueue(taskQueueStr); if (taskInstance != null){ - processDao.changeTaskState(ExecutionStatus.FAILURE, + processService.changeTaskState(ExecutionStatus.FAILURE, taskInstance.getStartTime(), taskInstance.getHost(), null, @@ -347,7 +347,7 @@ public class FetchTaskThread implements Runnable{ int retryTimes = 30; while (taskInstance == null && retryTimes > 0) { Thread.sleep(Constants.SLEEP_TIME_MILLIS); - taskInstance = processDao.findTaskInstanceById(taskInstId); + taskInstance = processService.findTaskInstanceById(taskInstId); retryTimes--; } } diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/TaskScheduleThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/TaskScheduleThread.java index f179d6344a..a69cffd58d 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/TaskScheduleThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/TaskScheduleThread.java @@ -31,15 +31,15 @@ import org.apache.dolphinscheduler.common.task.TaskTimeoutParameter; import org.apache.dolphinscheduler.common.utils.CommonUtils; import org.apache.dolphinscheduler.common.utils.HadoopUtils; import org.apache.dolphinscheduler.common.utils.TaskParametersUtils; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.TaskInstance; -import org.apache.dolphinscheduler.dao.permission.PermissionCheck; import org.apache.dolphinscheduler.common.utils.LoggerUtils; import org.apache.dolphinscheduler.common.log.TaskLogDiscriminator; import org.apache.dolphinscheduler.server.worker.task.AbstractTask; import org.apache.dolphinscheduler.server.worker.task.TaskManager; import org.apache.dolphinscheduler.server.worker.task.TaskProps; +import org.apache.dolphinscheduler.service.permission.PermissionCheck; +import org.apache.dolphinscheduler.service.process.ProcessService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -64,9 +64,9 @@ public class TaskScheduleThread implements Runnable { private TaskInstance taskInstance; /** - * process database access + * process service */ - private final ProcessDao processDao; + private final ProcessService processService; /** * abstract task @@ -77,10 +77,10 @@ public class TaskScheduleThread implements Runnable { * constructor * * @param taskInstance task instance - * @param processDao process dao + * @param processService process dao */ - public TaskScheduleThread(TaskInstance taskInstance, ProcessDao processDao){ - this.processDao = processDao; + public TaskScheduleThread(TaskInstance taskInstance, ProcessService processService){ + this.processService = processService; this.taskInstance = taskInstance; } @@ -152,7 +152,7 @@ public class TaskScheduleThread implements Runnable { logger.error("task scheduler failure", e); kill(); // update task instance state - processDao.changeTaskState(ExecutionStatus.FAILURE, + processService.changeTaskState(ExecutionStatus.FAILURE, new Date(), taskInstance.getId()); } @@ -161,7 +161,7 @@ public class TaskScheduleThread implements Runnable { taskInstance.getId(), task.getExitStatus()); // update task instance state - processDao.changeTaskState(task.getExitStatus(), + processService.changeTaskState(task.getExitStatus(), new Date(), taskInstance.getId()); } @@ -191,14 +191,14 @@ public class TaskScheduleThread implements Runnable { // update task status is running if(taskType.equals(TaskType.SQL.name()) || taskType.equals(TaskType.PROCEDURE.name())){ - processDao.changeTaskState(ExecutionStatus.RUNNING_EXEUTION, + processService.changeTaskState(ExecutionStatus.RUNNING_EXEUTION, taskInstance.getStartTime(), taskInstance.getHost(), null, getTaskLogPath(), taskInstance.getId()); }else{ - processDao.changeTaskState(ExecutionStatus.RUNNING_EXEUTION, + processService.changeTaskState(ExecutionStatus.RUNNING_EXEUTION, taskInstance.getStartTime(), taskInstance.getHost(), taskInstance.getExecutePath(), @@ -311,7 +311,7 @@ public class TaskScheduleThread implements Runnable { if (!resFile.exists()) { try { // query the tenant code of the resource according to the name of the resource - String tentnCode = processDao.queryTenantCodeByResName(res); + String tentnCode = processService.queryTenantCodeByResName(res); String resHdfsPath = HadoopUtils.getHdfsFilename(tentnCode, res); logger.info("get resource file from hdfs :{}", resHdfsPath); @@ -334,7 +334,7 @@ public class TaskScheduleThread implements Runnable { private void checkDownloadPermission(List projectRes) throws Exception { int userId = taskInstance.getProcessInstance().getExecutorId(); String[] resNames = projectRes.toArray(new String[projectRes.size()]); - PermissionCheck permissionCheck = new PermissionCheck<>(AuthorizationType.RESOURCE_FILE,processDao,resNames,userId,logger); + PermissionCheck permissionCheck = new PermissionCheck<>(AuthorizationType.RESOURCE_FILE, processService,resNames,userId,logger); permissionCheck.checkPermission(); } } \ No newline at end of file diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractCommandExecutor.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractCommandExecutor.java index 04098215dd..459007e050 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractCommandExecutor.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractCommandExecutor.java @@ -21,10 +21,10 @@ import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.thread.ThreadUtils; import org.apache.dolphinscheduler.common.utils.HadoopUtils; import org.apache.dolphinscheduler.common.utils.StringUtils; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.common.utils.LoggerUtils; import org.apache.dolphinscheduler.server.utils.ProcessUtils; +import org.apache.dolphinscheduler.service.process.ProcessService; import org.slf4j.Logger; import java.io.*; @@ -121,10 +121,10 @@ public abstract class AbstractCommandExecutor { * task specific execution logic * * @param execCommand exec command - * @param processDao process dao + * @param processService process dao * @return exit status code */ - public int run(String execCommand, ProcessDao processDao) { + public int run(String execCommand, ProcessService processService) { int exitStatusCode; try { @@ -147,7 +147,7 @@ public abstract class AbstractCommandExecutor { // get process id int pid = getProcessId(process); - processDao.updatePidByTaskInstId(taskInstId, pid, ""); + processService.updatePidByTaskInstId(taskInstId, pid, ""); logger.info("process start, process id is: {}", pid); @@ -161,10 +161,10 @@ public abstract class AbstractCommandExecutor { exitStatusCode = process.exitValue(); logger.info("process has exited, work dir:{}, pid:{} ,exitStatusCode:{}", taskDir, pid,exitStatusCode); //update process state to db - exitStatusCode = updateState(processDao, exitStatusCode, pid, taskInstId); + exitStatusCode = updateState(processService, exitStatusCode, pid, taskInstId); } else { - TaskInstance taskInstance = processDao.findTaskInstanceById(taskInstId); + TaskInstance taskInstance = processService.findTaskInstanceById(taskInstId); if (taskInstance == null) { logger.error("task instance id:{} not exist", taskInstId); } else { @@ -219,23 +219,23 @@ public abstract class AbstractCommandExecutor { /** * update process state to db * - * @param processDao process dao + * @param processService process dao * @param exitStatusCode exit status code * @param pid process id * @param taskInstId task instance id * @return exit status code */ - private int updateState(ProcessDao processDao, int exitStatusCode, int pid, int taskInstId) { + private int updateState(ProcessService processService, int exitStatusCode, int pid, int taskInstId) { //get yarn state by log if (exitStatusCode == 0) { - TaskInstance taskInstance = processDao.findTaskInstanceById(taskInstId); + TaskInstance taskInstance = processService.findTaskInstanceById(taskInstId); logger.info("process id is {}", pid); List appIds = getAppLinks(taskInstance.getLogPath()); if (appIds.size() > 0) { String appUrl = String.join(Constants.COMMA, appIds); logger.info("yarn log url:{}",appUrl); - processDao.updatePidByTaskInstId(taskInstId, pid, appUrl); + processService.updatePidByTaskInstId(taskInstId, pid, appUrl); } // check if all operations are completed diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractYarnTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractYarnTask.java index 6846617408..3d5f0b8865 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractYarnTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractYarnTask.java @@ -17,9 +17,9 @@ package org.apache.dolphinscheduler.server.worker.task; import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.server.utils.ProcessUtils; +import org.apache.dolphinscheduler.service.process.ProcessService; import org.slf4j.Logger; /** @@ -39,7 +39,7 @@ public abstract class AbstractYarnTask extends AbstractTask { /** * process database access */ - protected ProcessDao processDao; + protected ProcessService processService; /** * Abstract Yarn Task @@ -48,7 +48,7 @@ public abstract class AbstractYarnTask extends AbstractTask { */ public AbstractYarnTask(TaskProps taskProps, Logger logger) { super(taskProps, logger); - this.processDao = SpringApplicationContext.getBean(ProcessDao.class); + this.processService = SpringApplicationContext.getBean(ProcessService.class); this.shellCommandExecutor = new ShellCommandExecutor(this::logHandle, taskProps.getTaskDir(), taskProps.getTaskAppId(), @@ -64,7 +64,7 @@ public abstract class AbstractYarnTask extends AbstractTask { public void handle() throws Exception { try { // construct process - exitStatusCode = shellCommandExecutor.run(buildCommand(), processDao); + exitStatusCode = shellCommandExecutor.run(buildCommand(), processService); } catch (Exception e) { logger.error("yarn process failure", e); exitStatusCode = -1; @@ -82,7 +82,7 @@ public abstract class AbstractYarnTask extends AbstractTask { cancel = true; // cancel process shellCommandExecutor.cancelApplication(); - TaskInstance taskInstance = processDao.findTaskInstanceById(taskProps.getTaskInstId()); + TaskInstance taskInstance = processService.findTaskInstanceById(taskProps.getTaskInstId()); if (status && taskInstance != null){ ProcessUtils.killYarnJob(taskInstance); } diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java index 0de2bbc7c6..4270ef9a69 100755 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java @@ -47,7 +47,6 @@ import org.apache.dolphinscheduler.common.task.datax.DataxParameters; import org.apache.dolphinscheduler.common.utils.CollectionUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.ParameterUtils; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.DataSource; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.server.utils.DataxUtils; @@ -56,6 +55,7 @@ import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; import org.apache.dolphinscheduler.server.worker.task.AbstractTask; import org.apache.dolphinscheduler.server.worker.task.ShellCommandExecutor; import org.apache.dolphinscheduler.server.worker.task.TaskProps; +import org.apache.dolphinscheduler.service.process.ProcessService; import org.slf4j.Logger; import com.alibaba.druid.sql.ast.SQLStatement; @@ -106,9 +106,9 @@ public class DataxTask extends AbstractTask { private ShellCommandExecutor shellCommandExecutor; /** - * process database access + * process dao */ - private ProcessDao processDao; + private ProcessService processService; /** * constructor @@ -128,7 +128,7 @@ public class DataxTask extends AbstractTask { props.getTaskInstId(), props.getTenantCode(), props.getEnvFile(), props.getTaskStartTime(), props.getTaskTimeout(), logger); - this.processDao = SpringApplicationContext.getBean(ProcessDao.class); + this.processService = SpringApplicationContext.getBean(ProcessService.class); } /** @@ -160,7 +160,7 @@ public class DataxTask extends AbstractTask { // run datax process String jsonFilePath = buildDataxJsonFile(); String shellCommandFilePath = buildShellCommandFile(jsonFilePath); - exitStatusCode = shellCommandExecutor.run(shellCommandFilePath, processDao); + exitStatusCode = shellCommandExecutor.run(shellCommandFilePath, processService); } catch (Exception e) { exitStatusCode = -1; @@ -220,11 +220,11 @@ public class DataxTask extends AbstractTask { */ private List buildDataxJobContentJson() throws SQLException { - DataSource dataSource = processDao.findDataSourceById(dataXParameters.getDataSource()); + DataSource dataSource = processService.findDataSourceById(dataXParameters.getDataSource()); BaseDataSource dataSourceCfg = DataSourceFactory.getDatasource(dataSource.getType(), dataSource.getConnectionParams()); - DataSource dataTarget = processDao.findDataSourceById(dataXParameters.getDataTarget()); + DataSource dataTarget = processService.findDataSourceById(dataXParameters.getDataTarget()); BaseDataSource dataTargetCfg = DataSourceFactory.getDatasource(dataTarget.getType(), dataTarget.getConnectionParams()); @@ -355,7 +355,7 @@ public class DataxTask extends AbstractTask { String dataxCommand = sbr.toString(); // find process instance by task id - ProcessInstance processInstance = processDao.findProcessInstanceByTaskId(taskProps.getTaskInstId()); + ProcessInstance processInstance = processService.findProcessInstanceByTaskId(taskProps.getTaskInstId()); // combining local and global parameters Map paramsMap = ParamUtils.convert(taskProps.getUserDefParamsMap(), diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/dependent/DependentExecute.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/dependent/DependentExecute.java index 4be65ed49d..7625c8542b 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/dependent/DependentExecute.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/dependent/DependentExecute.java @@ -24,9 +24,9 @@ import org.apache.dolphinscheduler.common.model.DateInterval; import org.apache.dolphinscheduler.common.model.DependentItem; import org.apache.dolphinscheduler.common.utils.DependentUtils; import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.TaskInstance; +import org.apache.dolphinscheduler.service.process.ProcessService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,9 +37,9 @@ import java.util.*; */ public class DependentExecute { /** - * process dao + * process service */ - private final ProcessDao processDao = SpringApplicationContext.getBean(ProcessDao.class); + private final ProcessService processService = SpringApplicationContext.getBean(ProcessService.class); /** * depend item list @@ -108,7 +108,7 @@ public class DependentExecute { result = getDependResultByState(processInstance.getState()); }else{ TaskInstance taskInstance = null; - List taskInstanceList = processDao.findValidTaskListByProcessId(processInstance.getId()); + List taskInstanceList = processService.findValidTaskListByProcessId(processInstance.getId()); for(TaskInstance task : taskInstanceList){ if(task.getName().equals(dependentItem.getDepTasks())){ @@ -141,16 +141,16 @@ public class DependentExecute { */ private ProcessInstance findLastProcessInterval(int definitionId, DateInterval dateInterval) { - ProcessInstance runningProcess = processDao.findLastRunningProcess(definitionId, dateInterval); + ProcessInstance runningProcess = processService.findLastRunningProcess(definitionId, dateInterval); if(runningProcess != null){ return runningProcess; } - ProcessInstance lastSchedulerProcess = processDao.findLastSchedulerProcessInterval( + ProcessInstance lastSchedulerProcess = processService.findLastSchedulerProcessInterval( definitionId, dateInterval ); - ProcessInstance lastManualProcess = processDao.findLastManualProcessInterval( + ProcessInstance lastManualProcess = processService.findLastManualProcessInterval( definitionId, dateInterval ); diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/dependent/DependentTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/dependent/DependentTask.java index 9af29e01dd..ba0727d422 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/dependent/DependentTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/dependent/DependentTask.java @@ -26,10 +26,10 @@ import org.apache.dolphinscheduler.common.thread.Stopper; import org.apache.dolphinscheduler.common.utils.DependentUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.server.worker.task.AbstractTask; import org.apache.dolphinscheduler.server.worker.task.TaskProps; +import org.apache.dolphinscheduler.service.process.ProcessService; import org.slf4j.Logger; import java.util.*; @@ -63,9 +63,9 @@ public class DependentTask extends AbstractTask { private Date dependentDate; /** - * process dao + * process service */ - private ProcessDao processDao; + private ProcessService processService; /** * constructor @@ -88,7 +88,7 @@ public class DependentTask extends AbstractTask { taskModel.getDependItemList(), taskModel.getRelation())); } - this.processDao = SpringApplicationContext.getBean(ProcessDao.class); + this.processService = SpringApplicationContext.getBean(ProcessService.class); if(taskProps.getScheduleTime() != null){ this.dependentDate = taskProps.getScheduleTime(); @@ -107,7 +107,7 @@ public class DependentTask extends AbstractTask { try{ TaskInstance taskInstance = null; while(Stopper.isRunning()){ - taskInstance = processDao.findTaskInstanceById(this.taskProps.getTaskInstId()); + taskInstance = processService.findTaskInstanceById(this.taskProps.getTaskInstId()); if(taskInstance == null){ exitStatusCode = -1; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/flink/FlinkTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/flink/FlinkTask.java index 0fa9e11ce5..c562fbe4dd 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/flink/FlinkTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/flink/FlinkTask.java @@ -68,7 +68,7 @@ public class FlinkTask extends AbstractYarnTask { if (StringUtils.isNotEmpty(flinkParameters.getMainArgs())) { String args = flinkParameters.getMainArgs(); // get process instance by task instance id - ProcessInstance processInstance = processDao.findProcessInstanceByTaskId(taskProps.getTaskInstId()); + ProcessInstance processInstance = processService.findProcessInstanceByTaskId(taskProps.getTaskInstId()); /** * combining local and global parameters diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/http/HttpTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/http/HttpTask.java index 97e6cb7bee..b1dbd54959 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/http/HttpTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/http/HttpTask.java @@ -30,11 +30,11 @@ import org.apache.dolphinscheduler.common.utils.DateUtils; import org.apache.dolphinscheduler.common.utils.ParameterUtils; import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; import org.apache.dolphinscheduler.common.utils.StringUtils; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.server.utils.ParamUtils; import org.apache.dolphinscheduler.server.worker.task.AbstractTask; import org.apache.dolphinscheduler.server.worker.task.TaskProps; +import org.apache.dolphinscheduler.service.process.ProcessService; import org.apache.http.HttpEntity; import org.apache.http.ParseException; import org.apache.http.client.config.RequestConfig; @@ -66,9 +66,9 @@ public class HttpTask extends AbstractTask { private HttpParameters httpParameters; /** - * process database access + * process service */ - private ProcessDao processDao; + private ProcessService processService; /** * Convert mill seconds to second unit @@ -92,7 +92,7 @@ public class HttpTask extends AbstractTask { */ public HttpTask(TaskProps props, Logger logger) { super(props, logger); - this.processDao = SpringApplicationContext.getBean(ProcessDao.class); + this.processService = SpringApplicationContext.getBean(ProcessService.class); } @Override @@ -138,7 +138,7 @@ public class HttpTask extends AbstractTask { */ protected CloseableHttpResponse sendRequest(CloseableHttpClient client) throws IOException { RequestBuilder builder = createRequestBuilder(); - ProcessInstance processInstance = processDao.findProcessInstanceByTaskId(taskProps.getTaskInstId()); + ProcessInstance processInstance = processService.findProcessInstanceByTaskId(taskProps.getTaskInstId()); Map paramsMap = ParamUtils.convert(taskProps.getUserDefParamsMap(), taskProps.getDefinedParams(), diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java index 9b4952bbd2..745468db06 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java @@ -30,11 +30,11 @@ import org.apache.dolphinscheduler.common.task.procedure.ProcedureParameters; import org.apache.dolphinscheduler.common.utils.CollectionUtils; import org.apache.dolphinscheduler.common.utils.ParameterUtils; import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.DataSource; import org.apache.dolphinscheduler.server.utils.ParamUtils; import org.apache.dolphinscheduler.server.worker.task.AbstractTask; import org.apache.dolphinscheduler.server.worker.task.TaskProps; +import org.apache.dolphinscheduler.service.process.ProcessService; import org.slf4j.Logger; import java.sql.*; @@ -56,9 +56,9 @@ public class ProcedureTask extends AbstractTask { private ProcedureParameters procedureParameters; /** - * process database access + * process service */ - private ProcessDao processDao; + private ProcessService processService; /** * base datasource @@ -82,7 +82,7 @@ public class ProcedureTask extends AbstractTask { throw new RuntimeException("procedure task params is not valid"); } - this.processDao = SpringApplicationContext.getBean(ProcessDao.class); + this.processService = SpringApplicationContext.getBean(ProcessService.class); } @Override @@ -97,7 +97,7 @@ public class ProcedureTask extends AbstractTask { procedureParameters.getMethod(), procedureParameters.getLocalParams()); - DataSource dataSource = processDao.findDataSourceById(procedureParameters.getDatasource()); + DataSource dataSource = processService.findDataSourceById(procedureParameters.getDatasource()); if (dataSource == null){ logger.error("datasource not exists"); exitStatusCode = -1; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/python/PythonTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/python/PythonTask.java index 585d62f154..74dfe3c776 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/python/PythonTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/python/PythonTask.java @@ -23,11 +23,11 @@ import org.apache.dolphinscheduler.common.task.python.PythonParameters; import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.ParameterUtils; import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.server.utils.ParamUtils; import org.apache.dolphinscheduler.server.worker.task.AbstractTask; import org.apache.dolphinscheduler.server.worker.task.PythonCommandExecutor; import org.apache.dolphinscheduler.server.worker.task.TaskProps; +import org.apache.dolphinscheduler.service.process.ProcessService; import org.slf4j.Logger; import java.util.Map; @@ -53,9 +53,9 @@ public class PythonTask extends AbstractTask { private PythonCommandExecutor pythonCommandExecutor; /** - * process database access + * process service */ - private ProcessDao processDao; + private ProcessService processService; /** * constructor @@ -76,7 +76,7 @@ public class PythonTask extends AbstractTask { taskProps.getTaskStartTime(), taskProps.getTaskTimeout(), logger); - this.processDao = SpringApplicationContext.getBean(ProcessDao.class); + this.processService = SpringApplicationContext.getBean(ProcessService.class); } @Override @@ -94,7 +94,7 @@ public class PythonTask extends AbstractTask { public void handle() throws Exception { try { // construct process - exitStatusCode = pythonCommandExecutor.run(buildCommand(), processDao); + exitStatusCode = pythonCommandExecutor.run(buildCommand(), processService); } catch (Exception e) { logger.error("python task failure", e); exitStatusCode = -1; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTask.java index 789a0c5302..32f5833c54 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTask.java @@ -24,11 +24,11 @@ import org.apache.dolphinscheduler.common.task.shell.ShellParameters; import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.ParameterUtils; import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.server.utils.ParamUtils; import org.apache.dolphinscheduler.server.worker.task.AbstractTask; import org.apache.dolphinscheduler.server.worker.task.ShellCommandExecutor; import org.apache.dolphinscheduler.server.worker.task.TaskProps; +import org.apache.dolphinscheduler.service.process.ProcessService; import org.slf4j.Logger; import java.io.File; @@ -64,7 +64,7 @@ public class ShellTask extends AbstractTask { /** * process database access */ - private ProcessDao processDao; + private ProcessService processService; /** * constructor @@ -84,7 +84,7 @@ public class ShellTask extends AbstractTask { taskProps.getTaskStartTime(), taskProps.getTaskTimeout(), logger); - this.processDao = SpringApplicationContext.getBean(ProcessDao.class); + this.processService = SpringApplicationContext.getBean(ProcessService.class); } @Override @@ -102,7 +102,7 @@ public class ShellTask extends AbstractTask { public void handle() throws Exception { try { // construct process - exitStatusCode = shellCommandExecutor.run(buildCommand(), processDao); + exitStatusCode = shellCommandExecutor.run(buildCommand(), processService); } catch (Exception e) { logger.error("shell task failure", e); exitStatusCode = -1; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java index eba05a0d21..bc306b9f06 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java @@ -36,16 +36,16 @@ import org.apache.dolphinscheduler.common.task.sql.SqlParameters; import org.apache.dolphinscheduler.common.task.sql.SqlType; import org.apache.dolphinscheduler.common.utils.*; import org.apache.dolphinscheduler.dao.AlertDao; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.DataSource; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.UdfFunc; import org.apache.dolphinscheduler.dao.entity.User; -import org.apache.dolphinscheduler.dao.permission.PermissionCheck; import org.apache.dolphinscheduler.server.utils.ParamUtils; import org.apache.dolphinscheduler.server.utils.UDFUtils; import org.apache.dolphinscheduler.server.worker.task.AbstractTask; import org.apache.dolphinscheduler.server.worker.task.TaskProps; +import org.apache.dolphinscheduler.service.permission.PermissionCheck; +import org.apache.dolphinscheduler.service.process.ProcessService; import org.slf4j.Logger; import java.sql.*; @@ -67,9 +67,9 @@ public class SqlTask extends AbstractTask { private SqlParameters sqlParameters; /** - * process database access + * process service */ - private ProcessDao processDao; + private ProcessService processService; /** * alert dao @@ -96,7 +96,7 @@ public class SqlTask extends AbstractTask { if (!sqlParameters.checkParameters()) { throw new RuntimeException("sql task params is not valid"); } - this.processDao = SpringApplicationContext.getBean(ProcessDao.class); + this.processService = SpringApplicationContext.getBean(ProcessService.class); this.alertDao = SpringApplicationContext.getBean(AlertDao.class); } @@ -122,7 +122,7 @@ public class SqlTask extends AbstractTask { return; } - dataSource= processDao.findDataSourceById(sqlParameters.getDatasource()); + dataSource= processService.findDataSourceById(sqlParameters.getDatasource()); // data source is null if (dataSource == null){ @@ -171,7 +171,7 @@ public class SqlTask extends AbstractTask { } // check udf permission checkUdfPermission(ArrayUtils.toObject(idsArray)); - List udfFuncList = processDao.queryUdfFunListByids(idsArray); + List udfFuncList = processService.queryUdfFunListByids(idsArray); createFuncs = UDFUtils.createFuncs(udfFuncList, taskProps.getTenantCode(), logger); } @@ -383,7 +383,7 @@ public class SqlTask extends AbstractTask { public void sendAttachment(String title,String content){ // process instance - ProcessInstance instance = processDao.findProcessInstanceByTaskId(taskProps.getTaskInstId()); + ProcessInstance instance = processService.findProcessInstanceByTaskId(taskProps.getTaskInstId()); List users = alertDao.queryUserByAlertGroupId(instance.getWarningGroupId()); @@ -470,10 +470,10 @@ public class SqlTask extends AbstractTask { */ private void checkUdfPermission(Integer[] udfFunIds) throws Exception{ // process instance - ProcessInstance processInstance = processDao.findProcessInstanceByTaskId(taskProps.getTaskInstId()); + ProcessInstance processInstance = processService.findProcessInstanceByTaskId(taskProps.getTaskInstId()); int userId = processInstance.getExecutorId(); - PermissionCheck permissionCheckUdf = new PermissionCheck(AuthorizationType.UDF,processDao,udfFunIds,userId,logger); + PermissionCheck permissionCheckUdf = new PermissionCheck(AuthorizationType.UDF, processService,udfFunIds,userId,logger); permissionCheckUdf.checkPermission(); } @@ -484,10 +484,10 @@ public class SqlTask extends AbstractTask { */ private void checkDataSourcePermission(int dataSourceId) throws Exception{ // process instance - ProcessInstance processInstance = processDao.findProcessInstanceByTaskId(taskProps.getTaskInstId()); + ProcessInstance processInstance = processService.findProcessInstanceByTaskId(taskProps.getTaskInstId()); int userId = processInstance.getExecutorId(); - PermissionCheck permissionCheckDataSource = new PermissionCheck(AuthorizationType.DATASOURCE,processDao,new Integer[]{dataSourceId},userId,logger); + PermissionCheck permissionCheckDataSource = new PermissionCheck(AuthorizationType.DATASOURCE, processService,new Integer[]{dataSourceId},userId,logger); permissionCheckDataSource.checkPermission(); } diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKMasterClient.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKMasterClient.java index c6a71ed066..fe4ec9130a 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKMasterClient.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKMasterClient.java @@ -21,10 +21,8 @@ import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.enums.ZKNodeType; import org.apache.dolphinscheduler.common.model.Server; -import org.apache.dolphinscheduler.common.zk.AbstractZKClient; import org.apache.dolphinscheduler.dao.AlertDao; import org.apache.dolphinscheduler.dao.DaoFactory; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.server.utils.ProcessUtils; @@ -32,6 +30,8 @@ import org.apache.commons.lang.StringUtils; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.locks.InterProcessMutex; import org.apache.curator.utils.ThreadUtils; +import org.apache.dolphinscheduler.service.process.ProcessService; +import org.apache.dolphinscheduler.service.zk.AbstractZKClient; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; @@ -70,10 +70,10 @@ public class ZKMasterClient extends AbstractZKClient { */ private AlertDao alertDao = null; /** - * flow database access + * process service */ @Autowired - private ProcessDao processDao; + private ProcessService processService; /** * default constructor @@ -374,7 +374,7 @@ public class ZKMasterClient extends AbstractZKClient { private void failoverWorker(String workerHost, boolean needCheckWorkerAlive) throws Exception { logger.info("start worker[{}] failover ...", workerHost); - List needFailoverTaskInstanceList = processDao.queryNeedFailoverTaskInstances(workerHost); + List needFailoverTaskInstanceList = processService.queryNeedFailoverTaskInstances(workerHost); for(TaskInstance taskInstance : needFailoverTaskInstanceList){ if(needCheckWorkerAlive){ if(!checkTaskInstanceNeedFailover(taskInstance)){ @@ -382,7 +382,7 @@ public class ZKMasterClient extends AbstractZKClient { } } - ProcessInstance instance = processDao.findProcessInstanceDetailById(taskInstance.getProcessInstanceId()); + ProcessInstance instance = processService.findProcessInstanceDetailById(taskInstance.getProcessInstanceId()); if(instance!=null){ taskInstance.setProcessInstance(instance); } @@ -390,7 +390,7 @@ public class ZKMasterClient extends AbstractZKClient { ProcessUtils.killYarnJob(taskInstance); taskInstance.setState(ExecutionStatus.NEED_FAULT_TOLERANCE); - processDao.saveTaskInstance(taskInstance); + processService.saveTaskInstance(taskInstance); } logger.info("end worker[{}] failover ...", workerHost); } @@ -403,11 +403,11 @@ public class ZKMasterClient extends AbstractZKClient { private void failoverMaster(String masterHost) { logger.info("start master failover ..."); - List needFailoverProcessInstanceList = processDao.queryNeedFailoverProcessInstances(masterHost); + List needFailoverProcessInstanceList = processService.queryNeedFailoverProcessInstances(masterHost); //updateProcessInstance host is null and insert into command for(ProcessInstance processInstance : needFailoverProcessInstanceList){ - processDao.processNeedFailoverProcessInstances(processInstance); + processService.processNeedFailoverProcessInstances(processInstance); } logger.info("master failover end"); diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKWorkerClient.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKWorkerClient.java index 88abfa3071..7ddee3b2a1 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKWorkerClient.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/zk/ZKWorkerClient.java @@ -19,9 +19,9 @@ package org.apache.dolphinscheduler.server.zk; import org.apache.curator.framework.recipes.cache.TreeCacheEvent; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.ZKNodeType; -import org.apache.dolphinscheduler.common.zk.AbstractZKClient; import org.apache.commons.lang.StringUtils; import org.apache.curator.framework.CuratorFramework; +import org.apache.dolphinscheduler.service.zk.AbstractZKClient; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.stereotype.Component; diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterExecThreadTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterExecThreadTest.java index d7c3de13a5..ec6647869a 100644 --- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterExecThreadTest.java +++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterExecThreadTest.java @@ -21,12 +21,12 @@ import org.apache.dolphinscheduler.common.enums.*; import org.apache.dolphinscheduler.common.graph.DAG; import org.apache.dolphinscheduler.common.utils.DateUtils; import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.Schedule; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.runner.MasterExecThread; +import org.apache.dolphinscheduler.service.process.ProcessService; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -57,7 +57,7 @@ public class MasterExecThreadTest { private ProcessInstance processInstance; - private ProcessDao processDao; + private ProcessService processService; private int processDefinitionId = 1; @@ -67,7 +67,7 @@ public class MasterExecThreadTest { @Before public void init() throws Exception{ - processDao = mock(ProcessDao.class); + processService = mock(ProcessService.class); applicationContext = mock(ApplicationContext.class); config = new MasterConfig(); @@ -91,7 +91,7 @@ public class MasterExecThreadTest { processDefinition.setGlobalParamList(Collections.EMPTY_LIST); Mockito.when(processInstance.getProcessDefinition()).thenReturn(processDefinition); - masterExecThread = PowerMockito.spy(new MasterExecThread(processInstance, processDao)); + masterExecThread = PowerMockito.spy(new MasterExecThread(processInstance, processService)); // prepareProcess init dag Field dag = MasterExecThread.class.getDeclaredField("dag"); dag.setAccessible(true); @@ -110,12 +110,12 @@ public class MasterExecThreadTest { @Test public void testParallelWithOutSchedule() throws ParseException { try{ - Mockito.when(processDao.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(zeroSchedulerList()); + Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(zeroSchedulerList()); Method method = MasterExecThread.class.getDeclaredMethod("executeComplementProcess"); method.setAccessible(true); method.invoke(masterExecThread); // one create save, and 1-30 for next save, and last day 31 no save - verify(processDao, times(31)).saveProcessInstance(processInstance); + verify(processService, times(31)).saveProcessInstance(processInstance); }catch (Exception e){ e.printStackTrace(); Assert.assertTrue(false); @@ -129,12 +129,12 @@ public class MasterExecThreadTest { @Test public void testParallelWithSchedule() throws ParseException { try{ - Mockito.when(processDao.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(oneSchedulerList()); + Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId)).thenReturn(oneSchedulerList()); Method method = MasterExecThread.class.getDeclaredMethod("executeComplementProcess"); method.setAccessible(true); method.invoke(masterExecThread); // one create save, and 15(1 to 31 step 2) for next save, and last day 31 no save - verify(processDao, times(15)).saveProcessInstance(processInstance); + verify(processService, times(15)).saveProcessInstance(processInstance); }catch (Exception e){ Assert.assertTrue(false); } diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/shell/ShellCommandExecutorTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/shell/ShellCommandExecutorTest.java index 04c844827f..0ab0b4b92d 100644 --- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/shell/ShellCommandExecutorTest.java +++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/shell/ShellCommandExecutorTest.java @@ -21,12 +21,12 @@ import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.model.TaskNode; import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.common.utils.LoggerUtils; import org.apache.dolphinscheduler.server.worker.task.AbstractTask; import org.apache.dolphinscheduler.server.worker.task.TaskManager; import org.apache.dolphinscheduler.server.worker.task.TaskProps; +import org.apache.dolphinscheduler.service.process.ProcessService; import org.junit.Before; import org.junit.Ignore; import org.junit.Test; @@ -43,11 +43,11 @@ public class ShellCommandExecutorTest { private static final Logger logger = LoggerFactory.getLogger(ShellCommandExecutorTest.class); - private ProcessDao processDao = null; + private ProcessService processService = null; @Before public void before(){ - processDao = SpringApplicationContext.getBean(ProcessDao.class); + processService = SpringApplicationContext.getBean(ProcessService.class); } @Test @@ -65,7 +65,7 @@ public class ShellCommandExecutorTest { - TaskInstance taskInstance = processDao.findTaskInstanceById(7657); + TaskInstance taskInstance = processService.findTaskInstanceById(7657); String taskJson = taskInstance.getTaskJson(); TaskNode taskNode = JSONObject.parseObject(taskJson, TaskNode.class); diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/sql/SqlExecutorTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/sql/SqlExecutorTest.java index 7da3f710b6..c8e92da7e0 100644 --- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/sql/SqlExecutorTest.java +++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/sql/SqlExecutorTest.java @@ -22,12 +22,12 @@ import org.apache.dolphinscheduler.common.enums.CommandType; import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.model.TaskNode; import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.common.utils.LoggerUtils; import org.apache.dolphinscheduler.server.worker.task.AbstractTask; import org.apache.dolphinscheduler.server.worker.task.TaskManager; import org.apache.dolphinscheduler.server.worker.task.TaskProps; +import org.apache.dolphinscheduler.service.process.ProcessService; import org.junit.Before; import org.junit.Ignore; import org.junit.Test; @@ -44,11 +44,11 @@ public class SqlExecutorTest { private static final Logger logger = LoggerFactory.getLogger(SqlExecutorTest.class); - private ProcessDao processDao = null; + private ProcessService processService = null; @Before public void before(){ - processDao = SpringApplicationContext.getBean(ProcessDao.class); + processService = SpringApplicationContext.getBean(ProcessService.class); } @Test @@ -109,7 +109,7 @@ public class SqlExecutorTest { taskProps.setCmdTypeIfComplement(CommandType.START_PROCESS); - TaskInstance taskInstance = processDao.findTaskInstanceById(taskInstId); + TaskInstance taskInstance = processService.findTaskInstanceById(taskInstId); String taskJson = taskInstance.getTaskJson(); TaskNode taskNode = JSONObject.parseObject(taskJson, TaskNode.class); diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTaskTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTaskTest.java index 7a6073e05d..6b71d6dc19 100644 --- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTaskTest.java +++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTaskTest.java @@ -28,12 +28,12 @@ import org.apache.dolphinscheduler.common.enums.DbType; import org.apache.dolphinscheduler.common.job.db.BaseDataSource; import org.apache.dolphinscheduler.common.job.db.DataSourceFactory; import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.DataSource; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.server.utils.DataxUtils; import org.apache.dolphinscheduler.server.worker.task.ShellCommandExecutor; import org.apache.dolphinscheduler.server.worker.task.TaskProps; +import org.apache.dolphinscheduler.service.process.ProcessService; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -53,7 +53,7 @@ public class DataxTaskTest { private DataxTask dataxTask; - private ProcessDao processDao; + private ProcessService processService; private ShellCommandExecutor shellCommandExecutor; @@ -62,13 +62,13 @@ public class DataxTaskTest { @Before public void before() throws Exception { - processDao = Mockito.mock(ProcessDao.class); + processService = Mockito.mock(ProcessService.class); shellCommandExecutor = Mockito.mock(ShellCommandExecutor.class); applicationContext = Mockito.mock(ApplicationContext.class); SpringApplicationContext springApplicationContext = new SpringApplicationContext(); springApplicationContext.setApplicationContext(applicationContext); - Mockito.when(applicationContext.getBean(ProcessDao.class)).thenReturn(processDao); + Mockito.when(applicationContext.getBean(ProcessService.class)).thenReturn(processService); TaskProps props = new TaskProps(); props.setTaskDir("/tmp"); @@ -83,12 +83,12 @@ public class DataxTaskTest { dataxTask = PowerMockito.spy(new DataxTask(props, logger)); dataxTask.init(); - Mockito.when(processDao.findDataSourceById(1)).thenReturn(getDataSource()); - Mockito.when(processDao.findDataSourceById(2)).thenReturn(getDataSource()); - Mockito.when(processDao.findProcessInstanceByTaskId(1)).thenReturn(getProcessInstance()); + Mockito.when(processService.findDataSourceById(1)).thenReturn(getDataSource()); + Mockito.when(processService.findDataSourceById(2)).thenReturn(getDataSource()); + Mockito.when(processService.findProcessInstanceByTaskId(1)).thenReturn(getProcessInstance()); String fileName = String.format("%s/%s_node.sh", props.getTaskDir(), props.getTaskAppId()); - Mockito.when(shellCommandExecutor.run(fileName, processDao)).thenReturn(0); + Mockito.when(shellCommandExecutor.run(fileName, processService)).thenReturn(0); } private DataSource getDataSource() { diff --git a/dolphinscheduler-service/pom.xml b/dolphinscheduler-service/pom.xml index 31a2837254..7d775d5497 100644 --- a/dolphinscheduler-service/pom.xml +++ b/dolphinscheduler-service/pom.xml @@ -12,18 +12,45 @@ dolphinscheduler-service dolphinscheduler-service - http://www.example.com - - - UTF-8 - 1.7 - 1.7 - org.apache.dolphinscheduler dolphinscheduler-remote + + org.apache.dolphinscheduler + dolphinscheduler-dao + + + org.apache.curator + curator-client + ${curator.version} + + + log4j-1.2-api + org.apache.logging.log4j + + + io.netty + netty + + + + + org.quartz-scheduler + quartz + + + c3p0 + c3p0 + + + + + + org.quartz-scheduler + quartz-jobs + diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/MasterResponseCommand.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/MasterResponseCommand.java deleted file mode 100644 index 7607159c7c..0000000000 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/MasterResponseCommand.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * 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.service; - -import org.apache.dolphinscheduler.remote.command.Command; -import org.apache.dolphinscheduler.remote.command.CommandType; -import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer; - -import java.io.Serializable; - -/** - * view log response command - */ -public class MasterResponseCommand implements Serializable { - - private String msg; - - public MasterResponseCommand() { - } - - public MasterResponseCommand(String msg) { - this.msg = msg; - } - - public String getMsg() { - return msg; - } - - public void setMsg(String msg) { - this.msg = msg; - } - - public Command convert2Command(long opaque){ - Command command = new Command(opaque); - command.setType(CommandType.MASTER_RESPONSE); - byte[] body = FastJsonSerializer.serialize(this); - command.setBody(body); - return command; - } -} diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/WorkerRequestCommand.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/WorkerRequestCommand.java deleted file mode 100644 index 419add4d7c..0000000000 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/WorkerRequestCommand.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * 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.service; - -import org.apache.dolphinscheduler.remote.command.Command; -import org.apache.dolphinscheduler.remote.command.CommandType; -import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer; - -import java.io.Serializable; -import java.util.concurrent.atomic.AtomicLong; - -/** - * view log request command - */ -public class WorkerRequestCommand implements Serializable { - - private static final AtomicLong REQUEST = new AtomicLong(1); - - private String path; - - public WorkerRequestCommand() { - } - - public WorkerRequestCommand(String path) { - this.path = path; - } - - public String getPath() { - return path; - } - - public void setPath(String path) { - this.path = path; - } - - public Command convert2Command(){ - Command command = new Command(REQUEST.getAndIncrement()); - command.setType(CommandType.WORKER_REQUEST); - byte[] body = FastJsonSerializer.serialize(this); - command.setBody(body); - return command; - } -} diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/permission/PermissionCheck.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/permission/PermissionCheck.java similarity index 80% rename from dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/permission/PermissionCheck.java rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/permission/PermissionCheck.java index 63d4c1c8af..027666f053 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/permission/PermissionCheck.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/permission/PermissionCheck.java @@ -14,13 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.dao.permission; +package org.apache.dolphinscheduler.service.permission; import org.apache.dolphinscheduler.common.enums.AuthorizationType; import org.apache.dolphinscheduler.common.enums.UserType; import org.apache.dolphinscheduler.common.utils.CollectionUtils; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.User; +import org.apache.dolphinscheduler.service.process.ProcessService; import org.slf4j.Logger; import java.util.List; @@ -38,7 +38,7 @@ public class PermissionCheck { /** * Authorization Type */ - private ProcessDao processDao; + private ProcessService processService; /** * need check array @@ -53,23 +53,23 @@ public class PermissionCheck { /** * permission check * @param authorizationType authorization type - * @param processDao process dao + * @param processService process dao */ - public PermissionCheck(AuthorizationType authorizationType, ProcessDao processDao) { + public PermissionCheck(AuthorizationType authorizationType, ProcessService processService) { this.authorizationType = authorizationType; - this.processDao = processDao; + this.processService = processService; } /** * permission check * @param authorizationType - * @param processDao + * @param processService * @param needChecks * @param userId */ - public PermissionCheck(AuthorizationType authorizationType, ProcessDao processDao, T[] needChecks, int userId) { + public PermissionCheck(AuthorizationType authorizationType, ProcessService processService, T[] needChecks, int userId) { this.authorizationType = authorizationType; - this.processDao = processDao; + this.processService = processService; this.needChecks = needChecks; this.userId = userId; } @@ -77,14 +77,14 @@ public class PermissionCheck { /** * permission check * @param authorizationType - * @param processDao + * @param processService * @param needChecks * @param userId * @param logger */ - public PermissionCheck(AuthorizationType authorizationType, ProcessDao processDao, T[] needChecks, int userId,Logger logger) { + public PermissionCheck(AuthorizationType authorizationType, ProcessService processService, T[] needChecks, int userId, Logger logger) { this.authorizationType = authorizationType; - this.processDao = processDao; + this.processService = processService; this.needChecks = needChecks; this.userId = userId; this.logger = logger; @@ -98,12 +98,12 @@ public class PermissionCheck { this.authorizationType = authorizationType; } - public ProcessDao getProcessDao() { - return processDao; + public ProcessService getProcessService() { + return processService; } - public void setProcessDao(ProcessDao processDao) { - this.processDao = processDao; + public void setProcessService(ProcessService processService) { + this.processService = processService; } public T[] getNeedChecks() { @@ -142,9 +142,9 @@ public class PermissionCheck { public void checkPermission() throws Exception{ if(this.needChecks.length > 0){ // get user type in order to judge whether the user is admin - User user = processDao.getUserById(userId); + User user = processService.getUserById(userId); if (user.getUserType() != UserType.ADMIN_USER){ - List unauthorizedList = processDao.listUnauthorized(userId,needChecks,authorizationType); + List unauthorizedList = processService.listUnauthorized(userId,needChecks,authorizationType); // if exist unauthorized resource if(CollectionUtils.isNotEmpty(unauthorizedList)){ logger.error("user {} didn't has permission of {}: {}", user.getUserName(), authorizationType.getDescp(),unauthorizedList.toString()); diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/ProcessDao.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java similarity index 99% rename from dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/ProcessDao.java rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java index 820b2fdaf4..a26044e417 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/ProcessDao.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.dao; +package org.apache.dolphinscheduler.service.process; import com.alibaba.fastjson.JSONObject; import com.cronutils.model.Cron; @@ -24,16 +24,12 @@ import org.apache.dolphinscheduler.common.enums.*; import org.apache.dolphinscheduler.common.model.DateInterval; import org.apache.dolphinscheduler.common.model.TaskNode; import org.apache.dolphinscheduler.common.process.Property; -import org.apache.dolphinscheduler.common.queue.ITaskQueue; import org.apache.dolphinscheduler.common.task.subprocess.SubProcessParameters; -import org.apache.dolphinscheduler.common.utils.DateUtils; -import org.apache.dolphinscheduler.common.utils.IpUtils; -import org.apache.dolphinscheduler.common.utils.JSONUtils; -import org.apache.dolphinscheduler.common.utils.ParameterUtils; -import org.apache.dolphinscheduler.common.utils.StringUtils; +import org.apache.dolphinscheduler.common.utils.*; import org.apache.dolphinscheduler.dao.entity.*; import org.apache.dolphinscheduler.dao.mapper.*; -import org.apache.dolphinscheduler.dao.utils.cron.CronUtils; +import org.apache.dolphinscheduler.service.quartz.cron.CronUtils; +import org.apache.dolphinscheduler.service.queue.ITaskQueue; import org.quartz.CronExpression; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,7 +47,7 @@ import static org.apache.dolphinscheduler.common.Constants.*; * process relative dao that some mappers in this. */ @Component -public class ProcessDao { +public class ProcessService { private final Logger logger = LoggerFactory.getLogger(getClass()); diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/quartz/DruidConnectionProvider.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/DruidConnectionProvider.java similarity index 99% rename from dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/quartz/DruidConnectionProvider.java rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/DruidConnectionProvider.java index 8a4ceba927..d51e8e82bf 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/quartz/DruidConnectionProvider.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/DruidConnectionProvider.java @@ -14,11 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.dao.quartz; +package org.apache.dolphinscheduler.service.quartz; import com.alibaba.druid.pool.DruidDataSource; import org.quartz.SchedulerException; import org.quartz.utils.ConnectionProvider; + import java.sql.Connection; import java.sql.SQLException; diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/quartz/ProcessScheduleJob.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/ProcessScheduleJob.java similarity index 83% rename from dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/quartz/ProcessScheduleJob.java rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/ProcessScheduleJob.java index ac461296a9..69a80e65f5 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/quartz/ProcessScheduleJob.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/ProcessScheduleJob.java @@ -14,17 +14,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.dao.quartz; +package org.apache.dolphinscheduler.service.quartz; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.CommandType; import org.apache.dolphinscheduler.common.enums.ReleaseState; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.Command; import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; import org.apache.dolphinscheduler.dao.entity.Schedule; -import org.quartz.*; +import org.apache.dolphinscheduler.service.process.ProcessService; +import org.quartz.Job; +import org.quartz.JobDataMap; +import org.quartz.JobExecutionContext; +import org.quartz.JobExecutionException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.util.Assert; @@ -42,17 +45,17 @@ public class ProcessScheduleJob implements Job { private static final Logger logger = LoggerFactory.getLogger(ProcessScheduleJob.class); /** - * process dao + * process service */ - private static ProcessDao processDao; + private static ProcessService processService; /** * init - * @param processDao process dao + * @param processService process dao */ - public static void init(ProcessDao processDao) { - ProcessScheduleJob.processDao = processDao; + public static void init(ProcessService processService) { + ProcessScheduleJob.processService = processService; } /** @@ -64,7 +67,7 @@ public class ProcessScheduleJob implements Job { @Override public void execute(JobExecutionContext context) throws JobExecutionException { - Assert.notNull(processDao, "please call init() method first"); + Assert.notNull(processService, "please call init() method first"); JobDataMap dataMap = context.getJobDetail().getJobDataMap(); @@ -80,7 +83,7 @@ public class ProcessScheduleJob implements Job { logger.info("scheduled fire time :{}, fire time :{}, process id :{}", scheduledFireTime, fireTime, scheduleId); // query schedule - Schedule schedule = processDao.querySchedule(scheduleId); + Schedule schedule = processService.querySchedule(scheduleId); if (schedule == null) { logger.warn("process schedule does not exist in db,delete schedule job in quartz, projectId:{}, scheduleId:{}", projectId, scheduleId); deleteJob(projectId, scheduleId); @@ -88,7 +91,7 @@ public class ProcessScheduleJob implements Job { } - ProcessDefinition processDefinition = processDao.findProcessDefineById(schedule.getProcessDefinitionId()); + ProcessDefinition processDefinition = processService.findProcessDefineById(schedule.getProcessDefinitionId()); // release state : online/offline ReleaseState releaseState = processDefinition.getReleaseState(); if (processDefinition == null || releaseState == ReleaseState.OFFLINE) { @@ -108,7 +111,7 @@ public class ProcessScheduleJob implements Job { command.setWarningType(schedule.getWarningType()); command.setProcessInstancePriority(schedule.getProcessInstancePriority()); - processDao.createCommand(command); + processService.createCommand(command); } diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/quartz/QuartzExecutors.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/QuartzExecutors.java similarity index 99% rename from dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/quartz/QuartzExecutors.java rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/QuartzExecutors.java index 054d7903fc..9d96264a60 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/quartz/QuartzExecutors.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/QuartzExecutors.java @@ -14,12 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.dao.quartz; +package org.apache.dolphinscheduler.service.quartz; +import org.apache.commons.lang.StringUtils; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.dao.entity.Schedule; -import org.apache.commons.lang.StringUtils; import org.quartz.*; import org.quartz.impl.StdSchedulerFactory; import org.quartz.impl.matchers.GroupMatcher; diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/cron/AbstractCycle.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/cron/AbstractCycle.java similarity index 99% rename from dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/cron/AbstractCycle.java rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/cron/AbstractCycle.java index 0cda336d7d..0a2e31b610 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/cron/AbstractCycle.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/cron/AbstractCycle.java @@ -14,13 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.dao.utils.cron; +package org.apache.dolphinscheduler.service.quartz.cron; -import org.apache.dolphinscheduler.common.enums.CycleEnum; import com.cronutils.model.Cron; import com.cronutils.model.field.CronField; import com.cronutils.model.field.CronFieldName; import com.cronutils.model.field.expression.*; +import org.apache.dolphinscheduler.common.enums.CycleEnum; /** * Cycle diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/cron/CronUtils.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/cron/CronUtils.java similarity index 98% rename from dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/cron/CronUtils.java rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/cron/CronUtils.java index 8a9087a33c..2d1d93817e 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/cron/CronUtils.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/cron/CronUtils.java @@ -14,15 +14,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.dao.utils.cron; +package org.apache.dolphinscheduler.service.quartz.cron; -import org.apache.dolphinscheduler.common.enums.CycleEnum; -import org.apache.dolphinscheduler.common.thread.Stopper; -import org.apache.dolphinscheduler.common.utils.DateUtils; import com.cronutils.model.Cron; import com.cronutils.model.definition.CronDefinitionBuilder; import com.cronutils.parser.CronParser; +import org.apache.dolphinscheduler.common.enums.CycleEnum; +import org.apache.dolphinscheduler.common.thread.Stopper; +import org.apache.dolphinscheduler.common.utils.DateUtils; import org.quartz.CronExpression; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -31,7 +31,7 @@ import java.text.ParseException; import java.util.*; import static com.cronutils.model.CronType.QUARTZ; -import static org.apache.dolphinscheduler.dao.utils.cron.CycleFactory.*; +import static org.apache.dolphinscheduler.service.quartz.cron.CycleFactory.*; /** diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/cron/CycleFactory.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/cron/CycleFactory.java similarity index 99% rename from dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/cron/CycleFactory.java rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/cron/CycleFactory.java index 10906b42a3..a956d6c417 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/cron/CycleFactory.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/cron/CycleFactory.java @@ -14,12 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.dao.utils.cron; +package org.apache.dolphinscheduler.service.quartz.cron; -import org.apache.dolphinscheduler.common.enums.CycleEnum; import com.cronutils.model.Cron; import com.cronutils.model.field.expression.Always; import com.cronutils.model.field.expression.QuestionMark; +import org.apache.dolphinscheduler.common.enums.CycleEnum; /** * Crontab Cycle Tool Factory diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/cron/CycleLinks.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/cron/CycleLinks.java similarity index 97% rename from dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/cron/CycleLinks.java rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/cron/CycleLinks.java index 63824bda8e..9f01b18868 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/cron/CycleLinks.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/cron/CycleLinks.java @@ -14,10 +14,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.dao.utils.cron; +package org.apache.dolphinscheduler.service.quartz.cron; -import org.apache.dolphinscheduler.common.enums.CycleEnum; import com.cronutils.model.Cron; +import org.apache.dolphinscheduler.common.enums.CycleEnum; import java.util.ArrayList; import java.util.List; diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/queue/ITaskQueue.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/ITaskQueue.java similarity index 97% rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/queue/ITaskQueue.java rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/ITaskQueue.java index 5beb8111ad..bed8a11247 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/queue/ITaskQueue.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/ITaskQueue.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.queue; +package org.apache.dolphinscheduler.service.queue; import java.util.List; import java.util.Set; diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/queue/TaskQueueFactory.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/TaskQueueFactory.java similarity index 97% rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/queue/TaskQueueFactory.java rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/TaskQueueFactory.java index 0a2d943118..ec0f1573bb 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/queue/TaskQueueFactory.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/TaskQueueFactory.java @@ -14,10 +14,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.queue; +package org.apache.dolphinscheduler.service.queue; -import org.apache.dolphinscheduler.common.utils.CommonUtils; import org.apache.commons.lang.StringUtils; +import org.apache.dolphinscheduler.common.utils.CommonUtils; import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/queue/TaskQueueZkImpl.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/TaskQueueZkImpl.java similarity index 99% rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/queue/TaskQueueZkImpl.java rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/TaskQueueZkImpl.java index d442c13ebc..874512c361 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/queue/TaskQueueZkImpl.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/TaskQueueZkImpl.java @@ -14,13 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.queue; +package org.apache.dolphinscheduler.service.queue; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.utils.IpUtils; import org.apache.dolphinscheduler.common.utils.OSUtils; -import org.apache.dolphinscheduler.common.zk.ZookeeperOperator; +import org.apache.dolphinscheduler.service.zk.ZookeeperOperator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/worker/WorkerClientService.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/worker/WorkerClientService.java deleted file mode 100644 index c1071226a8..0000000000 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/worker/WorkerClientService.java +++ /dev/null @@ -1,107 +0,0 @@ -/* - * 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.service.worker; - -import io.netty.channel.Channel; -import org.apache.dolphinscheduler.remote.NettyRemotingClient; -import org.apache.dolphinscheduler.remote.command.Command; -import org.apache.dolphinscheduler.remote.command.CommandType; -import org.apache.dolphinscheduler.remote.command.log.*; -import org.apache.dolphinscheduler.remote.config.NettyClientConfig; -import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; -import org.apache.dolphinscheduler.remote.utils.Address; -import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer; -import org.apache.dolphinscheduler.service.MasterResponseCommand; -import org.apache.dolphinscheduler.service.WorkerRequestCommand; -import org.apache.dolphinscheduler.service.log.LogPromise; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - -/** - * log client - */ -public class WorkerClientService implements NettyRequestProcessor { - - private static final Logger logger = LoggerFactory.getLogger(WorkerClientService.class); - - private final NettyClientConfig clientConfig; - - private final NettyRemotingClient client; - - private final Address address; - - /** - * request time out - */ - private final long logRequestTimeout = 10 * 1000; - - /** - * construct client - * @param host host - * @param port port - */ - public WorkerClientService(String host, int port) { - this.address = new Address(host, port); - this.clientConfig = new NettyClientConfig(); - this.clientConfig.setWorkerThreads(1); - this.client = new NettyRemotingClient(clientConfig); - this.client.registerProcessor(CommandType.MASTER_RESPONSE, this); - - } - - /** - * shutdown - */ - public void shutdown() { - this.client.close(); - logger.info("logger client shutdown"); - } - - - public String reportResult() { - WorkerRequestCommand request = new WorkerRequestCommand(); - String result = ""; - try { - Command command = request.convert2Command(); - this.client.send(address, command); - LogPromise promise = new LogPromise(command.getOpaque(), logRequestTimeout); - result = ((String)promise.getResult()); - } catch (Exception e) { - e.printStackTrace(); - logger.error("roll view log error", e); - } - return result; - } - - - @Override - public void process(Channel channel, Command command) { - logger.info("received log response : {}", command); - MasterResponseCommand masterResponseCommand = FastJsonSerializer.deserialize( - command.getBody(), MasterResponseCommand.class); - LogPromise.notify(command.getOpaque(), masterResponseCommand.getMsg()); - } - - public static void main(String[] args) throws Exception{ - WorkerClientService workerClientService = new WorkerClientService("192.168.220.247", 1128); - String result = workerClientService.reportResult(); - System.out.println(result); - - } - -} \ No newline at end of file diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/AbstractZKClient.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/AbstractZKClient.java similarity index 99% rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/AbstractZKClient.java rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/AbstractZKClient.java index f62e106680..135bfdabc6 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/AbstractZKClient.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/AbstractZKClient.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.zk; +package org.apache.dolphinscheduler.service.zk; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.imps.CuratorFrameworkState; @@ -31,12 +31,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.*; + import static org.apache.dolphinscheduler.common.Constants.*; /** * abstract zookeeper client */ -public abstract class AbstractZKClient extends ZookeeperCachedOperator{ +public abstract class AbstractZKClient extends ZookeeperCachedOperator { private static final Logger logger = LoggerFactory.getLogger(AbstractZKClient.class); diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/DefaultEnsembleProvider.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/DefaultEnsembleProvider.java similarity index 96% rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/DefaultEnsembleProvider.java rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/DefaultEnsembleProvider.java index 0cf06c0503..9eedf7a4ca 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/DefaultEnsembleProvider.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/DefaultEnsembleProvider.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.zk; +package org.apache.dolphinscheduler.service.zk; import org.apache.curator.ensemble.EnsembleProvider; diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/ZookeeperCachedOperator.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/ZookeeperCachedOperator.java similarity index 90% rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/ZookeeperCachedOperator.java rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/ZookeeperCachedOperator.java index 5aa25552d7..dccb768f8b 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/ZookeeperCachedOperator.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/ZookeeperCachedOperator.java @@ -14,22 +14,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.zk; +package org.apache.dolphinscheduler.service.zk; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.ChildData; import org.apache.curator.framework.recipes.cache.TreeCache; import org.apache.curator.framework.recipes.cache.TreeCacheEvent; -import org.apache.curator.framework.recipes.cache.TreeCacheListener; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.stereotype.Component; import java.nio.charset.StandardCharsets; -import java.util.concurrent.ConcurrentHashMap; - -import static org.apache.dolphinscheduler.common.utils.Preconditions.*; -import static org.apache.dolphinscheduler.common.utils.Preconditions.checkNotNull; @Component public class ZookeeperCachedOperator extends ZookeeperOperator { diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/ZookeeperConfig.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/ZookeeperConfig.java similarity index 98% rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/ZookeeperConfig.java rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/ZookeeperConfig.java index 75a9f6c5f4..c6bdfc3b02 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/ZookeeperConfig.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/ZookeeperConfig.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.zk; +package org.apache.dolphinscheduler.service.zk; import org.springframework.beans.factory.annotation.Value; import org.springframework.context.annotation.PropertySource; diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/ZookeeperOperator.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/ZookeeperOperator.java similarity index 98% rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/ZookeeperOperator.java rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/ZookeeperOperator.java index 9442afd7a0..a2cabce805 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/zk/ZookeeperOperator.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/ZookeeperOperator.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.zk; +package org.apache.dolphinscheduler.service.zk; import org.apache.commons.lang.StringUtils; import org.apache.curator.framework.CuratorFramework; @@ -33,12 +33,10 @@ import org.slf4j.LoggerFactory; import org.springframework.beans.factory.InitializingBean; import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Component; -import org.springframework.util.CollectionUtils; import java.nio.charset.StandardCharsets; import java.util.List; -import static org.apache.dolphinscheduler.common.utils.Preconditions.*; import static org.apache.dolphinscheduler.common.utils.Preconditions.checkNotNull; /** diff --git a/dolphinscheduler-common/src/main/resources/quartz.properties b/dolphinscheduler-service/src/main/resources/quartz.properties similarity index 96% rename from dolphinscheduler-common/src/main/resources/quartz.properties rename to dolphinscheduler-service/src/main/resources/quartz.properties index 2e3a2a0dc1..9c8930b647 100644 --- a/dolphinscheduler-common/src/main/resources/quartz.properties +++ b/dolphinscheduler-service/src/main/resources/quartz.properties @@ -59,6 +59,6 @@ org.quartz.jobStore.dataSource = myDs #============================================================================ # Configure Datasources #============================================================================ -org.quartz.dataSource.myDs.connectionProvider.class = org.apache.dolphinscheduler.dao.quartz.DruidConnectionProvider +org.quartz.dataSource.myDs.connectionProvider.class = org.apache.dolphinscheduler.service.quartz.DruidConnectionProvider org.quartz.dataSource.myDs.maxConnections = 10 org.quartz.dataSource.myDs.validationQuery = select 1 \ No newline at end of file diff --git a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/cron/CronUtilsTest.java b/dolphinscheduler-service/src/test/java/cron/CronUtilsTest.java similarity index 98% rename from dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/cron/CronUtilsTest.java rename to dolphinscheduler-service/src/test/java/cron/CronUtilsTest.java index 1135cf20f5..4ddd5fcae3 100644 --- a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/cron/CronUtilsTest.java +++ b/dolphinscheduler-service/src/test/java/cron/CronUtilsTest.java @@ -14,11 +14,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.dao.cron; +package cron; -import org.apache.dolphinscheduler.common.enums.CycleEnum; -import org.apache.dolphinscheduler.common.utils.DateUtils; -import org.apache.dolphinscheduler.dao.utils.cron.CronUtils; import com.cronutils.builder.CronBuilder; import com.cronutils.model.Cron; import com.cronutils.model.CronType; @@ -26,6 +23,9 @@ import com.cronutils.model.definition.CronDefinitionBuilder; import com.cronutils.model.field.CronField; import com.cronutils.model.field.CronFieldName; import com.cronutils.model.field.expression.*; +import org.apache.dolphinscheduler.common.enums.CycleEnum; +import org.apache.dolphinscheduler.common.utils.DateUtils; +import org.apache.dolphinscheduler.service.quartz.cron.CronUtils; import org.junit.Assert; import org.junit.Test; import org.slf4j.Logger; diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/queue/BaseTaskQueueTest.java b/dolphinscheduler-service/src/test/java/queue/BaseTaskQueueTest.java similarity index 90% rename from dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/queue/BaseTaskQueueTest.java rename to dolphinscheduler-service/src/test/java/queue/BaseTaskQueueTest.java index 433e4fa30f..a0cc457e22 100644 --- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/queue/BaseTaskQueueTest.java +++ b/dolphinscheduler-service/src/test/java/queue/BaseTaskQueueTest.java @@ -14,9 +14,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.queue; +package queue; -import org.apache.dolphinscheduler.common.zk.ZKServer; +import org.apache.dolphinscheduler.service.queue.ITaskQueue; +import org.apache.dolphinscheduler.service.queue.TaskQueueFactory; import org.junit.*; /** diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/queue/TaskQueueZKImplTest.java b/dolphinscheduler-service/src/test/java/queue/TaskQueueZKImplTest.java similarity index 99% rename from dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/queue/TaskQueueZKImplTest.java rename to dolphinscheduler-service/src/test/java/queue/TaskQueueZKImplTest.java index b34a7d6924..d29c5aa610 100644 --- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/queue/TaskQueueZKImplTest.java +++ b/dolphinscheduler-service/src/test/java/queue/TaskQueueZKImplTest.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.queue; +package queue; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.utils.IpUtils; diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/zk/ZKServer.java b/dolphinscheduler-service/src/test/java/queue/ZKServer.java similarity index 99% rename from dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/zk/ZKServer.java rename to dolphinscheduler-service/src/test/java/queue/ZKServer.java index fc39e62ed8..65fb95c02b 100644 --- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/zk/ZKServer.java +++ b/dolphinscheduler-service/src/test/java/queue/ZKServer.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.zk; +package queue; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue index 682dd5b51a..e05de8e880 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue @@ -121,6 +121,7 @@ @@ -128,6 +129,7 @@ @@ -136,6 +138,7 @@ @@ -167,6 +170,7 @@ @@ -174,6 +178,7 @@ @@ -181,6 +186,7 @@ @@ -248,6 +254,8 @@ resourcesList: [], // dependence dependence: {}, + // cache dependence + cacheDependence: {}, // Current node params data params: {}, // Running sign @@ -283,6 +291,12 @@ _onDependent (o) { this.dependence = Object.assign(this.dependence, {}, o) }, + /** + * cache dependent + */ + _onCacheDependent (o) { + this.cacheDependence = Object.assign(this.cacheDependence, {}, o) + }, /** * Task timeout alarm */ @@ -356,9 +370,10 @@ type: this.taskType, id: this.id, name: this.name, + params: this.params, description: this.description, runFlag: this.runFlag, - dependence: this.dependence, + dependence: this.cacheDependence, maxRetryTimes: this.maxRetryTimes, retryInterval: this.retryInterval, timeout: this.timeout, @@ -522,6 +537,7 @@ this.params = o.params || {} this.dependence = o.dependence || {} + this.cacheDependence = o.dependence || {} } this.isContentBox = true @@ -551,7 +567,7 @@ name: this.name, description: this.description, runFlag: this.runFlag, - dependence: this.dependence, + dependence: this.cacheDependence, maxRetryTimes: this.maxRetryTimes, retryInterval: this.retryInterval, timeout: this.timeout, diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/dependent.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/dependent.vue index cca9ec7003..79d127a108 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/dependent.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/dependent.vue @@ -131,6 +131,9 @@ setTimeout(() => { this.isLoading = false }, 600) + }, + cacheDependence (val) { + this.$emit('on-cache-dependent', val) } }, beforeCreate () { @@ -151,7 +154,19 @@ }, destroyed () { }, - computed: {}, + computed: { + cacheDependence () { + return { + relation: this.relation, + dependTaskList: _.map(this.dependTaskList, v => { + return { + relation: v.relation, + dependItemList: _.map(v.dependItemList, v1 => _.omit(v1, ['depTasksList', 'state', 'dateValueList'])) + } + }) + } + } + }, components: { mListBox, mDependItemList } } diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/flink.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/flink.vue index f0d69ec420..03e53fe5e5 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/flink.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/flink.vue @@ -379,7 +379,7 @@ // Non-null objects represent backfill if (!_.isEmpty(o)) { this.mainClass = o.params.mainClass || '' - this.mainJar = o.params.mainJar.res || '' + this.mainJar = o.params.mainJar && o.params.mainJar.res ? o.params.mainJar.res : '' this.deployMode = o.params.deployMode || '' this.slot = o.params.slot || 1 this.taskManager = o.params.taskManager || '2' diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/mr.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/mr.vue index fa73e9bb89..706a35f4fe 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/mr.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/mr.vue @@ -91,6 +91,7 @@ @@ -127,6 +128,8 @@ mainJarList: [], // Resource(list) resourceList: [], + // Cache ResourceList + cacheResourceList: [], // Custom parameter localParams: [], // Command line argument @@ -156,6 +159,12 @@ _onResourcesData (a) { this.resourceList = a }, + /** + * cache resourceList + */ + _onCacheResourcesData (a) { + this.cacheResourceList = a + }, /** * verification */ @@ -220,6 +229,25 @@ if (type === 'PYTHON') { this.mainClass = '' } + }, + //Watch the cacheParams + cacheParams (val) { + this.$emit('on-cache-params', val); + } + }, + computed: { + cacheParams () { + return { + mainClass: this.mainClass, + mainJar: { + res: this.mainJar + }, + resourceList: this.cacheResourceList, + localParams: this.localParams, + mainArgs: this.mainArgs, + others: this.others, + programType: this.programType + } } }, created () { @@ -238,6 +266,7 @@ let resourceList = o.params.resourceList || [] if (resourceList.length) { this.resourceList = resourceList + this.cacheResourceList = resourceList } // backfill localParams diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/procedure.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/procedure.vue index d55f18a2cc..e84f37d7a9 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/procedure.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/procedure.vue @@ -70,7 +70,9 @@ // Data source type type: '', // data source - datasource: '' + datasource: '', + // Return to the selected data source + rtDatasource: '' } }, mixins: [disabledState], @@ -83,7 +85,7 @@ */ _onDsData (o) { this.type = o.type - this.datasource = o.datasource + this.rtDatasource = o.datasource }, /** * return udp @@ -112,14 +114,29 @@ // storage this.$emit('on-params', { type: this.type, - datasource: this.datasource, + datasource: this.rtDatasource, method: this.method, localParams: this.localParams }) return true } }, - watch: {}, + watch: { + //Watch the cacheParams + cacheParams (val) { + this.$emit('on-cache-params', val); + } + }, + computed: { + cacheParams () { + return { + type: this.type, + datasource: this.rtDatasource, + method: this.method, + localParams: this.localParams + } + } + }, created () { let o = this.backfillItem diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/python.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/python.vue index 39a7cd858b..e565b4a6bd 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/python.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/python.vue @@ -31,6 +31,7 @@ @@ -69,7 +70,9 @@ // Custom parameter localParams: [], // resource(list) - resourceList: [] + resourceList: [], + // Cache ResourceList + cacheResourceList: [] } }, mixins: [disabledState], @@ -89,6 +92,12 @@ _onResourcesData (a) { this.resourceList = a }, + /** + * cache resourceList + */ + _onCacheResourcesData (a) { + this.cacheResourceList = a + }, /** * verification */ @@ -142,18 +151,33 @@ return editor } }, - watch: {}, + watch: { + //Watch the cacheParams + cacheParams (val) { + this.$emit('on-cache-params', val); + } + }, + computed: { + cacheParams () { + return { + resourceList: this.cacheResourceList, + localParams: this.localParams, + rawScript: editor ? editor.getValue() : '' + } + } + }, created () { let o = this.backfillItem // Non-null objects represent backfill if (!_.isEmpty(o)) { - this.rawScript = o.params.rawScript + this.rawScript = o.params.rawScript || '' // backfill resourceList let resourceList = o.params.resourceList || [] if (resourceList.length) { this.resourceList = resourceList + this.cacheResourceList = resourceList } // backfill localParams @@ -174,4 +198,4 @@ }, components: { mLocalParams, mListBox, mResources } } - \ No newline at end of file + diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/shell.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/shell.vue index 17184eed75..ad40c586b9 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/shell.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/shell.vue @@ -37,6 +37,7 @@ @@ -75,7 +76,9 @@ // Custom parameter localParams: [], // resource(list) - resourceList: [] + resourceList: [], + // Cache ResourceList + cacheResourceList: [] } }, mixins: [disabledState], @@ -119,11 +122,17 @@ }, /** * return resourceList - * + * */ _onResourcesData (a) { this.resourceList = a }, + /** + * cache resourceList + */ + _onCacheResourcesData (a) { + this.cacheResourceList = a + }, /** * verification */ @@ -175,18 +184,33 @@ return editor } }, - watch: {}, + watch: { + //Watch the cacheParams + cacheParams (val) { + this.$emit('on-cache-params', val); + } + }, + computed: { + cacheParams () { + return { + resourceList: this.cacheResourceList, + localParams: this.localParams, + rawScript: editor ? editor.getValue() : '' + } + } + }, created () { let o = this.backfillItem // Non-null objects represent backfill if (!_.isEmpty(o)) { - this.rawScript = o.params.rawScript + this.rawScript = o.params.rawScript || '' // backfill resourceList let resourceList = o.params.resourceList || [] if (resourceList.length) { this.resourceList = resourceList + this.cacheResourceList = resourceList } // backfill localParams @@ -229,5 +253,5 @@ right: -12px; top: -16px; } - + diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/spark.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/spark.vue index 66a89a4944..feef19856c 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/spark.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/spark.vue @@ -412,7 +412,7 @@ // Non-null objects represent backfill if (!_.isEmpty(o)) { this.mainClass = o.params.mainClass || '' - this.mainJar = o.params.mainJar.res || '' + this.mainJar = o.params.mainJar && o.params.mainJar.res ? o.params.mainJar.res : '' this.deployMode = o.params.deployMode || '' this.driverCores = o.params.driverCores || 1 this.driverMemory = o.params.driverMemory || '512M' diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/sub_process.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/sub_process.vue index ee03513249..477038f18f 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/sub_process.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/sub_process.vue @@ -86,7 +86,13 @@ return _.filter(this.processDefinitionList, v => id === v.id)[0].code } }, - watch: {}, + watch: { + wdiCurr (val) { + this.$emit('on-cache-params', { + processDefinitionId: this.wdiCurr + }) + } + }, created () { let processListS = _.cloneDeep(this.store.state.dag.processListS) let id = this.router.history.current.params.id || null @@ -115,4 +121,4 @@ mounted () { } } - \ No newline at end of file + diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/definition/pages/list/_source/list.vue b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/definition/pages/list/_source/list.vue index 5550864176..53939f3f7b 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/definition/pages/list/_source/list.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/definition/pages/list/_source/list.vue @@ -40,6 +40,9 @@ {{$t('Description')}} + + {{$t('Modify User')}} + {{$t('Timing state')}} @@ -72,6 +75,10 @@ {{item.description}} - + + {{item.modifyBy}} + - + {{$t('offline')}} {{$t('online')}} diff --git a/dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js b/dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js index 6e8c113b30..0402d7e398 100644 --- a/dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js +++ b/dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js @@ -518,4 +518,5 @@ export default { 'SpeedRecord': 'speed(record count)', '0 means unlimited by byte': '0 means unlimited', '0 means unlimited by count': '0 means unlimited', + 'Modify User': 'Modify User' } diff --git a/dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js b/dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js index 52a98773a6..95eb4a1081 100644 --- a/dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js +++ b/dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js @@ -518,4 +518,5 @@ export default { 'SpeedRecord': '限流(记录数)', '0 means unlimited by byte': 'KB,0代表不限制', '0 means unlimited by count': '0代表不限制', + 'Modify User': '修改用户' } diff --git a/pom.xml b/pom.xml index f6009dc689..23f4a24bf2 100644 --- a/pom.xml +++ b/pom.xml @@ -348,6 +348,7 @@ mysql mysql-connector-java ${mysql.connector.version} + test org.slf4j diff --git a/sql/dolphinscheduler-postgre.sql b/sql/dolphinscheduler-postgre.sql index b3c61ebce4..c68fd17be1 100644 --- a/sql/dolphinscheduler-postgre.sql +++ b/sql/dolphinscheduler-postgre.sql @@ -319,6 +319,7 @@ CREATE TABLE t_ds_process_definition ( timeout int DEFAULT '0' , tenant_id int NOT NULL DEFAULT '-1' , update_time timestamp DEFAULT NULL , + modify_by varchar(36) DEFAULT '' , PRIMARY KEY (id) ) ; diff --git a/sql/dolphinscheduler_mysql.sql b/sql/dolphinscheduler_mysql.sql index fec2771fff..ea0f9cb022 100644 --- a/sql/dolphinscheduler_mysql.sql +++ b/sql/dolphinscheduler_mysql.sql @@ -366,6 +366,7 @@ CREATE TABLE `t_ds_process_definition` ( `timeout` int(11) DEFAULT '0' COMMENT 'time out', `tenant_id` int(11) NOT NULL DEFAULT '-1' COMMENT 'tenant id', `update_time` datetime DEFAULT NULL COMMENT 'update time', + `modify_by` varchar(36) DEFAULT '' COMMENT 'modify user', PRIMARY KEY (`id`), KEY `process_definition_index` (`project_id`,`id`) USING BTREE ) ENGINE=InnoDB DEFAULT CHARSET=utf8; diff --git a/sql/upgrade/1.2.2_schema/mysql/dolphinscheduler_ddl.sql b/sql/upgrade/1.2.2_schema/mysql/dolphinscheduler_ddl.sql new file mode 100644 index 0000000000..9fe246a8c2 --- /dev/null +++ b/sql/upgrade/1.2.2_schema/mysql/dolphinscheduler_ddl.sql @@ -0,0 +1,37 @@ +/* + * 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. +*/ + +SET sql_mode=(SELECT REPLACE(@@sql_mode,'ONLY_FULL_GROUP_BY','')); +-- uc_dolphin_T_t_ds_process_definition_A_modify_by +drop PROCEDURE if EXISTS uc_dolphin_T_t_ds_process_definition_A_modify_by; +delimiter d// +CREATE PROCEDURE uc_dolphin_T_t_ds_process_definition_A_modify_by() + BEGIN + IF NOT EXISTS (SELECT 1 FROM information_schema.COLUMNS + WHERE TABLE_NAME='t_ds_process_definition' + AND TABLE_SCHEMA=(SELECT DATABASE()) + AND COLUMN_NAME ='modify_by') + THEN + ALTER TABLE t_ds_process_definition ADD `modify_by` varchar(36) DEFAULT '' COMMENT 'modify user'; + END IF; + END; + +d// + +delimiter ; +CALL uc_dolphin_T_t_ds_process_definition_A_modify_by; +DROP PROCEDURE uc_dolphin_T_t_ds_process_definition_A_modify_by; diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/zk/TestZk.java b/sql/upgrade/1.2.2_schema/mysql/dolphinscheduler_dml.sql similarity index 64% rename from dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/zk/TestZk.java rename to sql/upgrade/1.2.2_schema/mysql/dolphinscheduler_dml.sql index 5c3db2d5d1..38964cc551 100644 --- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/zk/TestZk.java +++ b/sql/upgrade/1.2.2_schema/mysql/dolphinscheduler_dml.sql @@ -13,31 +13,4 @@ * 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.common.zk; - -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -/** - * demo for using zkServer - */ -public class TestZk { - - @Before - public void before(){ - ZKServer.start(); - } - - @Test - public void test(){ - Assert.assertTrue(ZKServer.isStarted()); - } - - @After - public void after(){ - ZKServer.stop(); - } -} +*/ \ No newline at end of file diff --git a/sql/upgrade/1.2.2_schema/postgresql/dolphinscheduler_ddl.sql b/sql/upgrade/1.2.2_schema/postgresql/dolphinscheduler_ddl.sql new file mode 100644 index 0000000000..7fc12900e4 --- /dev/null +++ b/sql/upgrade/1.2.2_schema/postgresql/dolphinscheduler_ddl.sql @@ -0,0 +1,34 @@ +/* + * 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. +*/ +-- uc_dolphin_T_t_ds_process_definition_A_modify_by +delimiter d// +CREATE OR REPLACE FUNCTION uc_dolphin_T_t_ds_process_definition_A_modify_by() RETURNS void AS $$ +BEGIN + IF NOT EXISTS (SELECT 1 FROM information_schema.COLUMNS + WHERE TABLE_NAME='t_ds_process_definition' + AND COLUMN_NAME ='modify_by') + THEN + ALTER TABLE t_ds_process_definition ADD COLUMN modify_by varchar(36) DEFAULT ''; + END IF; +END; +$$ LANGUAGE plpgsql; +d// + +delimiter ; +SELECT uc_dolphin_T_t_ds_process_definition_A_modify_by(); +DROP FUNCTION IF EXISTS uc_dolphin_T_t_ds_process_definition_A_modify_by(); + diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/zk/TestZkServer.java b/sql/upgrade/1.2.2_schema/postgresql/dolphinscheduler_dml.sql similarity index 64% rename from dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/zk/TestZkServer.java rename to sql/upgrade/1.2.2_schema/postgresql/dolphinscheduler_dml.sql index d1a0526309..38964cc551 100644 --- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/zk/TestZkServer.java +++ b/sql/upgrade/1.2.2_schema/postgresql/dolphinscheduler_dml.sql @@ -13,31 +13,4 @@ * 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.common.zk; - -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -/** - * demo for using zkServer - */ -public class TestZkServer { - - @Before - public void before(){ - ZKServer.start(); - } - - @Test - public void test(){ - Assert.assertTrue(ZKServer.isStarted()); - } - - @After - public void after(){ - ZKServer.stop(); - } -} +*/ \ No newline at end of file From e0a5a15d49c325cac0d90de46fc9f0af099be18d Mon Sep 17 00:00:00 2001 From: qiaozhanwei Date: Tue, 11 Feb 2020 16:11:41 +0800 Subject: [PATCH 003/221] 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 --- dolphinscheduler-common/pom.xml | 5 --- .../common/utils/Preconditions.java | 23 +++++------ .../utils/SpringApplicationContext.java | 40 ------------------- .../server/master/MasterServer.java | 13 +++--- .../runner/MasterBaseTaskExecThread.java | 2 +- .../master/runner/MasterExecThread.java | 1 + .../master/runner/MasterSchedulerThread.java | 2 +- .../server/worker/WorkerServer.java | 16 ++++---- .../server/worker/runner/FetchTaskThread.java | 1 + .../server/worker/task/AbstractYarnTask.java | 2 +- .../server/worker/task/datax/DataxTask.java | 2 +- .../task/dependent/DependentExecute.java | 2 +- .../worker/task/dependent/DependentTask.java | 2 +- .../server/worker/task/http/HttpTask.java | 2 +- .../task/processdure/ProcedureTask.java | 2 +- .../server/worker/task/python/PythonTask.java | 2 +- .../server/worker/task/shell/ShellTask.java | 2 +- .../server/worker/task/sql/SqlTask.java | 1 + .../server/master/MasterExecThreadTest.java | 2 +- .../shell/ShellCommandExecutorTest.java | 2 +- .../server/worker/sql/SqlExecutorTest.java | 2 +- .../worker/task/datax/DataxTaskTest.java | 2 +- .../bean/SpringApplicationContext.java | 23 +++++++++++ .../service/queue/TaskQueueFactory.java | 2 +- .../test/java}/utils/PreconditionsTest.java | 5 ++- 25 files changed, 70 insertions(+), 88 deletions(-) delete mode 100644 dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/SpringApplicationContext.java create mode 100644 dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/bean/SpringApplicationContext.java rename {dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common => dolphinscheduler-service/src/test/java}/utils/PreconditionsTest.java (97%) diff --git a/dolphinscheduler-common/pom.xml b/dolphinscheduler-common/pom.xml index 9eb4bf7d04..e1cdfb48fd 100644 --- a/dolphinscheduler-common/pom.xml +++ b/dolphinscheduler-common/pom.xml @@ -587,11 +587,6 @@ compile - - org.springframework - spring-context - - org.codehaus.janino janino diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/Preconditions.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/Preconditions.java index 92337f5de6..e59cbd1b96 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/Preconditions.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/Preconditions.java @@ -16,7 +16,6 @@ */ package org.apache.dolphinscheduler.common.utils; -import org.springframework.lang.Nullable; /** * A collection of static utility methods to validate input. @@ -57,7 +56,7 @@ public final class Preconditions { * * @throws NullPointerException Thrown, if the passed reference was null. */ - public static T checkNotNull(T reference, @Nullable String errorMessage) { + public static T checkNotNull(T reference, String errorMessage) { if (reference == null) { throw new NullPointerException(String.valueOf(errorMessage)); } @@ -84,8 +83,8 @@ public final class Preconditions { * @throws NullPointerException Thrown, if the passed reference was null. */ public static T checkNotNull(T reference, - @Nullable String errorMessageTemplate, - @Nullable Object... errorMessageArgs) { + String errorMessageTemplate, + Object... errorMessageArgs) { if (reference == null) { throw new NullPointerException(format(errorMessageTemplate, errorMessageArgs)); @@ -121,7 +120,7 @@ public final class Preconditions { * * @throws IllegalArgumentException Thrown, if the condition is violated. */ - public static void checkArgument(boolean condition, @Nullable Object errorMessage) { + public static void checkArgument(boolean condition, Object errorMessage) { if (!condition) { throw new IllegalArgumentException(String.valueOf(errorMessage)); } @@ -141,8 +140,8 @@ public final class Preconditions { * @throws IllegalArgumentException Thrown, if the condition is violated. */ public static void checkArgument(boolean condition, - @Nullable String errorMessageTemplate, - @Nullable Object... errorMessageArgs) { + String errorMessageTemplate, + Object... errorMessageArgs) { if (!condition) { throw new IllegalArgumentException(format(errorMessageTemplate, errorMessageArgs)); @@ -177,7 +176,7 @@ public final class Preconditions { * * @throws IllegalStateException Thrown, if the condition is violated. */ - public static void checkState(boolean condition, @Nullable Object errorMessage) { + public static void checkState(boolean condition, Object errorMessage) { if (!condition) { throw new IllegalStateException(String.valueOf(errorMessage)); } @@ -197,8 +196,8 @@ public final class Preconditions { * @throws IllegalStateException Thrown, if the condition is violated. */ public static void checkState(boolean condition, - @Nullable String errorMessageTemplate, - @Nullable Object... errorMessageArgs) { + String errorMessageTemplate, + Object... errorMessageArgs) { if (!condition) { throw new IllegalStateException(format(errorMessageTemplate, errorMessageArgs)); @@ -231,7 +230,7 @@ public final class Preconditions { * @throws IllegalArgumentException Thrown, if size is negative. * @throws IndexOutOfBoundsException Thrown, if the index negative or greater than or equal to size */ - public static void checkElementIndex(int index, int size, @Nullable String errorMessage) { + public static void checkElementIndex(int index, int size, String errorMessage) { checkArgument(size >= 0, "Size was negative."); if (index < 0 || index >= size) { throw new IndexOutOfBoundsException(String.valueOf(errorMessage) + " Index: " + index + ", Size: " + size); @@ -248,7 +247,7 @@ public final class Preconditions { * *

This method is taken quasi verbatim from the Guava Preconditions class. */ - private static String format(@Nullable String template, @Nullable Object... args) { + private static String format( String template, Object... args) { final int numArgs = args == null ? 0 : args.length; template = String.valueOf(template); // null -> "null" diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/SpringApplicationContext.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/SpringApplicationContext.java deleted file mode 100644 index 97618e1b39..0000000000 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/SpringApplicationContext.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * 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.common.utils; - -import org.springframework.beans.BeansException; -import org.springframework.context.ApplicationContext; -import org.springframework.context.ApplicationContextAware; -import org.springframework.stereotype.Component; - - -@Component -public class SpringApplicationContext implements ApplicationContextAware { - - private static ApplicationContext applicationContext; - - @Override - public void setApplicationContext(ApplicationContext applicationContext) throws BeansException { - SpringApplicationContext.applicationContext = applicationContext; - } - - public static T getBean(Class requiredType){ - return applicationContext.getBean(requiredType); - } - - -} diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java index 9ce59a5aae..13ab019ef2 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java @@ -22,11 +22,11 @@ import org.apache.dolphinscheduler.common.thread.Stopper; import org.apache.dolphinscheduler.common.thread.ThreadPoolExecutors; import org.apache.dolphinscheduler.common.thread.ThreadUtils; import org.apache.dolphinscheduler.common.utils.OSUtils; -import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; import org.apache.dolphinscheduler.common.utils.StringUtils; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.runner.MasterSchedulerThread; import org.apache.dolphinscheduler.server.zk.ZKMasterClient; +import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.process.ProcessService; import org.apache.dolphinscheduler.service.quartz.ProcessScheduleJob; import org.apache.dolphinscheduler.service.quartz.QuartzExecutors; @@ -77,17 +77,18 @@ public class MasterServer implements IStoppable { private ExecutorService masterSchedulerService; /** - * spring application context - * only use it for initialization + * master config */ @Autowired - private SpringApplicationContext springApplicationContext; + private MasterConfig masterConfig; + /** - * master config + * spring application context + * only use it for initialization */ @Autowired - private MasterConfig masterConfig; + private SpringApplicationContext springApplicationContext; /** diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterBaseTaskExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterBaseTaskExecThread.java index 46541f6ce8..f8fcb1456d 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterBaseTaskExecThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterBaseTaskExecThread.java @@ -16,12 +16,12 @@ */ package org.apache.dolphinscheduler.server.master.runner; -import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; import org.apache.dolphinscheduler.dao.AlertDao; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.dao.utils.BeanContext; import org.apache.dolphinscheduler.server.master.config.MasterConfig; +import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.process.ProcessService; import org.apache.dolphinscheduler.service.queue.ITaskQueue; import org.apache.dolphinscheduler.service.queue.TaskQueueFactory; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java index b9d21f60c2..f5e31210a0 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java @@ -34,6 +34,7 @@ import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.dao.utils.DagHelper; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.utils.AlertManager; +import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.process.ProcessService; import org.apache.dolphinscheduler.service.quartz.cron.CronUtils; import org.slf4j.Logger; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerThread.java index 84b1432b0d..c0ddb1cb5c 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerThread.java @@ -22,11 +22,11 @@ import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.thread.Stopper; import org.apache.dolphinscheduler.common.thread.ThreadUtils; import org.apache.dolphinscheduler.common.utils.OSUtils; -import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; import org.apache.dolphinscheduler.dao.entity.Command; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.zk.ZKMasterClient; +import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.process.ProcessService; import org.apache.dolphinscheduler.service.zk.AbstractZKClient; import org.slf4j.Logger; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/WorkerServer.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/WorkerServer.java index 92ce585da5..0fc667255c 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/WorkerServer.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/WorkerServer.java @@ -27,13 +27,13 @@ import org.apache.dolphinscheduler.common.thread.ThreadPoolExecutors; import org.apache.dolphinscheduler.common.thread.ThreadUtils; import org.apache.dolphinscheduler.common.utils.CollectionUtils; import org.apache.dolphinscheduler.common.utils.OSUtils; -import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; import org.apache.dolphinscheduler.dao.AlertDao; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.server.utils.ProcessUtils; import org.apache.dolphinscheduler.server.worker.config.WorkerConfig; import org.apache.dolphinscheduler.server.worker.runner.FetchTaskThread; import org.apache.dolphinscheduler.server.zk.ZKWorkerClient; +import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.process.ProcessService; import org.apache.dolphinscheduler.service.queue.ITaskQueue; import org.apache.dolphinscheduler.service.queue.TaskQueueFactory; @@ -104,13 +104,6 @@ public class WorkerServer implements IStoppable { */ private ExecutorService fetchTaskExecutorService; - /** - * spring application context - * only use it for initialization - */ - @Autowired - private SpringApplicationContext springApplicationContext; - /** * CountDownLatch latch */ @@ -122,6 +115,13 @@ public class WorkerServer implements IStoppable { @Autowired private WorkerConfig workerConfig; + /** + * spring application context + * only use it for initialization + */ + @Autowired + private SpringApplicationContext springApplicationContext; + /** * master server startup * diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/FetchTaskThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/FetchTaskThread.java index 357ac9d92a..013db83761 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/FetchTaskThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/FetchTaskThread.java @@ -27,6 +27,7 @@ import org.apache.dolphinscheduler.dao.entity.Tenant; import org.apache.dolphinscheduler.dao.entity.WorkerGroup; import org.apache.dolphinscheduler.server.worker.config.WorkerConfig; import org.apache.dolphinscheduler.server.zk.ZKWorkerClient; +import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.process.ProcessService; import org.apache.dolphinscheduler.service.queue.ITaskQueue; import org.apache.dolphinscheduler.service.zk.AbstractZKClient; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractYarnTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractYarnTask.java index 3d5f0b8865..39f4dfbb97 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractYarnTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractYarnTask.java @@ -16,9 +16,9 @@ */ package org.apache.dolphinscheduler.server.worker.task; -import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.server.utils.ProcessUtils; +import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.process.ProcessService; import org.slf4j.Logger; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java index 4270ef9a69..e731ae9879 100755 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java @@ -51,10 +51,10 @@ import org.apache.dolphinscheduler.dao.entity.DataSource; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.server.utils.DataxUtils; import org.apache.dolphinscheduler.server.utils.ParamUtils; -import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; import org.apache.dolphinscheduler.server.worker.task.AbstractTask; import org.apache.dolphinscheduler.server.worker.task.ShellCommandExecutor; import org.apache.dolphinscheduler.server.worker.task.TaskProps; +import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.process.ProcessService; import org.slf4j.Logger; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/dependent/DependentExecute.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/dependent/DependentExecute.java index 7625c8542b..b08cabc2e9 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/dependent/DependentExecute.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/dependent/DependentExecute.java @@ -23,9 +23,9 @@ import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.model.DateInterval; import org.apache.dolphinscheduler.common.model.DependentItem; import org.apache.dolphinscheduler.common.utils.DependentUtils; -import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.TaskInstance; +import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.process.ProcessService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/dependent/DependentTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/dependent/DependentTask.java index ba0727d422..f074d57e6c 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/dependent/DependentTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/dependent/DependentTask.java @@ -25,10 +25,10 @@ import org.apache.dolphinscheduler.common.task.dependent.DependentParameters; import org.apache.dolphinscheduler.common.thread.Stopper; import org.apache.dolphinscheduler.common.utils.DependentUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils; -import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.server.worker.task.AbstractTask; import org.apache.dolphinscheduler.server.worker.task.TaskProps; +import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.process.ProcessService; import org.slf4j.Logger; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/http/HttpTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/http/HttpTask.java index b1dbd54959..c925f90b9e 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/http/HttpTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/http/HttpTask.java @@ -28,12 +28,12 @@ import org.apache.dolphinscheduler.common.task.AbstractParameters; import org.apache.dolphinscheduler.common.task.http.HttpParameters; import org.apache.dolphinscheduler.common.utils.DateUtils; import org.apache.dolphinscheduler.common.utils.ParameterUtils; -import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; import org.apache.dolphinscheduler.common.utils.StringUtils; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.server.utils.ParamUtils; import org.apache.dolphinscheduler.server.worker.task.AbstractTask; import org.apache.dolphinscheduler.server.worker.task.TaskProps; +import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.process.ProcessService; import org.apache.http.HttpEntity; import org.apache.http.ParseException; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java index 745468db06..3898357f85 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java @@ -29,11 +29,11 @@ import org.apache.dolphinscheduler.common.task.AbstractParameters; import org.apache.dolphinscheduler.common.task.procedure.ProcedureParameters; import org.apache.dolphinscheduler.common.utils.CollectionUtils; import org.apache.dolphinscheduler.common.utils.ParameterUtils; -import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; import org.apache.dolphinscheduler.dao.entity.DataSource; import org.apache.dolphinscheduler.server.utils.ParamUtils; import org.apache.dolphinscheduler.server.worker.task.AbstractTask; import org.apache.dolphinscheduler.server.worker.task.TaskProps; +import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.process.ProcessService; import org.slf4j.Logger; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/python/PythonTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/python/PythonTask.java index 74dfe3c776..fc212f866b 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/python/PythonTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/python/PythonTask.java @@ -22,11 +22,11 @@ import org.apache.dolphinscheduler.common.task.AbstractParameters; import org.apache.dolphinscheduler.common.task.python.PythonParameters; import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.ParameterUtils; -import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; import org.apache.dolphinscheduler.server.utils.ParamUtils; import org.apache.dolphinscheduler.server.worker.task.AbstractTask; import org.apache.dolphinscheduler.server.worker.task.PythonCommandExecutor; import org.apache.dolphinscheduler.server.worker.task.TaskProps; +import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.process.ProcessService; import org.slf4j.Logger; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTask.java index 32f5833c54..5704c8052e 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTask.java @@ -23,11 +23,11 @@ import org.apache.dolphinscheduler.common.task.AbstractParameters; import org.apache.dolphinscheduler.common.task.shell.ShellParameters; import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.ParameterUtils; -import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; import org.apache.dolphinscheduler.server.utils.ParamUtils; import org.apache.dolphinscheduler.server.worker.task.AbstractTask; import org.apache.dolphinscheduler.server.worker.task.ShellCommandExecutor; import org.apache.dolphinscheduler.server.worker.task.TaskProps; +import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.process.ProcessService; import org.slf4j.Logger; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java index bc306b9f06..23bda089f8 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java @@ -44,6 +44,7 @@ import org.apache.dolphinscheduler.server.utils.ParamUtils; import org.apache.dolphinscheduler.server.utils.UDFUtils; import org.apache.dolphinscheduler.server.worker.task.AbstractTask; import org.apache.dolphinscheduler.server.worker.task.TaskProps; +import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.permission.PermissionCheck; import org.apache.dolphinscheduler.service.process.ProcessService; import org.slf4j.Logger; diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterExecThreadTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterExecThreadTest.java index ec6647869a..770ab3cff6 100644 --- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterExecThreadTest.java +++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterExecThreadTest.java @@ -20,12 +20,12 @@ import com.alibaba.fastjson.JSONObject; import org.apache.dolphinscheduler.common.enums.*; import org.apache.dolphinscheduler.common.graph.DAG; import org.apache.dolphinscheduler.common.utils.DateUtils; -import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.Schedule; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.runner.MasterExecThread; +import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.process.ProcessService; import org.junit.Assert; import org.junit.Before; diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/shell/ShellCommandExecutorTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/shell/ShellCommandExecutorTest.java index 0ab0b4b92d..5d4263644b 100644 --- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/shell/ShellCommandExecutorTest.java +++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/shell/ShellCommandExecutorTest.java @@ -20,12 +20,12 @@ import com.alibaba.fastjson.JSONObject; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.model.TaskNode; -import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.common.utils.LoggerUtils; import org.apache.dolphinscheduler.server.worker.task.AbstractTask; import org.apache.dolphinscheduler.server.worker.task.TaskManager; import org.apache.dolphinscheduler.server.worker.task.TaskProps; +import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.process.ProcessService; import org.junit.Before; import org.junit.Ignore; diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/sql/SqlExecutorTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/sql/SqlExecutorTest.java index c8e92da7e0..c395eabe51 100644 --- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/sql/SqlExecutorTest.java +++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/sql/SqlExecutorTest.java @@ -21,12 +21,12 @@ import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.CommandType; import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.model.TaskNode; -import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.common.utils.LoggerUtils; import org.apache.dolphinscheduler.server.worker.task.AbstractTask; import org.apache.dolphinscheduler.server.worker.task.TaskManager; import org.apache.dolphinscheduler.server.worker.task.TaskProps; +import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.process.ProcessService; import org.junit.Before; import org.junit.Ignore; diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTaskTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTaskTest.java index 6b71d6dc19..4024dc36dd 100644 --- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTaskTest.java +++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTaskTest.java @@ -27,12 +27,12 @@ import org.apache.dolphinscheduler.common.enums.CommandType; import org.apache.dolphinscheduler.common.enums.DbType; import org.apache.dolphinscheduler.common.job.db.BaseDataSource; import org.apache.dolphinscheduler.common.job.db.DataSourceFactory; -import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; import org.apache.dolphinscheduler.dao.entity.DataSource; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.server.utils.DataxUtils; import org.apache.dolphinscheduler.server.worker.task.ShellCommandExecutor; import org.apache.dolphinscheduler.server.worker.task.TaskProps; +import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.process.ProcessService; import org.junit.After; import org.junit.Assert; diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/bean/SpringApplicationContext.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/bean/SpringApplicationContext.java new file mode 100644 index 0000000000..17118f89c6 --- /dev/null +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/bean/SpringApplicationContext.java @@ -0,0 +1,23 @@ +package org.apache.dolphinscheduler.service.bean; + +import org.springframework.beans.BeansException; +import org.springframework.context.ApplicationContext; +import org.springframework.context.ApplicationContextAware; +import org.springframework.stereotype.Component; + +@Component +public class SpringApplicationContext implements ApplicationContextAware { + + private static ApplicationContext applicationContext; + + @Override + public void setApplicationContext(ApplicationContext applicationContext) throws BeansException { + SpringApplicationContext.applicationContext = applicationContext; + } + + public static T getBean(Class requiredType){ + return applicationContext.getBean(requiredType); + } + + +} diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/TaskQueueFactory.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/TaskQueueFactory.java index ec0f1573bb..6be419f5a9 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/TaskQueueFactory.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/TaskQueueFactory.java @@ -18,7 +18,7 @@ package org.apache.dolphinscheduler.service.queue; import org.apache.commons.lang.StringUtils; import org.apache.dolphinscheduler.common.utils.CommonUtils; -import org.apache.dolphinscheduler.common.utils.SpringApplicationContext; +import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/PreconditionsTest.java b/dolphinscheduler-service/src/test/java/utils/PreconditionsTest.java similarity index 97% rename from dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/PreconditionsTest.java rename to dolphinscheduler-service/src/test/java/utils/PreconditionsTest.java index dcb0e1370e..a1b85f1b12 100644 --- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/PreconditionsTest.java +++ b/dolphinscheduler-service/src/test/java/utils/PreconditionsTest.java @@ -14,8 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.utils; +package utils; +import org.apache.dolphinscheduler.common.utils.Preconditions; import org.junit.Assert; import org.junit.Test; import org.slf4j.Logger; @@ -35,7 +36,7 @@ public class PreconditionsTest { public void testCheckNotNull() throws Exception { String testReference = "test reference"; //test reference is not null - Assert.assertEquals(testReference,Preconditions.checkNotNull(testReference)); + Assert.assertEquals(testReference, Preconditions.checkNotNull(testReference)); Assert.assertEquals(testReference,Preconditions.checkNotNull(testReference,"reference is null")); Assert.assertEquals(testReference,Preconditions.checkNotNull(testReference,"%s is null",testReference)); From e6de6dbd51b93f25ff75384af6835c4271bbcc90 Mon Sep 17 00:00:00 2001 From: qiaozhanwei Date: Tue, 11 Feb 2020 16:21:28 +0800 Subject: [PATCH 004/221] 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 --- .../service/bean/SpringApplicationContext.java | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/bean/SpringApplicationContext.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/bean/SpringApplicationContext.java index 17118f89c6..ddf1fecf76 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/bean/SpringApplicationContext.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/bean/SpringApplicationContext.java @@ -1,3 +1,19 @@ +/* + * 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.service.bean; import org.springframework.beans.BeansException; From b1a68c645df5f47a4d6e25d50090b01c981debbb Mon Sep 17 00:00:00 2001 From: Technoboy- Date: Tue, 11 Feb 2020 21:17:29 +0800 Subject: [PATCH 006/221] move datasource classes to dao module --- .../dolphinscheduler/api/service/DataSourceService.java | 3 +-- .../dolphinscheduler/dao/datasource}/BaseDataSource.java | 2 +- .../dao/datasource}/ClickHouseDataSource.java | 2 +- .../dao/datasource}/DB2ServerDataSource.java | 2 +- .../dolphinscheduler/dao/datasource}/DataSourceFactory.java | 4 ++-- .../dolphinscheduler/dao/datasource}/HiveDataSource.java | 6 ++++-- .../dolphinscheduler/dao/datasource}/MySQLDataSource.java | 2 +- .../dolphinscheduler/dao/datasource}/OracleDataSource.java | 2 +- .../dolphinscheduler/dao/datasource}/PostgreDataSource.java | 2 +- .../dao/datasource}/SQLServerDataSource.java | 2 +- .../dolphinscheduler/dao/datasource}/SparkDataSource.java | 2 +- .../server/worker/task/datax/DataxTask.java | 4 ++-- .../server/worker/task/processdure/ProcedureTask.java | 4 ++-- .../dolphinscheduler/server/worker/task/sql/SqlTask.java | 4 ++-- .../server/worker/task/datax/DataxTaskTest.java | 4 ++-- 15 files changed, 23 insertions(+), 22 deletions(-) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/BaseDataSource.java (97%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/ClickHouseDataSource.java (97%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/DB2ServerDataSource.java (97%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/DataSourceFactory.java (98%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/HiveDataSource.java (93%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/MySQLDataSource.java (97%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/OracleDataSource.java (97%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/PostgreDataSource.java (97%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/SQLServerDataSource.java (97%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/SparkDataSource.java (97%) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataSourceService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataSourceService.java index 5d33b46bd2..f6d8903dd8 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataSourceService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataSourceService.java @@ -21,10 +21,9 @@ import org.apache.dolphinscheduler.api.utils.PageInfo; import org.apache.dolphinscheduler.api.utils.Result; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.DbType; -import org.apache.dolphinscheduler.common.enums.UserType; -import org.apache.dolphinscheduler.common.job.db.*; import org.apache.dolphinscheduler.common.utils.CommonUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils; +import org.apache.dolphinscheduler.dao.datasource.*; import org.apache.dolphinscheduler.dao.entity.DataSource; import org.apache.dolphinscheduler.dao.entity.Resource; import org.apache.dolphinscheduler.dao.entity.User; diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/BaseDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/BaseDataSource.java similarity index 97% rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/BaseDataSource.java rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/BaseDataSource.java index 41a9b3a566..a46e5aabcc 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/BaseDataSource.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/BaseDataSource.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.job.db; +package org.apache.dolphinscheduler.dao.datasource; /** * data source base class diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/ClickHouseDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/ClickHouseDataSource.java similarity index 97% rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/ClickHouseDataSource.java rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/ClickHouseDataSource.java index fe76497ff8..cfa739ba25 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/ClickHouseDataSource.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/ClickHouseDataSource.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.job.db; +package org.apache.dolphinscheduler.dao.datasource; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.utils.StringUtils; diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/DB2ServerDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/DB2ServerDataSource.java similarity index 97% rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/DB2ServerDataSource.java rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/DB2ServerDataSource.java index 44ee200c5d..3c2366b5b0 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/DB2ServerDataSource.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/DB2ServerDataSource.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.job.db; +package org.apache.dolphinscheduler.dao.datasource; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.utils.StringUtils; diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/DataSourceFactory.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/DataSourceFactory.java similarity index 98% rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/DataSourceFactory.java rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/DataSourceFactory.java index 48ec319eaa..9571f9c9f6 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/DataSourceFactory.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/DataSourceFactory.java @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.job.db; +package org.apache.dolphinscheduler.dao.datasource; +import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.DbType; import org.apache.dolphinscheduler.common.utils.JSONUtils; -import org.apache.dolphinscheduler.common.Constants; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/HiveDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/HiveDataSource.java similarity index 93% rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/HiveDataSource.java rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/HiveDataSource.java index cddedd1f73..0a8f527536 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/HiveDataSource.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/HiveDataSource.java @@ -14,14 +14,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.job.db; +package org.apache.dolphinscheduler.dao.datasource; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.utils.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.sql.*; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; /** * data source of hive diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/MySQLDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/MySQLDataSource.java similarity index 97% rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/MySQLDataSource.java rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/MySQLDataSource.java index fa149e67e2..6e2fbe3dd8 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/MySQLDataSource.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/MySQLDataSource.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.job.db; +package org.apache.dolphinscheduler.dao.datasource; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.utils.StringUtils; diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/OracleDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/OracleDataSource.java similarity index 97% rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/OracleDataSource.java rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/OracleDataSource.java index c3dc3a96df..cefaf879b5 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/OracleDataSource.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/OracleDataSource.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.job.db; +package org.apache.dolphinscheduler.dao.datasource; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.utils.StringUtils; diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/PostgreDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/PostgreDataSource.java similarity index 97% rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/PostgreDataSource.java rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/PostgreDataSource.java index 4989e7681e..176cba2587 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/PostgreDataSource.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/PostgreDataSource.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.job.db; +package org.apache.dolphinscheduler.dao.datasource; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.utils.StringUtils; diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/SQLServerDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/SQLServerDataSource.java similarity index 97% rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/SQLServerDataSource.java rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/SQLServerDataSource.java index 8554992efc..07770c06a7 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/SQLServerDataSource.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/SQLServerDataSource.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.job.db; +package org.apache.dolphinscheduler.dao.datasource; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.utils.StringUtils; diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/SparkDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/SparkDataSource.java similarity index 97% rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/SparkDataSource.java rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/SparkDataSource.java index 5d10c63e5d..81a5ac6f04 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/SparkDataSource.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/SparkDataSource.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.job.db; +package org.apache.dolphinscheduler.dao.datasource; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.utils.StringUtils; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java index e731ae9879..ef941cd062 100755 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java @@ -39,14 +39,14 @@ import java.util.Set; import org.apache.commons.io.FileUtils; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.DbType; -import org.apache.dolphinscheduler.common.job.db.BaseDataSource; -import org.apache.dolphinscheduler.common.job.db.DataSourceFactory; import org.apache.dolphinscheduler.common.process.Property; import org.apache.dolphinscheduler.common.task.AbstractParameters; import org.apache.dolphinscheduler.common.task.datax.DataxParameters; import org.apache.dolphinscheduler.common.utils.CollectionUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.ParameterUtils; +import org.apache.dolphinscheduler.dao.datasource.BaseDataSource; +import org.apache.dolphinscheduler.dao.datasource.DataSourceFactory; import org.apache.dolphinscheduler.dao.entity.DataSource; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.server.utils.DataxUtils; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java index 3898357f85..fb881453e9 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java @@ -22,13 +22,13 @@ import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.DataType; import org.apache.dolphinscheduler.common.enums.Direct; import org.apache.dolphinscheduler.common.enums.TaskTimeoutStrategy; -import org.apache.dolphinscheduler.common.job.db.BaseDataSource; -import org.apache.dolphinscheduler.common.job.db.DataSourceFactory; import org.apache.dolphinscheduler.common.process.Property; import org.apache.dolphinscheduler.common.task.AbstractParameters; import org.apache.dolphinscheduler.common.task.procedure.ProcedureParameters; import org.apache.dolphinscheduler.common.utils.CollectionUtils; import org.apache.dolphinscheduler.common.utils.ParameterUtils; +import org.apache.dolphinscheduler.dao.datasource.BaseDataSource; +import org.apache.dolphinscheduler.dao.datasource.DataSourceFactory; import org.apache.dolphinscheduler.dao.entity.DataSource; import org.apache.dolphinscheduler.server.utils.ParamUtils; import org.apache.dolphinscheduler.server.worker.task.AbstractTask; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java index 23bda089f8..aae11f5530 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java @@ -27,8 +27,6 @@ import org.apache.dolphinscheduler.common.enums.AuthorizationType; import org.apache.dolphinscheduler.common.enums.ShowType; import org.apache.dolphinscheduler.common.enums.TaskTimeoutStrategy; import org.apache.dolphinscheduler.common.enums.UdfType; -import org.apache.dolphinscheduler.common.job.db.BaseDataSource; -import org.apache.dolphinscheduler.common.job.db.DataSourceFactory; import org.apache.dolphinscheduler.common.process.Property; import org.apache.dolphinscheduler.common.task.AbstractParameters; import org.apache.dolphinscheduler.common.task.sql.SqlBinds; @@ -36,6 +34,8 @@ import org.apache.dolphinscheduler.common.task.sql.SqlParameters; import org.apache.dolphinscheduler.common.task.sql.SqlType; import org.apache.dolphinscheduler.common.utils.*; import org.apache.dolphinscheduler.dao.AlertDao; +import org.apache.dolphinscheduler.dao.datasource.BaseDataSource; +import org.apache.dolphinscheduler.dao.datasource.DataSourceFactory; import org.apache.dolphinscheduler.dao.entity.DataSource; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.UdfFunc; diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTaskTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTaskTest.java index 4024dc36dd..bd7f27530a 100644 --- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTaskTest.java +++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTaskTest.java @@ -25,8 +25,8 @@ import java.util.List; import com.alibaba.fastjson.JSONObject; import org.apache.dolphinscheduler.common.enums.CommandType; import org.apache.dolphinscheduler.common.enums.DbType; -import org.apache.dolphinscheduler.common.job.db.BaseDataSource; -import org.apache.dolphinscheduler.common.job.db.DataSourceFactory; +import org.apache.dolphinscheduler.dao.datasource.BaseDataSource; +import org.apache.dolphinscheduler.dao.datasource.DataSourceFactory; import org.apache.dolphinscheduler.dao.entity.DataSource; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.server.utils.DataxUtils; From 52c87a6fda8ff58dd4ce6998a2385aaa2e3f2706 Mon Sep 17 00:00:00 2001 From: Technoboy- Date: Tue, 11 Feb 2020 21:47:18 +0800 Subject: [PATCH 007/221] fix send4LetterWord bug --- .../api/utils/FourLetterWordMain.java | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/FourLetterWordMain.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/FourLetterWordMain.java index b04e773aea..d4c11c769a 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/FourLetterWordMain.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/FourLetterWordMain.java @@ -60,21 +60,22 @@ public class FourLetterWordMain { public static String send4LetterWord(String host, int port, String cmd, int timeout) throws IOException { LOG.info("connecting to {} {}", host, port); + Socket sock = new Socket(); InetSocketAddress hostaddress= host != null ? new InetSocketAddress(host, port) : new InetSocketAddress(InetAddress.getByName(null), port); - - try (Socket sock = new Socket(); - OutputStream outstream = sock.getOutputStream(); - BufferedReader reader = - new BufferedReader( - new InputStreamReader(sock.getInputStream()))) { + BufferedReader reader = null; + try { sock.setSoTimeout(timeout); sock.connect(hostaddress, timeout); + OutputStream outstream = sock.getOutputStream(); outstream.write(cmd.getBytes()); outstream.flush(); // this replicates NC - close the output stream before reading sock.shutdownOutput(); + reader = + new BufferedReader( + new InputStreamReader(sock.getInputStream())); StringBuilder sb = new StringBuilder(); String line; while((line = reader.readLine()) != null) { @@ -83,6 +84,11 @@ public class FourLetterWordMain { return sb.toString(); } catch (SocketTimeoutException e) { throw new IOException("Exception while executing four letter word: " + cmd, e); + } finally { + sock.close(); + if (reader != null) { + reader.close(); + } } } } From d3f8bb0a2b80f2a0348e2396c537f41be480cf84 Mon Sep 17 00:00:00 2001 From: Tboy Date: Tue, 11 Feb 2020 21:55:14 +0800 Subject: [PATCH 008/221] Refactor architecture (#1936) * move datasource classes to dao module * fix send4LetterWord bug --- .../api/service/DataSourceService.java | 3 +-- .../api/utils/FourLetterWordMain.java | 18 ++++++++++++------ .../dao/datasource}/BaseDataSource.java | 2 +- .../dao/datasource}/ClickHouseDataSource.java | 2 +- .../dao/datasource}/DB2ServerDataSource.java | 2 +- .../dao/datasource}/DataSourceFactory.java | 4 ++-- .../dao/datasource}/HiveDataSource.java | 6 ++++-- .../dao/datasource}/MySQLDataSource.java | 2 +- .../dao/datasource}/OracleDataSource.java | 2 +- .../dao/datasource}/PostgreDataSource.java | 2 +- .../dao/datasource}/SQLServerDataSource.java | 2 +- .../dao/datasource}/SparkDataSource.java | 2 +- .../server/worker/task/datax/DataxTask.java | 4 ++-- .../worker/task/processdure/ProcedureTask.java | 4 ++-- .../server/worker/task/sql/SqlTask.java | 4 ++-- .../worker/task/datax/DataxTaskTest.java | 4 ++-- 16 files changed, 35 insertions(+), 28 deletions(-) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/BaseDataSource.java (97%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/ClickHouseDataSource.java (97%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/DB2ServerDataSource.java (97%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/DataSourceFactory.java (98%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/HiveDataSource.java (93%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/MySQLDataSource.java (97%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/OracleDataSource.java (97%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/PostgreDataSource.java (97%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/SQLServerDataSource.java (97%) rename {dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db => dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource}/SparkDataSource.java (97%) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataSourceService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataSourceService.java index 5d33b46bd2..f6d8903dd8 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataSourceService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataSourceService.java @@ -21,10 +21,9 @@ import org.apache.dolphinscheduler.api.utils.PageInfo; import org.apache.dolphinscheduler.api.utils.Result; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.DbType; -import org.apache.dolphinscheduler.common.enums.UserType; -import org.apache.dolphinscheduler.common.job.db.*; import org.apache.dolphinscheduler.common.utils.CommonUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils; +import org.apache.dolphinscheduler.dao.datasource.*; import org.apache.dolphinscheduler.dao.entity.DataSource; import org.apache.dolphinscheduler.dao.entity.Resource; import org.apache.dolphinscheduler.dao.entity.User; diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/FourLetterWordMain.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/FourLetterWordMain.java index b04e773aea..d4c11c769a 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/FourLetterWordMain.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/utils/FourLetterWordMain.java @@ -60,21 +60,22 @@ public class FourLetterWordMain { public static String send4LetterWord(String host, int port, String cmd, int timeout) throws IOException { LOG.info("connecting to {} {}", host, port); + Socket sock = new Socket(); InetSocketAddress hostaddress= host != null ? new InetSocketAddress(host, port) : new InetSocketAddress(InetAddress.getByName(null), port); - - try (Socket sock = new Socket(); - OutputStream outstream = sock.getOutputStream(); - BufferedReader reader = - new BufferedReader( - new InputStreamReader(sock.getInputStream()))) { + BufferedReader reader = null; + try { sock.setSoTimeout(timeout); sock.connect(hostaddress, timeout); + OutputStream outstream = sock.getOutputStream(); outstream.write(cmd.getBytes()); outstream.flush(); // this replicates NC - close the output stream before reading sock.shutdownOutput(); + reader = + new BufferedReader( + new InputStreamReader(sock.getInputStream())); StringBuilder sb = new StringBuilder(); String line; while((line = reader.readLine()) != null) { @@ -83,6 +84,11 @@ public class FourLetterWordMain { return sb.toString(); } catch (SocketTimeoutException e) { throw new IOException("Exception while executing four letter word: " + cmd, e); + } finally { + sock.close(); + if (reader != null) { + reader.close(); + } } } } diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/BaseDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/BaseDataSource.java similarity index 97% rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/BaseDataSource.java rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/BaseDataSource.java index 41a9b3a566..a46e5aabcc 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/BaseDataSource.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/BaseDataSource.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.job.db; +package org.apache.dolphinscheduler.dao.datasource; /** * data source base class diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/ClickHouseDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/ClickHouseDataSource.java similarity index 97% rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/ClickHouseDataSource.java rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/ClickHouseDataSource.java index fe76497ff8..cfa739ba25 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/ClickHouseDataSource.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/ClickHouseDataSource.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.job.db; +package org.apache.dolphinscheduler.dao.datasource; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.utils.StringUtils; diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/DB2ServerDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/DB2ServerDataSource.java similarity index 97% rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/DB2ServerDataSource.java rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/DB2ServerDataSource.java index 44ee200c5d..3c2366b5b0 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/DB2ServerDataSource.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/DB2ServerDataSource.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.job.db; +package org.apache.dolphinscheduler.dao.datasource; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.utils.StringUtils; diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/DataSourceFactory.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/DataSourceFactory.java similarity index 98% rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/DataSourceFactory.java rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/DataSourceFactory.java index 48ec319eaa..9571f9c9f6 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/DataSourceFactory.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/DataSourceFactory.java @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.job.db; +package org.apache.dolphinscheduler.dao.datasource; +import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.DbType; import org.apache.dolphinscheduler.common.utils.JSONUtils; -import org.apache.dolphinscheduler.common.Constants; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/HiveDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/HiveDataSource.java similarity index 93% rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/HiveDataSource.java rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/HiveDataSource.java index cddedd1f73..0a8f527536 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/HiveDataSource.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/HiveDataSource.java @@ -14,14 +14,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.job.db; +package org.apache.dolphinscheduler.dao.datasource; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.utils.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.sql.*; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; /** * data source of hive diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/MySQLDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/MySQLDataSource.java similarity index 97% rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/MySQLDataSource.java rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/MySQLDataSource.java index fa149e67e2..6e2fbe3dd8 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/MySQLDataSource.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/MySQLDataSource.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.job.db; +package org.apache.dolphinscheduler.dao.datasource; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.utils.StringUtils; diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/OracleDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/OracleDataSource.java similarity index 97% rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/OracleDataSource.java rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/OracleDataSource.java index c3dc3a96df..cefaf879b5 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/OracleDataSource.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/OracleDataSource.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.job.db; +package org.apache.dolphinscheduler.dao.datasource; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.utils.StringUtils; diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/PostgreDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/PostgreDataSource.java similarity index 97% rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/PostgreDataSource.java rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/PostgreDataSource.java index 4989e7681e..176cba2587 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/PostgreDataSource.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/PostgreDataSource.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.job.db; +package org.apache.dolphinscheduler.dao.datasource; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.utils.StringUtils; diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/SQLServerDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/SQLServerDataSource.java similarity index 97% rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/SQLServerDataSource.java rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/SQLServerDataSource.java index 8554992efc..07770c06a7 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/SQLServerDataSource.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/SQLServerDataSource.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.job.db; +package org.apache.dolphinscheduler.dao.datasource; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.utils.StringUtils; diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/SparkDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/SparkDataSource.java similarity index 97% rename from dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/SparkDataSource.java rename to dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/SparkDataSource.java index 5d10c63e5d..81a5ac6f04 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/job/db/SparkDataSource.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/SparkDataSource.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.dolphinscheduler.common.job.db; +package org.apache.dolphinscheduler.dao.datasource; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.utils.StringUtils; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java index e731ae9879..ef941cd062 100755 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java @@ -39,14 +39,14 @@ import java.util.Set; import org.apache.commons.io.FileUtils; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.DbType; -import org.apache.dolphinscheduler.common.job.db.BaseDataSource; -import org.apache.dolphinscheduler.common.job.db.DataSourceFactory; import org.apache.dolphinscheduler.common.process.Property; import org.apache.dolphinscheduler.common.task.AbstractParameters; import org.apache.dolphinscheduler.common.task.datax.DataxParameters; import org.apache.dolphinscheduler.common.utils.CollectionUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.ParameterUtils; +import org.apache.dolphinscheduler.dao.datasource.BaseDataSource; +import org.apache.dolphinscheduler.dao.datasource.DataSourceFactory; import org.apache.dolphinscheduler.dao.entity.DataSource; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.server.utils.DataxUtils; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java index 3898357f85..fb881453e9 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java @@ -22,13 +22,13 @@ import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.DataType; import org.apache.dolphinscheduler.common.enums.Direct; import org.apache.dolphinscheduler.common.enums.TaskTimeoutStrategy; -import org.apache.dolphinscheduler.common.job.db.BaseDataSource; -import org.apache.dolphinscheduler.common.job.db.DataSourceFactory; import org.apache.dolphinscheduler.common.process.Property; import org.apache.dolphinscheduler.common.task.AbstractParameters; import org.apache.dolphinscheduler.common.task.procedure.ProcedureParameters; import org.apache.dolphinscheduler.common.utils.CollectionUtils; import org.apache.dolphinscheduler.common.utils.ParameterUtils; +import org.apache.dolphinscheduler.dao.datasource.BaseDataSource; +import org.apache.dolphinscheduler.dao.datasource.DataSourceFactory; import org.apache.dolphinscheduler.dao.entity.DataSource; import org.apache.dolphinscheduler.server.utils.ParamUtils; import org.apache.dolphinscheduler.server.worker.task.AbstractTask; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java index 23bda089f8..aae11f5530 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java @@ -27,8 +27,6 @@ import org.apache.dolphinscheduler.common.enums.AuthorizationType; import org.apache.dolphinscheduler.common.enums.ShowType; import org.apache.dolphinscheduler.common.enums.TaskTimeoutStrategy; import org.apache.dolphinscheduler.common.enums.UdfType; -import org.apache.dolphinscheduler.common.job.db.BaseDataSource; -import org.apache.dolphinscheduler.common.job.db.DataSourceFactory; import org.apache.dolphinscheduler.common.process.Property; import org.apache.dolphinscheduler.common.task.AbstractParameters; import org.apache.dolphinscheduler.common.task.sql.SqlBinds; @@ -36,6 +34,8 @@ import org.apache.dolphinscheduler.common.task.sql.SqlParameters; import org.apache.dolphinscheduler.common.task.sql.SqlType; import org.apache.dolphinscheduler.common.utils.*; import org.apache.dolphinscheduler.dao.AlertDao; +import org.apache.dolphinscheduler.dao.datasource.BaseDataSource; +import org.apache.dolphinscheduler.dao.datasource.DataSourceFactory; import org.apache.dolphinscheduler.dao.entity.DataSource; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.UdfFunc; diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTaskTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTaskTest.java index 4024dc36dd..bd7f27530a 100644 --- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTaskTest.java +++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTaskTest.java @@ -25,8 +25,8 @@ import java.util.List; import com.alibaba.fastjson.JSONObject; import org.apache.dolphinscheduler.common.enums.CommandType; import org.apache.dolphinscheduler.common.enums.DbType; -import org.apache.dolphinscheduler.common.job.db.BaseDataSource; -import org.apache.dolphinscheduler.common.job.db.DataSourceFactory; +import org.apache.dolphinscheduler.dao.datasource.BaseDataSource; +import org.apache.dolphinscheduler.dao.datasource.DataSourceFactory; import org.apache.dolphinscheduler.dao.entity.DataSource; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.server.utils.DataxUtils; From d14a10887f35916ae672a903c09b2445c00ba74f Mon Sep 17 00:00:00 2001 From: Technoboy- Date: Tue, 11 Feb 2020 22:31:24 +0800 Subject: [PATCH 009/221] exclude jasper-compiler in case of runtime conflict --- dolphinscheduler-common/pom.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dolphinscheduler-common/pom.xml b/dolphinscheduler-common/pom.xml index e1cdfb48fd..2ef61d7f36 100644 --- a/dolphinscheduler-common/pom.xml +++ b/dolphinscheduler-common/pom.xml @@ -534,6 +534,10 @@ log4j-web org.apache.logging.log4j + + jasper-compiler + tomcat + From f00537959ead0e741b173f1696d28bdb7cd1154b Mon Sep 17 00:00:00 2001 From: qiaozhanwei Date: Wed, 12 Feb 2020 17:44:35 +0800 Subject: [PATCH 010/221] 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 --- .../api/service/LoggerServiceTest.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/LoggerServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/LoggerServiceTest.java index c6ab6f8e74..20571577e3 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/LoggerServiceTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/LoggerServiceTest.java @@ -18,8 +18,8 @@ package org.apache.dolphinscheduler.api.service; import org.apache.dolphinscheduler.api.enums.Status; import org.apache.dolphinscheduler.api.utils.Result; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.TaskInstance; +import org.apache.dolphinscheduler.service.process.ProcessService; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -40,14 +40,14 @@ public class LoggerServiceTest { @InjectMocks private LoggerService loggerService; @Mock - private ProcessDao processDao; + private ProcessService processService; @Test public void testQueryDataSourceList(){ TaskInstance taskInstance = new TaskInstance(); - Mockito.when(processDao.findTaskInstanceById(1)).thenReturn(taskInstance); + Mockito.when(processService.findTaskInstanceById(1)).thenReturn(taskInstance); Result result = loggerService.queryLog(2,1,1); //TASK_INSTANCE_NOT_FOUND Assert.assertEquals(Status.TASK_INSTANCE_NOT_FOUND.getCode(),result.getCode().intValue()); @@ -59,7 +59,7 @@ public class LoggerServiceTest { //SUCCESS taskInstance.setHost("127.0.0.1"); taskInstance.setLogPath("/temp/log"); - Mockito.when(processDao.findTaskInstanceById(1)).thenReturn(taskInstance); + Mockito.when(processService.findTaskInstanceById(1)).thenReturn(taskInstance); result = loggerService.queryLog(1,1,1); Assert.assertEquals(Status.SUCCESS.getCode(),result.getCode().intValue()); } @@ -68,7 +68,7 @@ public class LoggerServiceTest { public void testGetLogBytes(){ TaskInstance taskInstance = new TaskInstance(); - Mockito.when(processDao.findTaskInstanceById(1)).thenReturn(taskInstance); + Mockito.when(processService.findTaskInstanceById(1)).thenReturn(taskInstance); //task instance is null try{ From a5c2b87c6483e5ec30fb365aac12975037274449 Mon Sep 17 00:00:00 2001 From: Tboy Date: Wed, 12 Feb 2020 17:47:11 +0800 Subject: [PATCH 011/221] 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 --- dolphinscheduler-common/pom.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dolphinscheduler-common/pom.xml b/dolphinscheduler-common/pom.xml index e1cdfb48fd..2ef61d7f36 100644 --- a/dolphinscheduler-common/pom.xml +++ b/dolphinscheduler-common/pom.xml @@ -534,6 +534,10 @@ log4j-web org.apache.logging.log4j + + jasper-compiler + tomcat + From fb5de1784e22072d0597e51d6c7de5a6c126bfbf Mon Sep 17 00:00:00 2001 From: qiaozhanwei Date: Wed, 12 Feb 2020 19:53:47 +0800 Subject: [PATCH 012/221] DataAnaylysisServiceTest and ProcessDefinitionService modify --- .../api/service/DataAnalysisServiceTest.java | 8 ++++---- .../api/service/ProcessDefinitionServiceTest.java | 1 - 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/DataAnalysisServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/DataAnalysisServiceTest.java index c7afd76cc6..6f308e7b17 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/DataAnalysisServiceTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/DataAnalysisServiceTest.java @@ -21,15 +21,15 @@ import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.CommandType; import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.enums.UserType; -import org.apache.dolphinscheduler.common.queue.ITaskQueue; -import org.apache.dolphinscheduler.common.queue.TaskQueueFactory; import org.apache.dolphinscheduler.common.utils.DateUtils; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.CommandCount; import org.apache.dolphinscheduler.dao.entity.ExecuteStatusCount; import org.apache.dolphinscheduler.dao.entity.Project; import org.apache.dolphinscheduler.dao.entity.User; import org.apache.dolphinscheduler.dao.mapper.*; +import org.apache.dolphinscheduler.service.process.ProcessService; +import org.apache.dolphinscheduler.service.queue.ITaskQueue; +import org.apache.dolphinscheduler.service.queue.TaskQueueFactory; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -78,7 +78,7 @@ public class DataAnalysisServiceTest { ITaskQueue taskQueue; @Mock - ProcessDao processDao; + ProcessService processService; private Project project; diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionServiceTest.java index 4f626beff0..aa9f5fe20f 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionServiceTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionServiceTest.java @@ -28,7 +28,6 @@ import org.apache.dolphinscheduler.common.enums.*; import org.apache.dolphinscheduler.common.utils.DateUtils; import org.apache.dolphinscheduler.common.utils.FileUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.*; import org.apache.dolphinscheduler.dao.mapper.*; import org.apache.dolphinscheduler.service.process.ProcessService; From 1ac1d582e6d0d2ba1adb45bd150e8a7854ec32f3 Mon Sep 17 00:00:00 2001 From: qiaozhanwei Date: Thu, 13 Feb 2020 15:55:10 +0800 Subject: [PATCH 013/221] remote module add comment --- .../remote/NettyRemotingClient.java | 77 ++++++++++++++++++- .../remote/NettyRemotingServer.java | 55 +++++++++++++ .../remote/codec/NettyDecoder.java | 17 ++++ .../remote/codec/NettyEncoder.java | 8 ++ .../remote/command/Command.java | 5 +- .../remote/command/CommandHeader.java | 9 +++ .../remote/command/CommandType.java | 2 +- .../command/ExecuteTaskRequestCommand.java | 2 +- .../command/ExecuteTaskResponseCommand.java | 2 +- .../dolphinscheduler/remote/command/Ping.java | 19 ++++- .../dolphinscheduler/remote/command/Pong.java | 23 +++++- .../log/GetLogBytesRequestCommand.java | 9 ++- .../log/GetLogBytesResponseCommand.java | 9 +++ .../log/RollViewLogRequestCommand.java | 17 ++++ .../log/RollViewLogResponseCommand.java | 9 +++ .../command/log/ViewLogRequestCommand.java | 8 ++ .../command/log/ViewLogResponseCommand.java | 9 +++ .../remote/config/NettyClientConfig.java | 15 ++++ .../remote/config/NettyServerConfig.java | 21 +++++ .../remote/handler/NettyClientHandler.java | 51 ++++++++++++ .../remote/handler/NettyServerHandler.java | 52 +++++++++++++ .../processor/NettyRequestProcessor.java | 5 ++ .../remote/utils/Address.java | 6 ++ .../remote/utils/ChannelUtils.java | 16 ++++ .../remote/utils/Constants.java | 10 +++ .../remote/utils/FastJsonSerializer.java | 21 +++++ .../dolphinscheduler/remote/utils/Pair.java | 6 ++ .../remote/NettyRemotingClientTest.java | 6 ++ 28 files changed, 481 insertions(+), 8 deletions(-) diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java index 678fe84f90..df0c13ad38 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java @@ -50,22 +50,51 @@ public class NettyRemotingClient { private final Logger logger = LoggerFactory.getLogger(NettyRemotingClient.class); + /** + * bootstrap + */ private final Bootstrap bootstrap = new Bootstrap(); + /** + * encoder + */ private final NettyEncoder encoder = new NettyEncoder(); + /** + * channels + */ private final ConcurrentHashMap channels = new ConcurrentHashMap(); + /** + * default executor + */ private final ExecutorService defaultExecutor = Executors.newFixedThreadPool(Constants.CPUS); + /** + * started flag + */ private final AtomicBoolean isStarted = new AtomicBoolean(false); + /** + * worker group + */ private final NioEventLoopGroup workerGroup; + /** + * client handler + */ private final NettyClientHandler clientHandler = new NettyClientHandler(this); + /** + * netty client config + */ private final NettyClientConfig clientConfig; + /** + * netty client init + * + * @param clientConfig client config + */ public NettyRemotingClient(final NettyClientConfig clientConfig){ this.clientConfig = clientConfig; this.workerGroup = new NioEventLoopGroup(clientConfig.getWorkerThreads(), new ThreadFactory() { @@ -79,6 +108,9 @@ public class NettyRemotingClient { this.start(); } + /** + * netty server start + */ private void start(){ this.bootstrap @@ -97,18 +129,36 @@ public class NettyRemotingClient { encoder); } }); - // isStarted.compareAndSet(false, true); } + /** + * register processor + * + * @param commandType command type + * @param processor processor + */ public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor) { registerProcessor(commandType, processor, null); } + /** + * register processor + * + * @param commandType command type + * @param processor processor + * @param executor thread executor + */ public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor, final ExecutorService executor) { this.clientHandler.registerProcessor(commandType, processor, executor); } + /** + * send connect + * @param address address + * @param command command + * @throws RemotingException + */ public void send(final Address address, final Command command) throws RemotingException { final Channel channel = getChannel(address); if (channel == null) { @@ -132,6 +182,11 @@ public class NettyRemotingClient { } } + /** + * get channel + * @param address address + * @return channel + */ public Channel getChannel(Address address) { Channel channel = channels.get(address); if(channel != null && channel.isActive()){ @@ -140,6 +195,12 @@ public class NettyRemotingClient { return createChannel(address, true); } + /** + * create channel + * @param address address + * @param isSync is sync + * @return channel + */ public Channel createChannel(Address address, boolean isSync) { ChannelFuture future; try { @@ -160,10 +221,17 @@ public class NettyRemotingClient { return null; } + /** + * get default thread executor + * @return thread executor + */ public ExecutorService getDefaultExecutor() { return defaultExecutor; } + /** + * close client + */ public void close() { if(isStarted.compareAndSet(true, false)){ try { @@ -181,6 +249,9 @@ public class NettyRemotingClient { } } + /** + * close channel + */ private void closeChannels(){ for (Channel channel : this.channels.values()) { channel.close(); @@ -188,6 +259,10 @@ public class NettyRemotingClient { this.channels.clear(); } + /** + * remove channel + * @param address address + */ public void removeChannel(Address address){ Channel channel = this.channels.remove(address); if(channel != null){ diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingServer.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingServer.java index 5823dbb088..c69bf09540 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingServer.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingServer.java @@ -48,28 +48,58 @@ public class NettyRemotingServer { private final Logger logger = LoggerFactory.getLogger(NettyRemotingServer.class); + /** + * server bootstart + */ private final ServerBootstrap serverBootstrap = new ServerBootstrap(); + /** + * encoder + */ private final NettyEncoder encoder = new NettyEncoder(); + /** + * default executor + */ private final ExecutorService defaultExecutor = Executors.newFixedThreadPool(Constants.CPUS); + /** + * boss group + */ private final NioEventLoopGroup bossGroup; + /** + * worker group + */ private final NioEventLoopGroup workGroup; + /** + * server config + */ private final NettyServerConfig serverConfig; + /** + * server handler + */ private final NettyServerHandler serverHandler = new NettyServerHandler(this); + /** + * started flag + */ private final AtomicBoolean isStarted = new AtomicBoolean(false); + /** + * server init + * + * @param serverConfig server config + */ public NettyRemotingServer(final NettyServerConfig serverConfig){ this.serverConfig = serverConfig; this.bossGroup = new NioEventLoopGroup(1, new ThreadFactory() { private AtomicInteger threadIndex = new AtomicInteger(0); + @Override public Thread newThread(Runnable r) { return new Thread(r, String.format("NettyServerBossThread_%d", this.threadIndex.incrementAndGet())); } @@ -78,12 +108,16 @@ public class NettyRemotingServer { this.workGroup = new NioEventLoopGroup(serverConfig.getWorkerThread(), new ThreadFactory() { private AtomicInteger threadIndex = new AtomicInteger(0); + @Override public Thread newThread(Runnable r) { return new Thread(r, String.format("NettyServerWorkerThread_%d", this.threadIndex.incrementAndGet())); } }); } + /** + * server start + */ public void start(){ if(this.isStarted.get()){ @@ -125,6 +159,11 @@ public class NettyRemotingServer { isStarted.compareAndSet(false, true); } + /** + * init netty channel + * @param ch socket channel + * @throws Exception + */ private void initNettyChannel(NioSocketChannel ch) throws Exception{ ChannelPipeline pipeline = ch.pipeline(); pipeline.addLast("encoder", encoder); @@ -132,14 +171,30 @@ public class NettyRemotingServer { pipeline.addLast("handler", serverHandler); } + /** + * register processor + * @param commandType command type + * @param processor processor + */ public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor) { this.registerProcessor(commandType, processor, null); } + /** + * register processor + * + * @param commandType command type + * @param processor processor + * @param executor thread executor + */ public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor, final ExecutorService executor) { this.serverHandler.registerProcessor(commandType, processor, executor); } + /** + * get default thread executor + * @return thread executor + */ public ExecutorService getDefaultExecutor() { return defaultExecutor; } diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/codec/NettyDecoder.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/codec/NettyDecoder.java index 998f4ee7d5..caa4fbdd17 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/codec/NettyDecoder.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/codec/NettyDecoder.java @@ -38,6 +38,14 @@ public class NettyDecoder extends ReplayingDecoder { private final CommandHeader commandHeader = new CommandHeader(); + /** + * decode + * + * @param ctx channel handler context + * @param in byte buffer + * @param out out content + * @throws Exception + */ @Override protected void decode(ChannelHandlerContext ctx, ByteBuf in, List out) throws Exception { switch (state()){ @@ -67,6 +75,11 @@ public class NettyDecoder extends ReplayingDecoder { } } + /** + * get command type + * @param type type + * @return + */ private CommandType commandType(byte type){ for(CommandType ct : CommandType.values()){ if(ct.ordinal() == type){ @@ -76,6 +89,10 @@ public class NettyDecoder extends ReplayingDecoder { return null; } + /** + * check magic + * @param magic magic + */ private void checkMagic(byte magic) { if (magic != Command.MAGIC) { throw new IllegalArgumentException("illegal packet [magic]" + magic); diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/codec/NettyEncoder.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/codec/NettyEncoder.java index dd4e523a14..4e9836a26f 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/codec/NettyEncoder.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/codec/NettyEncoder.java @@ -28,6 +28,14 @@ import org.apache.dolphinscheduler.remote.command.Command; @Sharable public class NettyEncoder extends MessageToByteEncoder { + /** + * encode + * + * @param ctx channel handler context + * @param msg command + * @param out byte buffer + * @throws Exception + */ @Override protected void encode(ChannelHandlerContext ctx, Command msg, ByteBuf out) throws Exception { if(msg == null){ diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Command.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Command.java index 4687db39c0..ee95044764 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Command.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Command.java @@ -36,7 +36,7 @@ public class Command implements Serializable { } /** - * comman type + * command type */ private CommandType type; @@ -45,6 +45,9 @@ public class Command implements Serializable { */ private long opaque; + /** + * data body + */ private byte[] body; public CommandType getType() { diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandHeader.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandHeader.java index 92f7ac35dc..78948a5c0c 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandHeader.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandHeader.java @@ -23,10 +23,19 @@ import java.io.Serializable; */ public class CommandHeader implements Serializable { + /** + * type + */ private byte type; + /** + * request unique identification + */ private long opaque; + /** + * body length + */ private int bodyLength; public int getBodyLength() { diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandType.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandType.java index 185358a73c..b1b24d3303 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandType.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/CommandType.java @@ -1 +1 @@ -/* * 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; public enum CommandType { /** * roll view log request */ ROLL_VIEW_LOG_REQUEST, /** * roll view log response */ ROLL_VIEW_LOG_RESPONSE, /** * view whole log request */ VIEW_WHOLE_LOG_REQUEST, /** * view whole log response */ VIEW_WHOLE_LOG_RESPONSE, /** * get log bytes request */ GET_LOG_BYTES_REQUEST, /** * get log bytes response */ GET_LOG_BYTES_RESPONSE, WORKER_REQUEST, MASTER_RESPONSE, /** * execute task request */ EXECUTE_TASK_REQUEST, /** * execute task response */ EXECUTE_TASK_RESPONSE, PING, PONG; } \ 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; public enum CommandType { /** * roll view log request */ ROLL_VIEW_LOG_REQUEST, /** * roll view log response */ ROLL_VIEW_LOG_RESPONSE, /** * view whole log request */ VIEW_WHOLE_LOG_REQUEST, /** * view whole log response */ VIEW_WHOLE_LOG_RESPONSE, /** * get log bytes request */ GET_LOG_BYTES_REQUEST, /** * get log bytes response */ GET_LOG_BYTES_RESPONSE, WORKER_REQUEST, MASTER_RESPONSE, /** * execute task request */ EXECUTE_TASK_REQUEST, /** * execute task response */ EXECUTE_TASK_RESPONSE, /** * ping */ PING, /** * pong */ PONG; } \ No newline at end of file diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskRequestCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskRequestCommand.java index e75c2de379..a582221cd3 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskRequestCommand.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskRequestCommand.java @@ -1 +1 @@ -/* * 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; public class ExecuteTaskRequestCommand implements Serializable { private static final AtomicLong REQUEST = new AtomicLong(1); private String taskId; private String attemptId; private String applicationName; private String groupName; private String taskName; private int connectorPort; private String description; private String className; private String methodName; private String params; private List shardItems; public List getShardItems() { return shardItems; } public void setShardItems(List 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; } 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 { 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 shardItems; public List getShardItems() { return shardItems; } public void setShardItems(List 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 diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskResponseCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskResponseCommand.java index fafb57535b..0268653b5d 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskResponseCommand.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskResponseCommand.java @@ -1 +1 @@ -/* * 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.concurrent.atomic.AtomicLong; public class ExecuteTaskResponseCommand implements Serializable { private static final AtomicLong REQUEST = new AtomicLong(1); private String taskId; private String attemptId; private Object result; private long receivedTime; private int executeCount; private long executeTime; public String getAttemptId() { return attemptId; } public void setAttemptId(String attemptId) { this.attemptId = attemptId; } public String getTaskId() { return taskId; } public void setTaskId(String taskId) { this.taskId = taskId; } public Object getResult() { return result; } public void setResult(Object result) { this.result = result; } public long getReceivedTime() { return receivedTime; } public void setReceivedTime(long receivedTime) { this.receivedTime = receivedTime; } public int getExecuteCount() { return executeCount; } public void setExecuteCount(int executeCount) { this.executeCount = executeCount; } public long getExecuteTime() { return executeTime; } public void setExecuteTime(long executeTime) { this.executeTime = executeTime; } public Command convert2Command(){ Command command = new Command(REQUEST.getAndIncrement()); command.setType(CommandType.EXECUTE_TASK_RESPONSE); 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.concurrent.atomic.AtomicLong; /** * execute taks response command */ public class ExecuteTaskResponseCommand implements Serializable { private static final AtomicLong REQUEST = new AtomicLong(1); /** * task id */ private String taskId; /** * attempt id */ private String attemptId; /** * result info */ private Object result; /** * receive time */ private long receivedTime; /** * execute count */ private int executeCount; /** * execute time */ private long executeTime; public String getAttemptId() { return attemptId; } public void setAttemptId(String attemptId) { this.attemptId = attemptId; } public String getTaskId() { return taskId; } public void setTaskId(String taskId) { this.taskId = taskId; } public Object getResult() { return result; } public void setResult(Object result) { this.result = result; } public long getReceivedTime() { return receivedTime; } public void setReceivedTime(long receivedTime) { this.receivedTime = receivedTime; } public int getExecuteCount() { return executeCount; } public void setExecuteCount(int executeCount) { this.executeCount = executeCount; } public long getExecuteTime() { return executeTime; } public void setExecuteTime(long executeTime) { this.executeTime = executeTime; } /** * package response command * * @return command */ public Command convert2Command(){ Command command = new Command(REQUEST.getAndIncrement()); command.setType(CommandType.EXECUTE_TASK_RESPONSE); byte[] body = FastJsonSerializer.serialize(this); command.setBody(body); return command; } } \ No newline at end of file diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Ping.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Ping.java index 365d451564..4f32d5f699 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Ping.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Ping.java @@ -23,13 +23,21 @@ import io.netty.buffer.Unpooled; import java.io.Serializable; import java.util.concurrent.atomic.AtomicLong; - +/** + * ping machine + */ public class Ping implements Serializable { private static final AtomicLong ID = new AtomicLong(1); + /** + * ping body + */ protected static ByteBuf EMPTY_BODY = Unpooled.EMPTY_BUFFER; + /** + * request command body + */ private static byte[] EMPTY_BODY_ARRAY = new byte[0]; private static final ByteBuf PING_BUF; @@ -44,10 +52,19 @@ public class Ping implements Serializable { PING_BUF = Unpooled.unreleasableBuffer(ping).asReadOnly(); } + /** + * ping connect + * @return result + */ public static ByteBuf pingContent(){ return PING_BUF.duplicate(); } + /** + * package ping command + * + * @return command + */ public static Command create(){ Command command = new Command(ID.getAndIncrement()); command.setType(CommandType.PING); diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Pong.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Pong.java index bc5abdad79..e52cef6d92 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Pong.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Pong.java @@ -22,13 +22,24 @@ import io.netty.buffer.Unpooled; import java.io.Serializable; - +/** + * Pong return after ping + */ public class Pong implements Serializable { + /** + * pong body + */ protected static ByteBuf EMPTY_BODY = Unpooled.EMPTY_BUFFER; + /** + * pong command body + */ private static byte[] EMPTY_BODY_ARRAY = new byte[0]; + /** + * ping byte buffer + */ private static final ByteBuf PONG_BUF; static { @@ -41,10 +52,20 @@ public class Pong implements Serializable { PONG_BUF = Unpooled.unreleasableBuffer(ping).asReadOnly(); } + /** + * ping content + * @return result + */ public static ByteBuf pingContent(){ return PONG_BUF.duplicate(); } + /** + * package pong command + * + * @param opaque request unique identification + * @return command + */ public static Command create(long opaque){ Command command = new Command(opaque); command.setType(CommandType.PONG); diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesRequestCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesRequestCommand.java index 1a2e6e4dd1..9b064b7136 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesRequestCommand.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesRequestCommand.java @@ -29,8 +29,14 @@ import java.util.concurrent.atomic.AtomicLong; */ public class GetLogBytesRequestCommand implements Serializable { + /** + * request id + */ private static final AtomicLong REQUEST = new AtomicLong(1); + /** + * log path + */ private String path; public GetLogBytesRequestCommand() { @@ -49,8 +55,9 @@ public class GetLogBytesRequestCommand implements Serializable { } /** + * package request command * - * @return + * @return command */ public Command convert2Command(){ Command command = new Command(REQUEST.getAndIncrement()); diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesResponseCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesResponseCommand.java index 05692fb5c9..deaf9b8d85 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesResponseCommand.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesResponseCommand.java @@ -28,6 +28,9 @@ import java.io.Serializable; */ public class GetLogBytesResponseCommand implements Serializable { + /** + * log byte data + */ private byte[] data; public GetLogBytesResponseCommand() { @@ -45,6 +48,12 @@ public class GetLogBytesResponseCommand implements Serializable { this.data = data; } + /** + * package response command + * + * @param opaque request unique identification + * @return command + */ public Command convert2Command(long opaque){ Command command = new Command(opaque); command.setType(CommandType.GET_LOG_BYTES_RESPONSE); diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogRequestCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogRequestCommand.java index 49d19aa1f2..f072c479f4 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogRequestCommand.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogRequestCommand.java @@ -29,12 +29,24 @@ import java.util.concurrent.atomic.AtomicLong; */ public class RollViewLogRequestCommand implements Serializable { + /** + * request id + */ private static final AtomicLong REQUEST = new AtomicLong(1); + /** + * log path + */ private String path; + /** + * skip line number + */ private int skipLineNum; + /** + * query log line number limit + */ private int limit; public RollViewLogRequestCommand() { @@ -70,6 +82,11 @@ public class RollViewLogRequestCommand implements Serializable { this.limit = limit; } + /** + * package request command + * + * @return command + */ public Command convert2Command(){ Command command = new Command(REQUEST.getAndIncrement()); command.setType(CommandType.ROLL_VIEW_LOG_REQUEST); diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogResponseCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogResponseCommand.java index def3257073..591d787200 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogResponseCommand.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogResponseCommand.java @@ -28,6 +28,9 @@ import java.io.Serializable; */ public class RollViewLogResponseCommand implements Serializable { + /** + * response data + */ private String msg; public RollViewLogResponseCommand() { @@ -45,6 +48,12 @@ public class RollViewLogResponseCommand implements Serializable { this.msg = msg; } + /** + * package response command + * + * @param opaque request unique identification + * @return command + */ public Command convert2Command(long opaque){ Command command = new Command(opaque); command.setType(CommandType.ROLL_VIEW_LOG_RESPONSE); diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogRequestCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogRequestCommand.java index 9ba9cd3c23..5dcefc6233 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogRequestCommand.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogRequestCommand.java @@ -29,6 +29,9 @@ import java.util.concurrent.atomic.AtomicLong; */ public class ViewLogRequestCommand implements Serializable { + /** + * request id + */ private static final AtomicLong REQUEST = new AtomicLong(1); private String path; @@ -48,6 +51,11 @@ public class ViewLogRequestCommand implements Serializable { this.path = path; } + /** + * package request command + * + * @return command + */ public Command convert2Command(){ Command command = new Command(REQUEST.getAndIncrement()); command.setType(CommandType.VIEW_WHOLE_LOG_REQUEST); diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogResponseCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogResponseCommand.java index 6e3c799a3d..dffadade26 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogResponseCommand.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogResponseCommand.java @@ -28,6 +28,9 @@ import java.io.Serializable; */ public class ViewLogResponseCommand implements Serializable { + /** + * response data + */ private String msg; public ViewLogResponseCommand() { @@ -45,6 +48,12 @@ public class ViewLogResponseCommand implements Serializable { this.msg = msg; } + /** + * package response command + * + * @param opaque request unique identification + * @return command + */ public Command convert2Command(long opaque){ Command command = new Command(opaque); command.setType(CommandType.VIEW_WHOLE_LOG_RESPONSE); diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/config/NettyClientConfig.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/config/NettyClientConfig.java index 56d2643a67..831e05f7e7 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/config/NettyClientConfig.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/config/NettyClientConfig.java @@ -23,14 +23,29 @@ import org.apache.dolphinscheduler.remote.utils.Constants; */ public class NettyClientConfig { + /** + * worker threads,default get machine cpus + */ private int workerThreads = Constants.CPUS; + /** + * whether tpc delay + */ private boolean tcpNoDelay = true; + /** + * whether keep alive + */ private boolean soKeepalive = true; + /** + * send buffer size + */ private int sendBufferSize = 65535; + /** + * receive buffer size + */ private int receiveBufferSize = 65535; public int getWorkerThreads() { diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/config/NettyServerConfig.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/config/NettyServerConfig.java index 847f316089..4ec8a0f7a7 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/config/NettyServerConfig.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/config/NettyServerConfig.java @@ -23,18 +23,39 @@ import org.apache.dolphinscheduler.remote.utils.Constants; */ public class NettyServerConfig { + /** + * init the server connectable queue + */ private int soBacklog = 1024; + /** + * whether tpc delay + */ private boolean tcpNoDelay = true; + /** + * whether keep alive + */ private boolean soKeepalive = true; + /** + * send buffer size + */ private int sendBufferSize = 65535; + /** + * receive buffer size + */ private int receiveBufferSize = 65535; + /** + * worker threads,default get machine cpus + */ private int workerThread = Constants.CPUS; + /** + * listen port + */ private int listenPort = 12346; public int getListenPort() { diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java index b06308090f..6aceb5a41b 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java @@ -38,29 +38,62 @@ public class NettyClientHandler extends ChannelInboundHandlerAdapter { private final Logger logger = LoggerFactory.getLogger(NettyClientHandler.class); + /** + * netty remote client + */ private final NettyRemotingClient nettyRemotingClient; + /** + * client processors queue + */ private final ConcurrentHashMap> processors = new ConcurrentHashMap(); public NettyClientHandler(NettyRemotingClient nettyRemotingClient){ this.nettyRemotingClient = nettyRemotingClient; } + /** + * When the current channel is not active, + * the current channel has reached the end of its life cycle + * + * @param ctx channel handler context + * @throws Exception + */ @Override public void channelInactive(ChannelHandlerContext ctx) throws Exception { nettyRemotingClient.removeChannel(ChannelUtils.toAddress(ctx.channel())); ctx.channel().close(); } + /** + * The current channel reads data from the remote + * + * @param ctx channel handler context + * @param msg message + * @throws Exception + */ @Override public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { processReceived(ctx.channel(), (Command)msg); } + /** + * register processor + * + * @param commandType command type + * @param processor processor + */ public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor) { this.registerProcessor(commandType, processor, nettyRemotingClient.getDefaultExecutor()); } + /** + * register processor + * + * @param commandType command type + * @param processor processor + * @param executor thread executor + */ public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor, final ExecutorService executor) { ExecutorService executorRef = executor; if(executorRef == null){ @@ -69,6 +102,12 @@ public class NettyClientHandler extends ChannelInboundHandlerAdapter { this.processors.putIfAbsent(commandType, new Pair(processor, executorRef)); } + /** + * process received logic + * + * @param channel channel + * @param msg message + */ private void processReceived(final Channel channel, final Command msg) { final CommandType commandType = msg.getType(); final Pair pair = processors.get(commandType); @@ -93,6 +132,13 @@ public class NettyClientHandler extends ChannelInboundHandlerAdapter { } } + /** + * caught exception + * + * @param ctx channel handler context + * @param cause cause + * @throws Exception + */ @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { logger.error("exceptionCaught : {}", cause); @@ -100,6 +146,11 @@ public class NettyClientHandler extends ChannelInboundHandlerAdapter { ctx.channel().close(); } + /** + * channel write changed + * @param ctx channel handler context + * @throws Exception + */ @Override public void channelWritabilityChanged(ChannelHandlerContext ctx) throws Exception { Channel ch = ctx.channel(); diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyServerHandler.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyServerHandler.java index 8a7ee39a77..eabd6560de 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyServerHandler.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyServerHandler.java @@ -39,28 +39,60 @@ public class NettyServerHandler extends ChannelInboundHandlerAdapter { private final Logger logger = LoggerFactory.getLogger(NettyServerHandler.class); + /** + * netty remote server + */ private final NettyRemotingServer nettyRemotingServer; + /** + * server processors queue + */ private final ConcurrentHashMap> processors = new ConcurrentHashMap(); public NettyServerHandler(NettyRemotingServer nettyRemotingServer){ this.nettyRemotingServer = nettyRemotingServer; } + /** + * When the current channel is not active, + * the current channel has reached the end of its life cycle + * @param ctx channel handler context + * @throws Exception + */ @Override public void channelInactive(ChannelHandlerContext ctx) throws Exception { ctx.channel().close(); } + /** + * The current channel reads data from the remote end + * + * @param ctx channel handler context + * @param msg message + * @throws Exception + */ @Override public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { processReceived(ctx.channel(), (Command)msg); } + /** + * register processor + * + * @param commandType command type + * @param processor processor + */ public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor) { this.registerProcessor(commandType, processor, null); } + /** + * register processor + * + * @param commandType command type + * @param processor processor + * @param executor thread executor + */ public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor, final ExecutorService executor) { ExecutorService executorRef = executor; if(executorRef == null){ @@ -69,11 +101,18 @@ public class NettyServerHandler extends ChannelInboundHandlerAdapter { this.processors.putIfAbsent(commandType, new Pair(processor, executorRef)); } + /** + * process received logic + * @param channel channel + * @param msg message + */ private void processReceived(final Channel channel, final Command msg) { final CommandType commandType = msg.getType(); final Pair pair = processors.get(commandType); if (pair != null) { Runnable r = new Runnable() { + + @Override public void run() { try { pair.getLeft().process(channel, msg); @@ -92,12 +131,25 @@ public class NettyServerHandler extends ChannelInboundHandlerAdapter { } } + /** + * caught exception + * + * @param ctx channel handler context + * @param cause cause + * @throws Exception + */ @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { logger.error("exceptionCaught : {}", cause); ctx.channel().close(); } + /** + * channel write changed + * + * @param ctx channel handler context + * @throws Exception + */ @Override public void channelWritabilityChanged(ChannelHandlerContext ctx) throws Exception { Channel ch = ctx.channel(); diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/NettyRequestProcessor.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/NettyRequestProcessor.java index 10a8195710..6966b53d17 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/NettyRequestProcessor.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/processor/NettyRequestProcessor.java @@ -24,5 +24,10 @@ import org.apache.dolphinscheduler.remote.command.Command; */ public interface NettyRequestProcessor { + /** + * process logic + * @param channel channel + * @param command command + */ void process(final Channel channel, final Command command); } diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Address.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Address.java index 221b895cb9..f61dcd615c 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Address.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Address.java @@ -23,8 +23,14 @@ import java.io.Serializable; */ public class Address implements Serializable { + /** + * host + */ private String host; + /** + * port + */ private int port; public Address(){ diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/ChannelUtils.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/ChannelUtils.java index e9d93da41d..d7af5fe165 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/ChannelUtils.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/ChannelUtils.java @@ -25,14 +25,30 @@ import java.net.InetSocketAddress; */ public class ChannelUtils { + /** + * get local address + * + * @param channel channel + * @return local address + */ public static String getLocalAddress(Channel channel){ return ((InetSocketAddress)channel.localAddress()).getAddress().getHostAddress(); } + /** + * get remote address + * @param channel channel + * @return remote address + */ public static String getRemoteAddress(Channel channel){ return ((InetSocketAddress)channel.remoteAddress()).getAddress().getHostAddress(); } + /** + * channel to address + * @param channel channel + * @return address + */ public static Address toAddress(Channel channel){ InetSocketAddress socketAddress = ((InetSocketAddress)channel.remoteAddress()); return new Address(socketAddress.getAddress().getHostAddress(), socketAddress.getPort()); diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Constants.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Constants.java index c0a930ca41..5733b17790 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Constants.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Constants.java @@ -18,14 +18,24 @@ package org.apache.dolphinscheduler.remote.utils; import java.nio.charset.Charset; + +/** + * constant + */ public class Constants { public static final String COMMA = ","; public static final String SLASH = "/"; + /** + * charset + */ public static final Charset UTF8 = Charset.forName("UTF-8"); + /** + * cpus + */ public static final int CPUS = Runtime.getRuntime().availableProcessors(); } diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/FastJsonSerializer.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/FastJsonSerializer.java index a9b85461ff..e96796a05c 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/FastJsonSerializer.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/FastJsonSerializer.java @@ -23,15 +23,36 @@ import com.alibaba.fastjson.JSON; */ public class FastJsonSerializer { + /** + * serialize to byte + * + * @param obj object + * @param object type + * @return byte array + */ public static byte[] serialize(T obj) { String json = JSON.toJSONString(obj); return json.getBytes(Constants.UTF8); } + /** + * serialize to string + * @param obj object + * @param object type + * @return string + */ public static String serializeToString(T obj) { return JSON.toJSONString(obj); } + /** + * deserialize + * + * @param src byte array + * @param clazz class + * @param deserialize type + * @return deserialize type + */ public static T deserialize(byte[] src, Class clazz) { return JSON.parseObject(new String(src, Constants.UTF8), clazz); } diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Pair.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Pair.java index a79a3748cd..2042191486 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Pair.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Pair.java @@ -18,6 +18,12 @@ package org.apache.dolphinscheduler.remote.utils; +/** + * key value pair + * + * @param L generic type + * @param R generic type + */ public class Pair { private L left; diff --git a/dolphinscheduler-remote/src/test/java/org/apache/dolphinscheduler/remote/NettyRemotingClientTest.java b/dolphinscheduler-remote/src/test/java/org/apache/dolphinscheduler/remote/NettyRemotingClientTest.java index 19fd564bd5..6f0a802af6 100644 --- a/dolphinscheduler-remote/src/test/java/org/apache/dolphinscheduler/remote/NettyRemotingClientTest.java +++ b/dolphinscheduler-remote/src/test/java/org/apache/dolphinscheduler/remote/NettyRemotingClientTest.java @@ -32,9 +32,15 @@ import org.junit.Test; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicLong; +/** + * netty remote client test + */ public class NettyRemotingClientTest { + /** + * test ping + */ @Test public void testSend(){ NettyServerConfig serverConfig = new NettyServerConfig(); From ad002682c3b950717189efd64aa0056415068c2d Mon Sep 17 00:00:00 2001 From: qiaozhanwei Date: Thu, 13 Feb 2020 18:35:59 +0800 Subject: [PATCH 014/221] OSUtilsTest modify --- .../org/apache/dolphinscheduler/common/utils/OSUtilsTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/OSUtilsTest.java b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/OSUtilsTest.java index 5b23847ba3..3d51aa830f 100644 --- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/OSUtilsTest.java +++ b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/OSUtilsTest.java @@ -19,7 +19,6 @@ package org.apache.dolphinscheduler.common.utils; import org.apache.commons.configuration.Configuration; import org.apache.commons.configuration.PropertiesConfiguration; import org.apache.dolphinscheduler.common.Constants; -import org.apache.yetus.audience.InterfaceAudience; import org.junit.Assert; import org.junit.Test; import org.slf4j.Logger; From e39d6ba9730b190dd36bc03b5395f82707fb6de6 Mon Sep 17 00:00:00 2001 From: Technoboy- Date: Thu, 13 Feb 2020 21:26:35 +0800 Subject: [PATCH 015/221] add finally block to close channel --- .../api/service/LoggerService.java | 27 ++++++++++++++----- .../service/ProcessDefinitionServiceTest.java | 1 - .../server/utils/ProcessUtils.java | 13 ++++++--- .../service/log/LogClientService.java | 6 ++--- 4 files changed, 33 insertions(+), 14 deletions(-) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/LoggerService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/LoggerService.java index 056af8eff5..bff54b6c21 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/LoggerService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/LoggerService.java @@ -64,11 +64,17 @@ 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 = new LogClientService(host, Constants.RPC_PORT); - String log = logClient.rollViewLog(taskInstance.getLogPath(),skipLineNum,limit); - result.setData(log); - logger.info(log); + LogClientService logClient = null; + try { + logClient = new LogClientService(host, Constants.RPC_PORT); + String log = logClient.rollViewLog(taskInstance.getLogPath(),skipLineNum,limit); + result.setData(log); + logger.info(log); + } finally { + if(logClient != null){ + logClient.close(); + } + } return result; } @@ -86,7 +92,14 @@ public class LoggerService { } String host = taskInstance.getHost(); - LogClientService logClient = new LogClientService(host, Constants.RPC_PORT); - return logClient.getLogBytes(taskInstance.getLogPath()); + LogClientService logClient = null; + try { + logClient = new LogClientService(host, Constants.RPC_PORT); + return logClient.getLogBytes(taskInstance.getLogPath()); + } finally { + if(logClient != null){ + logClient.close(); + } + } } } diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionServiceTest.java index 3e9dcb026e..ccbbc36ea4 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionServiceTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionServiceTest.java @@ -27,7 +27,6 @@ import org.apache.dolphinscheduler.common.enums.*; import org.apache.dolphinscheduler.common.utils.DateUtils; import org.apache.dolphinscheduler.common.utils.FileUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.*; import org.apache.dolphinscheduler.dao.mapper.*; import org.apache.dolphinscheduler.service.process.ProcessService; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ProcessUtils.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ProcessUtils.java index 69284ee69c..90711e1d14 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ProcessUtils.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ProcessUtils.java @@ -375,9 +375,16 @@ public class ProcessUtils { public static void killYarnJob(TaskInstance taskInstance) { try { Thread.sleep(Constants.SLEEP_TIME_MILLIS); - LogClientService logClient = new LogClientService(taskInstance.getHost(), Constants.RPC_PORT); - - String log = logClient.viewLog(taskInstance.getLogPath()); + LogClientService logClient = null; + String log = null; + try { + logClient = new LogClientService(taskInstance.getHost(), Constants.RPC_PORT); + log = logClient.viewLog(taskInstance.getLogPath()); + } finally { + if(logClient != null){ + logClient.close(); + } + } if (StringUtils.isNotEmpty(log)) { List appIds = LoggerUtils.getAppIds(log, logger); String workerDir = taskInstance.getExecutePath(); diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java index a316c7046d..aa6999ef0d 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java @@ -64,11 +64,11 @@ public class LogClientService implements NettyRequestProcessor { } /** - * shutdown + * close */ - public void shutdown() { + public void close() { this.client.close(); - logger.info("logger client shutdown"); + logger.info("logger client closed"); } /** From 1e812ca07884d384952573a654e47e098a2930dc Mon Sep 17 00:00:00 2001 From: Tboy Date: Thu, 13 Feb 2020 21:29:11 +0800 Subject: [PATCH 016/221] 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 --- .../api/service/LoggerService.java | 27 ++++++++++++++----- .../service/ProcessDefinitionServiceTest.java | 1 - .../server/utils/ProcessUtils.java | 13 ++++++--- .../service/log/LogClientService.java | 6 ++--- 4 files changed, 33 insertions(+), 14 deletions(-) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/LoggerService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/LoggerService.java index 056af8eff5..bff54b6c21 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/LoggerService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/LoggerService.java @@ -64,11 +64,17 @@ 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 = new LogClientService(host, Constants.RPC_PORT); - String log = logClient.rollViewLog(taskInstance.getLogPath(),skipLineNum,limit); - result.setData(log); - logger.info(log); + LogClientService logClient = null; + try { + logClient = new LogClientService(host, Constants.RPC_PORT); + String log = logClient.rollViewLog(taskInstance.getLogPath(),skipLineNum,limit); + result.setData(log); + logger.info(log); + } finally { + if(logClient != null){ + logClient.close(); + } + } return result; } @@ -86,7 +92,14 @@ public class LoggerService { } String host = taskInstance.getHost(); - LogClientService logClient = new LogClientService(host, Constants.RPC_PORT); - return logClient.getLogBytes(taskInstance.getLogPath()); + LogClientService logClient = null; + try { + logClient = new LogClientService(host, Constants.RPC_PORT); + return logClient.getLogBytes(taskInstance.getLogPath()); + } finally { + if(logClient != null){ + logClient.close(); + } + } } } diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionServiceTest.java index 3e9dcb026e..ccbbc36ea4 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionServiceTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessDefinitionServiceTest.java @@ -27,7 +27,6 @@ import org.apache.dolphinscheduler.common.enums.*; import org.apache.dolphinscheduler.common.utils.DateUtils; import org.apache.dolphinscheduler.common.utils.FileUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils; -import org.apache.dolphinscheduler.dao.ProcessDao; import org.apache.dolphinscheduler.dao.entity.*; import org.apache.dolphinscheduler.dao.mapper.*; import org.apache.dolphinscheduler.service.process.ProcessService; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ProcessUtils.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ProcessUtils.java index 69284ee69c..90711e1d14 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ProcessUtils.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ProcessUtils.java @@ -375,9 +375,16 @@ public class ProcessUtils { public static void killYarnJob(TaskInstance taskInstance) { try { Thread.sleep(Constants.SLEEP_TIME_MILLIS); - LogClientService logClient = new LogClientService(taskInstance.getHost(), Constants.RPC_PORT); - - String log = logClient.viewLog(taskInstance.getLogPath()); + LogClientService logClient = null; + String log = null; + try { + logClient = new LogClientService(taskInstance.getHost(), Constants.RPC_PORT); + log = logClient.viewLog(taskInstance.getLogPath()); + } finally { + if(logClient != null){ + logClient.close(); + } + } if (StringUtils.isNotEmpty(log)) { List appIds = LoggerUtils.getAppIds(log, logger); String workerDir = taskInstance.getExecutePath(); diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java index a316c7046d..aa6999ef0d 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java @@ -64,11 +64,11 @@ public class LogClientService implements NettyRequestProcessor { } /** - * shutdown + * close */ - public void shutdown() { + public void close() { this.client.close(); - logger.info("logger client shutdown"); + logger.info("logger client closed"); } /** From 8e154ecbe46035d9338371d4a8d8bea9fccc81d3 Mon Sep 17 00:00:00 2001 From: Technoboy- Date: Thu, 13 Feb 2020 22:06:34 +0800 Subject: [PATCH 017/221] refactor log client service --- .../api/service/LoggerService.java | 38 ++++++++---------- .../remote/NettyRemotingClient.java | 17 +++++++- .../remote/future/InvokeCallback.java | 10 +++++ .../remote/future/ResponseFuture.java | 29 ++++++++++++++ .../remote/handler/NettyClientHandler.java | 4 +- .../server/utils/ProcessUtils.java | 4 +- .../service/log/LogClientService.java | 40 ++++++++++--------- 7 files changed, 97 insertions(+), 45 deletions(-) create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/InvokeCallback.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/ResponseFuture.java diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/LoggerService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/LoggerService.java index bff54b6c21..f20f6574c3 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/LoggerService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/LoggerService.java @@ -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,17 +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); - result.setData(log); - logger.info(log); - } finally { - if(logClient != null){ - logClient.close(); - } - } + String log = logClient.rollViewLog(host, Constants.RPC_PORT, taskInstance.getLogPath(),skipLineNum,limit); + result.setData(log); + logger.info(log); return result; } @@ -90,16 +95,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()); } } diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java index 678fe84f90..f8357a388a 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java @@ -132,6 +132,21 @@ public class NettyRemotingClient { } } + //TODO + public void sendSync(final Address address, final Command command, final long timeoutMillis) throws RemotingException { + final Channel channel = getChannel(address); + if (channel == null) { + throw new RemotingException("network error"); + } + final long opaque = command.getOpaque(); + try { + + } catch (Exception ex) { + String msg = String.format("send command %s to address %s encounter error", command, address); + throw new RemotingException(msg, ex); + } + } + public Channel getChannel(Address address) { Channel channel = channels.get(address); if(channel != null && channel.isActive()){ @@ -188,7 +203,7 @@ public class NettyRemotingClient { this.channels.clear(); } - public void removeChannel(Address address){ + public void closeChannel(Address address){ Channel channel = this.channels.remove(address); if(channel != null){ channel.close(); diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/InvokeCallback.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/InvokeCallback.java new file mode 100644 index 0000000000..6ad6a7cbee --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/InvokeCallback.java @@ -0,0 +1,10 @@ +package org.apache.dolphinscheduler.remote.future; + +/** + * @Author: Tboy + */ +public interface InvokeCallback { + + void operationComplete(final ResponseFuture responseFuture); + +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/ResponseFuture.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/ResponseFuture.java new file mode 100644 index 0000000000..036f9906b1 --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/ResponseFuture.java @@ -0,0 +1,29 @@ +package org.apache.dolphinscheduler.remote.future; + +import org.apache.dolphinscheduler.remote.command.Command; + +import java.util.concurrent.CountDownLatch; + +/** + * @Author: Tboy + */ +public class ResponseFuture { + + private final int opaque; + + private final long timeoutMillis; + + private final InvokeCallback invokeCallback; + + private final long beginTimestamp = System.currentTimeMillis(); + + private final CountDownLatch latch = new CountDownLatch(1); + + public ResponseFuture(int opaque, long timeoutMillis, InvokeCallback invokeCallback) { + this.opaque = opaque; + this.timeoutMillis = timeoutMillis; + this.invokeCallback = invokeCallback; + } + + +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java index b06308090f..115f6e4953 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java @@ -48,7 +48,7 @@ public class NettyClientHandler extends ChannelInboundHandlerAdapter { @Override public void channelInactive(ChannelHandlerContext ctx) throws Exception { - nettyRemotingClient.removeChannel(ChannelUtils.toAddress(ctx.channel())); + nettyRemotingClient.closeChannel(ChannelUtils.toAddress(ctx.channel())); ctx.channel().close(); } @@ -96,7 +96,7 @@ public class NettyClientHandler extends ChannelInboundHandlerAdapter { @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { logger.error("exceptionCaught : {}", cause); - nettyRemotingClient.removeChannel(ChannelUtils.toAddress(ctx.channel())); + nettyRemotingClient.closeChannel(ChannelUtils.toAddress(ctx.channel())); ctx.channel().close(); } diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ProcessUtils.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ProcessUtils.java index 90711e1d14..e0c00c55d9 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ProcessUtils.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ProcessUtils.java @@ -378,8 +378,8 @@ public class ProcessUtils { LogClientService logClient = null; String log = null; try { - logClient = new LogClientService(taskInstance.getHost(), Constants.RPC_PORT); - log = logClient.viewLog(taskInstance.getLogPath()); + logClient = new LogClientService(); + log = logClient.viewLog(taskInstance.getHost(), Constants.RPC_PORT, taskInstance.getLogPath()); } finally { if(logClient != null){ logClient.close(); diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java index aa6999ef0d..4faff335fe 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java @@ -40,8 +40,6 @@ public class LogClientService implements NettyRequestProcessor { private final NettyRemotingClient client; - private final Address address; - /** * request time out */ @@ -49,18 +47,14 @@ public class LogClientService implements NettyRequestProcessor { /** * construct client - * @param host host - * @param port port */ - public LogClientService(String host, int port) { - this.address = new Address(host, port); + public LogClientService() { this.clientConfig = new NettyClientConfig(); - this.clientConfig.setWorkerThreads(1); + this.clientConfig.setWorkerThreads(4); this.client = new NettyRemotingClient(clientConfig); this.client.registerProcessor(CommandType.ROLL_VIEW_LOG_RESPONSE,this); this.client.registerProcessor(CommandType.VIEW_WHOLE_LOG_RESPONSE, this); this.client.registerProcessor(CommandType.GET_LOG_BYTES_RESPONSE, this); - } /** @@ -73,15 +67,18 @@ public class LogClientService implements NettyRequestProcessor { /** * roll view log + * @param host host + * @param port port * @param path path * @param skipLineNum skip line number * @param limit limit * @return log content */ - public String rollViewLog(String path,int skipLineNum,int limit) { - logger.info("roll view log, path {}, skipLineNum {} ,limit {}", path, skipLineNum, limit); + public String rollViewLog(String host, int port, String path,int skipLineNum,int limit) { + logger.info("roll view log, host : {}, port : {}, path {}, skipLineNum {} ,limit {}", host, port, path, skipLineNum, limit); RollViewLogRequestCommand request = new RollViewLogRequestCommand(path, skipLineNum, limit); String result = ""; + final Address address = new Address(host, port); try { Command command = request.convert2Command(); this.client.send(address, command); @@ -89,19 +86,24 @@ public class LogClientService implements NettyRequestProcessor { result = ((String)promise.getResult()); } catch (Exception e) { logger.error("roll view log error", e); + } finally { + this.client.closeChannel(address); } return result; } /** * view log + * @param host host + * @param port port * @param path path * @return log content */ - public String viewLog(String path) { + public String viewLog(String host, int port, String path) { logger.info("view log path {}", path); ViewLogRequestCommand request = new ViewLogRequestCommand(path); String result = ""; + final Address address = new Address(host, port); try { Command command = request.convert2Command(); this.client.send(address, command); @@ -109,19 +111,24 @@ public class LogClientService implements NettyRequestProcessor { result = ((String)promise.getResult()); } catch (Exception e) { logger.error("view log error", e); + } finally { + this.client.closeChannel(address); } return result; } /** * get log size + * @param host host + * @param port port * @param path log path * @return log content bytes */ - public byte[] getLogBytes(String path) { + public byte[] getLogBytes(String host, int port, String path) { logger.info("log path {}", path); GetLogBytesRequestCommand request = new GetLogBytesRequestCommand(path); byte[] result = null; + final Address address = new Address(host, port); try { Command command = request.convert2Command(); this.client.send(address, command); @@ -129,6 +136,8 @@ public class LogClientService implements NettyRequestProcessor { result = (byte[])promise.getResult(); } catch (Exception e) { logger.error("get log size error", e); + } finally { + this.client.closeChannel(address); } return result; } @@ -156,11 +165,4 @@ public class LogClientService implements NettyRequestProcessor { throw new UnsupportedOperationException(String.format("command type : %s is not supported ", command.getType())); } } - - public static void main(String[] args) throws Exception{ - LogClientService logClient = new LogClientService("192.168.220.247", 50051); - byte[] logBytes = logClient.getLogBytes("/opt/program/incubator-dolphinscheduler/logs/1/463/540.log"); - System.out.println(new String(logBytes)); - } - } \ No newline at end of file From 820b84bb46b68353f78e0243bf438c3faef9b776 Mon Sep 17 00:00:00 2001 From: Technoboy- Date: Fri, 14 Feb 2020 21:46:25 +0800 Subject: [PATCH 018/221] add sendSync method --- .../remote/NettyRemotingClient.java | 49 ++++++++---- .../remote/command/Command.java | 6 +- .../command/ExecuteTaskRequestCommand.java | 2 +- .../command/ExecuteTaskResponseCommand.java | 2 +- .../dolphinscheduler/remote/command/Ping.java | 4 +- .../log/GetLogBytesRequestCommand.java | 4 +- .../log/RollViewLogRequestCommand.java | 4 +- .../command/log/ViewLogRequestCommand.java | 4 +- .../exceptions/RemotingTimeoutException.java | 20 +++++ .../remote/future/ResponseFuture.java | 78 ++++++++++++++++++- .../remote/handler/NettyClientHandler.java | 52 +++---------- .../remote/NettyRemotingClientTest.java | 14 +--- .../service/log/LogClientService.java | 59 +++++--------- 13 files changed, 169 insertions(+), 129 deletions(-) create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingTimeoutException.java diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java index f8357a388a..3dcb8c0825 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java @@ -25,17 +25,19 @@ 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.future.InvokeCallback; +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.slf4j.Logger; import org.slf4j.LoggerFactory; import java.net.InetSocketAddress; +import java.rmi.RemoteException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -101,15 +103,8 @@ public class NettyRemotingClient { isStarted.compareAndSet(false, true); } - public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor) { - registerProcessor(commandType, processor, null); - } - - public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor, final ExecutorService executor) { - this.clientHandler.registerProcessor(commandType, processor, executor); - } - - public void send(final Address address, final Command command) throws RemotingException { + //TODO + public void send(final Address address, final Command command, final InvokeCallback invokeCallback) throws RemotingException { final Channel channel = getChannel(address); if (channel == null) { throw new RemotingException("network error"); @@ -132,17 +127,39 @@ public class NettyRemotingClient { } } - //TODO - public void sendSync(final Address address, final Command command, final long timeoutMillis) throws RemotingException { + public Command sendSync(final Address address, final Command command, final long timeoutMillis) throws RemotingException { final Channel channel = getChannel(address); if (channel == null) { - throw new RemotingException("network error"); + throw new RemotingException(String.format("connect to : %s fail", address)); } final long opaque = command.getOpaque(); try { - + final ResponseFuture responseFuture = new ResponseFuture(opaque, timeoutMillis, null); + channel.writeAndFlush(command).addListener(new ChannelFutureListener() { + @Override + public void operationComplete(ChannelFuture channelFuture) throws Exception { + if(channelFuture.isSuccess()){ + responseFuture.setSendOk(true); + return; + } else{ + responseFuture.setSendOk(false); + responseFuture.setCause(channelFuture.cause()); + responseFuture.putResponse(null); + logger.error("send command {} to address {} failed", command, address); + } + } + }); + Command result = responseFuture.waitResponse(); + if(result == null){ + if(responseFuture.isSendOK()){ + throw new RemotingTimeoutException(address.toString(), timeoutMillis, responseFuture.getCause()); + } else{ + throw new RemoteException(address.toString(), responseFuture.getCause()); + } + } + return result; } catch (Exception ex) { - String msg = String.format("send command %s to address %s encounter error", command, address); + String msg = String.format("send command %s to address %s error", command, address); throw new RemotingException(msg, ex); } } diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Command.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Command.java index 4687db39c0..9ed3c7d1c2 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Command.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Command.java @@ -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){ diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskRequestCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskRequestCommand.java index e75c2de379..b3801aceb0 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskRequestCommand.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskRequestCommand.java @@ -1 +1 @@ -/* * 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; public class ExecuteTaskRequestCommand implements Serializable { private static final AtomicLong REQUEST = new AtomicLong(1); private String taskId; private String attemptId; private String applicationName; private String groupName; private String taskName; private int connectorPort; private String description; private String className; private String methodName; private String params; private List shardItems; public List getShardItems() { return shardItems; } public void setShardItems(List 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; } 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; public class ExecuteTaskRequestCommand implements Serializable { private String taskId; private String attemptId; private String applicationName; private String groupName; private String taskName; private int connectorPort; private String description; private String className; private String methodName; private String params; private List shardItems; public List getShardItems() { return shardItems; } public void setShardItems(List 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; } 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 diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskResponseCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskResponseCommand.java index fafb57535b..aeb5f7d858 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskResponseCommand.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskResponseCommand.java @@ -1 +1 @@ -/* * 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.concurrent.atomic.AtomicLong; public class ExecuteTaskResponseCommand implements Serializable { private static final AtomicLong REQUEST = new AtomicLong(1); private String taskId; private String attemptId; private Object result; private long receivedTime; private int executeCount; private long executeTime; public String getAttemptId() { return attemptId; } public void setAttemptId(String attemptId) { this.attemptId = attemptId; } public String getTaskId() { return taskId; } public void setTaskId(String taskId) { this.taskId = taskId; } public Object getResult() { return result; } public void setResult(Object result) { this.result = result; } public long getReceivedTime() { return receivedTime; } public void setReceivedTime(long receivedTime) { this.receivedTime = receivedTime; } public int getExecuteCount() { return executeCount; } public void setExecuteCount(int executeCount) { this.executeCount = executeCount; } public long getExecuteTime() { return executeTime; } public void setExecuteTime(long executeTime) { this.executeTime = executeTime; } public Command convert2Command(){ Command command = new Command(REQUEST.getAndIncrement()); command.setType(CommandType.EXECUTE_TASK_RESPONSE); 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.concurrent.atomic.AtomicLong; public class ExecuteTaskResponseCommand implements Serializable { private String taskId; private String attemptId; private Object result; private long receivedTime; private int executeCount; private long executeTime; public String getAttemptId() { return attemptId; } public void setAttemptId(String attemptId) { this.attemptId = attemptId; } public String getTaskId() { return taskId; } public void setTaskId(String taskId) { this.taskId = taskId; } public Object getResult() { return result; } public void setResult(Object result) { this.result = result; } public long getReceivedTime() { return receivedTime; } public void setReceivedTime(long receivedTime) { this.receivedTime = receivedTime; } public int getExecuteCount() { return executeCount; } public void setExecuteCount(int executeCount) { this.executeCount = executeCount; } public long getExecuteTime() { return executeTime; } public void setExecuteTime(long executeTime) { this.executeTime = executeTime; } public Command convert2Command(long opaque){ Command command = new Command(); command.setType(CommandType.EXECUTE_TASK_RESPONSE); byte[] body = FastJsonSerializer.serialize(this); command.setBody(body); return command; } } \ No newline at end of file diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Ping.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Ping.java index 365d451564..38a8b14871 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Ping.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Ping.java @@ -26,8 +26,6 @@ import java.util.concurrent.atomic.AtomicLong; public class Ping implements Serializable { - private static final AtomicLong ID = new AtomicLong(1); - protected static ByteBuf EMPTY_BODY = Unpooled.EMPTY_BUFFER; private static byte[] EMPTY_BODY_ARRAY = new byte[0]; @@ -49,7 +47,7 @@ public class Ping implements Serializable { } public static Command create(){ - Command command = new Command(ID.getAndIncrement()); + Command command = new Command(); command.setType(CommandType.PING); command.setBody(EMPTY_BODY_ARRAY); return command; diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesRequestCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesRequestCommand.java index 1a2e6e4dd1..088bdd674e 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesRequestCommand.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesRequestCommand.java @@ -29,8 +29,6 @@ import java.util.concurrent.atomic.AtomicLong; */ public class GetLogBytesRequestCommand implements Serializable { - private static final AtomicLong REQUEST = new AtomicLong(1); - private String path; public GetLogBytesRequestCommand() { @@ -53,7 +51,7 @@ public class GetLogBytesRequestCommand implements Serializable { * @return */ public Command convert2Command(){ - Command command = new Command(REQUEST.getAndIncrement()); + Command command = new Command(); command.setType(CommandType.GET_LOG_BYTES_REQUEST); byte[] body = FastJsonSerializer.serialize(this); command.setBody(body); diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogRequestCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogRequestCommand.java index 49d19aa1f2..339c1a7f95 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogRequestCommand.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogRequestCommand.java @@ -29,8 +29,6 @@ import java.util.concurrent.atomic.AtomicLong; */ public class RollViewLogRequestCommand implements Serializable { - private static final AtomicLong REQUEST = new AtomicLong(1); - private String path; private int skipLineNum; @@ -71,7 +69,7 @@ public class RollViewLogRequestCommand implements Serializable { } public Command convert2Command(){ - Command command = new Command(REQUEST.getAndIncrement()); + Command command = new Command(); command.setType(CommandType.ROLL_VIEW_LOG_REQUEST); byte[] body = FastJsonSerializer.serialize(this); command.setBody(body); diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogRequestCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogRequestCommand.java index 9ba9cd3c23..69f6009256 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogRequestCommand.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogRequestCommand.java @@ -29,8 +29,6 @@ import java.util.concurrent.atomic.AtomicLong; */ public class ViewLogRequestCommand implements Serializable { - private static final AtomicLong REQUEST = new AtomicLong(1); - private String path; public ViewLogRequestCommand() { @@ -49,7 +47,7 @@ public class ViewLogRequestCommand implements Serializable { } public Command convert2Command(){ - Command command = new Command(REQUEST.getAndIncrement()); + Command command = new Command(); command.setType(CommandType.VIEW_WHOLE_LOG_REQUEST); byte[] body = FastJsonSerializer.serialize(this); command.setBody(body); diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingTimeoutException.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingTimeoutException.java new file mode 100644 index 0000000000..c0d986b063 --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingTimeoutException.java @@ -0,0 +1,20 @@ +package org.apache.dolphinscheduler.remote.exceptions; + +/** + * @Author: Tboy + */ +public class RemotingTimeoutException extends RemotingException{ + + public RemotingTimeoutException(String message) { + super(message); + } + + + public RemotingTimeoutException(String address, long timeoutMillis) { + this(address, timeoutMillis, null); + } + + public RemotingTimeoutException(String address, long timeoutMillis, Throwable cause) { + super(String.format("wait response on the channel %s timeout %s", address, timeoutMillis), cause); + } +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/ResponseFuture.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/ResponseFuture.java index 036f9906b1..ba2c7e36b8 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/ResponseFuture.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/ResponseFuture.java @@ -2,28 +2,100 @@ package org.apache.dolphinscheduler.remote.future; import org.apache.dolphinscheduler.remote.command.Command; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; /** * @Author: Tboy */ public class ResponseFuture { - private final int opaque; + private final static ConcurrentHashMap FUTURE_TABLE = new ConcurrentHashMap<>(256); + + private final long opaque; private final long timeoutMillis; private final InvokeCallback invokeCallback; + private final CountDownLatch latch = new CountDownLatch(1); + private final long beginTimestamp = System.currentTimeMillis(); - private final CountDownLatch latch = new CountDownLatch(1); + private volatile Command responseCommand; + + private volatile boolean sendOk = true; - public ResponseFuture(int opaque, long timeoutMillis, InvokeCallback invokeCallback) { + private volatile Throwable cause; + + + public ResponseFuture(long opaque, long timeoutMillis, InvokeCallback invokeCallback) { this.opaque = opaque; this.timeoutMillis = timeoutMillis; this.invokeCallback = invokeCallback; + FUTURE_TABLE.put(opaque, this); } + public Command waitResponse() throws InterruptedException { + this.latch.await(timeoutMillis, TimeUnit.MILLISECONDS); + return this.responseCommand; + } + public void putResponse(final Command responseCommand) { + this.responseCommand = responseCommand; + this.latch.countDown(); + FUTURE_TABLE.remove(opaque); + } + + public static ResponseFuture getFuture(long opaque){ + return FUTURE_TABLE.get(opaque); + } + + public boolean isTimeout() { + long diff = System.currentTimeMillis() - this.beginTimestamp; + return diff > this.timeoutMillis; + } + + public void executeInvokeCallback() { + if (invokeCallback != null) { + invokeCallback.operationComplete(this); + } + } + + public boolean isSendOK() { + return sendOk; + } + + public void setSendOk(boolean sendOk) { + this.sendOk = sendOk; + } + + public void setCause(Throwable cause) { + this.cause = cause; + } + + public Throwable getCause() { + return cause; + } + + public long getOpaque() { + return opaque; + } + + public long getTimeoutMillis() { + return timeoutMillis; + } + + public long getBeginTimestamp() { + return beginTimestamp; + } + + public Command getResponseCommand() { + return responseCommand; + } + + public InvokeCallback getInvokeCallback() { + return invokeCallback; + } } diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java index 115f6e4953..572957f9ab 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java @@ -19,17 +19,11 @@ package org.apache.dolphinscheduler.remote.handler; import io.netty.channel.*; import org.apache.dolphinscheduler.remote.NettyRemotingClient; import org.apache.dolphinscheduler.remote.command.Command; -import org.apache.dolphinscheduler.remote.command.CommandType; -import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; +import org.apache.dolphinscheduler.remote.future.ResponseFuture; import org.apache.dolphinscheduler.remote.utils.ChannelUtils; -import org.apache.dolphinscheduler.remote.utils.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.RejectedExecutionException; - /** * netty client request handler */ @@ -40,8 +34,6 @@ public class NettyClientHandler extends ChannelInboundHandlerAdapter { private final NettyRemotingClient nettyRemotingClient; - private final ConcurrentHashMap> processors = new ConcurrentHashMap(); - public NettyClientHandler(NettyRemotingClient nettyRemotingClient){ this.nettyRemotingClient = nettyRemotingClient; } @@ -54,42 +46,16 @@ public class NettyClientHandler extends ChannelInboundHandlerAdapter { @Override public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { - processReceived(ctx.channel(), (Command)msg); + processReceived((Command)msg); } - public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor) { - this.registerProcessor(commandType, processor, nettyRemotingClient.getDefaultExecutor()); - } - - public void registerProcessor(final CommandType commandType, final NettyRequestProcessor processor, final ExecutorService executor) { - ExecutorService executorRef = executor; - if(executorRef == null){ - executorRef = nettyRemotingClient.getDefaultExecutor(); - } - this.processors.putIfAbsent(commandType, new Pair(processor, executorRef)); - } - - private void processReceived(final Channel channel, final Command msg) { - final CommandType commandType = msg.getType(); - final Pair pair = processors.get(commandType); - if (pair != null) { - Runnable r = new Runnable() { - @Override - public void run() { - try { - pair.getLeft().process(channel, msg); - } catch (Throwable ex) { - logger.error("process msg {} error : {}", msg, ex); - } - } - }; - try { - pair.getRight().submit(r); - } catch (RejectedExecutionException e) { - logger.warn("thread pool is full, discard msg {} from {}", msg, ChannelUtils.getRemoteAddress(channel)); - } - } else { - logger.warn("commandType {} not support", commandType); + private void processReceived(final Command responseCommand) { + ResponseFuture future = ResponseFuture.getFuture(responseCommand.getOpaque()); + if(future != null){ + future.putResponse(responseCommand); + future.executeInvokeCallback(); + } else{ + logger.warn("receive response {}, but not matched any request ", responseCommand); } } diff --git a/dolphinscheduler-remote/src/test/java/org/apache/dolphinscheduler/remote/NettyRemotingClientTest.java b/dolphinscheduler-remote/src/test/java/org/apache/dolphinscheduler/remote/NettyRemotingClientTest.java index 19fd564bd5..15c556a2ca 100644 --- a/dolphinscheduler-remote/src/test/java/org/apache/dolphinscheduler/remote/NettyRemotingClientTest.java +++ b/dolphinscheduler-remote/src/test/java/org/apache/dolphinscheduler/remote/NettyRemotingClientTest.java @@ -48,24 +48,14 @@ public class NettyRemotingClientTest { }); server.start(); // - CountDownLatch latch = new CountDownLatch(1); - AtomicLong opaque = new AtomicLong(1); final NettyClientConfig clientConfig = new NettyClientConfig(); NettyRemotingClient client = new NettyRemotingClient(clientConfig); - client.registerProcessor(CommandType.PONG, new NettyRequestProcessor() { - @Override - public void process(Channel channel, Command command) { - opaque.set(command.getOpaque()); - latch.countDown(); - } - }); Command commandPing = Ping.create(); try { - client.send(new Address("127.0.0.1", serverConfig.getListenPort()), commandPing); - latch.await(); + Command response = client.sendSync(new Address("127.0.0.1", serverConfig.getListenPort()), commandPing, 2000); + Assert.assertEquals(commandPing.getOpaque(), response.getOpaque()); } catch (Exception e) { e.printStackTrace(); } - Assert.assertEquals(opaque.get(), commandPing.getOpaque()); } } diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java index 4faff335fe..3ca2d29782 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java @@ -16,13 +16,10 @@ */ package org.apache.dolphinscheduler.service.log; -import io.netty.channel.Channel; import org.apache.dolphinscheduler.remote.NettyRemotingClient; import org.apache.dolphinscheduler.remote.command.Command; -import org.apache.dolphinscheduler.remote.command.CommandType; import org.apache.dolphinscheduler.remote.command.log.*; import org.apache.dolphinscheduler.remote.config.NettyClientConfig; -import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; import org.apache.dolphinscheduler.remote.utils.Address; import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer; import org.slf4j.Logger; @@ -32,7 +29,7 @@ import org.slf4j.LoggerFactory; /** * log client */ -public class LogClientService implements NettyRequestProcessor { +public class LogClientService { private static final Logger logger = LoggerFactory.getLogger(LogClientService.class); @@ -52,9 +49,6 @@ public class LogClientService implements NettyRequestProcessor { this.clientConfig = new NettyClientConfig(); this.clientConfig.setWorkerThreads(4); this.client = new NettyRemotingClient(clientConfig); - this.client.registerProcessor(CommandType.ROLL_VIEW_LOG_RESPONSE,this); - this.client.registerProcessor(CommandType.VIEW_WHOLE_LOG_RESPONSE, this); - this.client.registerProcessor(CommandType.GET_LOG_BYTES_RESPONSE, this); } /** @@ -81,9 +75,12 @@ public class LogClientService implements NettyRequestProcessor { final Address address = new Address(host, port); try { Command command = request.convert2Command(); - this.client.send(address, command); - LogPromise promise = new LogPromise(command.getOpaque(), logRequestTimeout); - result = ((String)promise.getResult()); + Command response = this.client.sendSync(address, command, logRequestTimeout); + if(response != null){ + RollViewLogResponseCommand rollReviewLog = FastJsonSerializer.deserialize( + command.getBody(), RollViewLogResponseCommand.class); + return rollReviewLog.getMsg(); + } } catch (Exception e) { logger.error("roll view log error", e); } finally { @@ -106,9 +103,12 @@ public class LogClientService implements NettyRequestProcessor { final Address address = new Address(host, port); try { Command command = request.convert2Command(); - this.client.send(address, command); - LogPromise promise = new LogPromise(command.getOpaque(), logRequestTimeout); - result = ((String)promise.getResult()); + Command response = this.client.sendSync(address, command, logRequestTimeout); + if(response != null){ + ViewLogResponseCommand viewLog = FastJsonSerializer.deserialize( + response.getBody(), ViewLogResponseCommand.class); + return viewLog.getMsg(); + } } catch (Exception e) { logger.error("view log error", e); } finally { @@ -131,9 +131,12 @@ public class LogClientService implements NettyRequestProcessor { final Address address = new Address(host, port); try { Command command = request.convert2Command(); - this.client.send(address, command); - LogPromise promise = new LogPromise(command.getOpaque(), logRequestTimeout); - result = (byte[])promise.getResult(); + Command response = this.client.sendSync(address, command, logRequestTimeout); + if(response != null){ + GetLogBytesResponseCommand getLog = FastJsonSerializer.deserialize( + response.getBody(), GetLogBytesResponseCommand.class); + return getLog.getData(); + } } catch (Exception e) { logger.error("get log size error", e); } finally { @@ -141,28 +144,4 @@ public class LogClientService implements NettyRequestProcessor { } return result; } - - @Override - public void process(Channel channel, Command command) { - logger.info("received log response : {}", command); - switch (command.getType()){ - case ROLL_VIEW_LOG_RESPONSE: - RollViewLogResponseCommand rollReviewLog = FastJsonSerializer.deserialize( - command.getBody(), RollViewLogResponseCommand.class); - LogPromise.notify(command.getOpaque(), rollReviewLog.getMsg()); - break; - case VIEW_WHOLE_LOG_RESPONSE: - ViewLogResponseCommand viewLog = FastJsonSerializer.deserialize( - command.getBody(), ViewLogResponseCommand.class); - LogPromise.notify(command.getOpaque(), viewLog.getMsg()); - break; - case GET_LOG_BYTES_RESPONSE: - GetLogBytesResponseCommand getLog = FastJsonSerializer.deserialize( - command.getBody(), GetLogBytesResponseCommand.class); - LogPromise.notify(command.getOpaque(), getLog.getData()); - break; - default: - throw new UnsupportedOperationException(String.format("command type : %s is not supported ", command.getType())); - } - } } \ No newline at end of file From 38a7ad11dae50eea8e80ddc637e28f6e6849140f Mon Sep 17 00:00:00 2001 From: qiaozhanwei Date: Sat, 15 Feb 2020 15:25:05 +0800 Subject: [PATCH 019/221] =?UTF-8?q?1=EF=BC=8Cquartz.properties=20add=20con?= =?UTF-8?q?f=20category=202=EF=BC=8Cdolphinscheduler-daemon.sh=20modify?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../src/main/assembly/dolphinscheduler-binary.xml | 8 ++++++++ script/dolphinscheduler-daemon.sh | 2 +- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/dolphinscheduler-dist/src/main/assembly/dolphinscheduler-binary.xml b/dolphinscheduler-dist/src/main/assembly/dolphinscheduler-binary.xml index b4326c6795..498e7a7544 100644 --- a/dolphinscheduler-dist/src/main/assembly/dolphinscheduler-binary.xml +++ b/dolphinscheduler-dist/src/main/assembly/dolphinscheduler-binary.xml @@ -120,6 +120,14 @@ . + + ${basedir}/../dolphinscheduler-server/target/dolphinscheduler-service-${project.version} + + **/*.* + + . + + ${basedir}/../dolphinscheduler-api/target/dolphinscheduler-api-${project.version} diff --git a/script/dolphinscheduler-daemon.sh b/script/dolphinscheduler-daemon.sh index d4db103fe1..d942bca7d2 100644 --- a/script/dolphinscheduler-daemon.sh +++ b/script/dolphinscheduler-daemon.sh @@ -69,7 +69,7 @@ elif [ "$command" = "alert-server" ]; then LOG_FILE="-Dserver=alert-server" CLASS=org.apache.dolphinscheduler.alert.AlertServer elif [ "$command" = "logger-server" ]; then - CLASS=org.apache.dolphinscheduler.server.rpc.LoggerServer + CLASS=org.apache.dolphinscheduler.server.log.LoggerServer elif [ "$command" = "combined-server" ]; then LOG_FILE="-Dlogging.config=classpath:combined_logback.xml -Dspring.profiles.active=api -Dserver.is-combined-server=true" CLASS=org.apache.dolphinscheduler.api.CombinedApplicationServer From 9dd6979d750ca8360b2c45ab93c3ef285ecc3213 Mon Sep 17 00:00:00 2001 From: qiaozhanwei Date: Sat, 15 Feb 2020 15:37:13 +0800 Subject: [PATCH 020/221] dolphinscheduler-binary.xml modify --- .../src/main/assembly/dolphinscheduler-binary.xml | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/dolphinscheduler-dist/src/main/assembly/dolphinscheduler-binary.xml b/dolphinscheduler-dist/src/main/assembly/dolphinscheduler-binary.xml index 498e7a7544..28bbb361cd 100644 --- a/dolphinscheduler-dist/src/main/assembly/dolphinscheduler-binary.xml +++ b/dolphinscheduler-dist/src/main/assembly/dolphinscheduler-binary.xml @@ -112,16 +112,23 @@ + + - ${basedir}/../dolphinscheduler-server/target/dolphinscheduler-server-${project.version} + ${basedir}/../dolphinscheduler-service/src/main/resources - **/*.* + **/*.properties + **/*.xml + **/*.json + **/*.yml - . + conf + + - ${basedir}/../dolphinscheduler-server/target/dolphinscheduler-service-${project.version} + ${basedir}/../dolphinscheduler-server/target/dolphinscheduler-server-${project.version} **/*.* From ac932ba2cfbeb1c962b45dedcb308a256c4dafea Mon Sep 17 00:00:00 2001 From: Tboy Date: Sun, 16 Feb 2020 11:53:29 +0800 Subject: [PATCH 021/221] add sendAsync method (#1962) * add sendAsync method * add sendAsync method --- .../remote/NettyRemotingClient.java | 135 ++++++++++-------- .../exceptions/RemotingTimeoutException.java | 21 ++- .../RemotingTooMuchRequestException.java | 24 ++++ .../remote/future/InvokeCallback.java | 18 ++- .../remote/future/ReleaseSemaphore.java | 41 ++++++ .../remote/future/ResponseFuture.java | 39 ++++- .../remote/handler/NettyClientHandler.java | 21 ++- .../utils/CallerThreadExecutePolicy.java | 38 +++++ .../remote/utils/NamedThreadFactory.java | 47 ++++++ .../remote/NettyRemotingClientTest.java | 39 ++++- 10 files changed, 349 insertions(+), 74 deletions(-) create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingTooMuchRequestException.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/ReleaseSemaphore.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/CallerThreadExecutePolicy.java create mode 100644 dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/NamedThreadFactory.java diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java index 3dcb8c0825..e07cfd6404 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java @@ -28,20 +28,19 @@ import org.apache.dolphinscheduler.remote.command.Command; 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.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.rmi.RemoteException; -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; @@ -56,18 +55,20 @@ public class NettyRemotingClient { private final NettyEncoder encoder = new NettyEncoder(); - private final ConcurrentHashMap channels = new ConcurrentHashMap(); - - private final ExecutorService defaultExecutor = Executors.newFixedThreadPool(Constants.CPUS); + private final ConcurrentHashMap channels = new ConcurrentHashMap(128); private final AtomicBoolean isStarted = new AtomicBoolean(false); private final NioEventLoopGroup workerGroup; - private final NettyClientHandler clientHandler = new NettyClientHandler(this); - private final NettyClientConfig clientConfig; + private final Semaphore asyncSemaphore = new Semaphore(200, true); + + private final ExecutorService callbackExecutor; + + private final NettyClientHandler clientHandler; + public NettyRemotingClient(final NettyClientConfig clientConfig){ this.clientConfig = clientConfig; this.workerGroup = new NioEventLoopGroup(clientConfig.getWorkerThreads(), new ThreadFactory() { @@ -78,6 +79,10 @@ 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(); } @@ -103,65 +108,79 @@ public class NettyRemotingClient { isStarted.compareAndSet(false, true); } - //TODO - public void send(final Address address, final Command command, final InvokeCallback invokeCallback) throws RemotingException { + 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"); } - try { - channel.writeAndFlush(command).addListener(new ChannelFutureListener(){ - - @Override - public void operationComplete(ChannelFuture future) throws Exception { - if(future.isSuccess()){ - logger.info("sent command {} to {}", command, address); - } else{ - logger.error("send command {} to {} failed, error {}", command, address, future.cause()); + final long opaque = command.getOpaque(); + boolean acquired = this.asyncSemaphore.tryAcquire(timeoutMillis, TimeUnit.MILLISECONDS); + if(acquired){ + final ReleaseSemaphore releaseSemaphore = new ReleaseSemaphore(this.asyncSemaphore); + 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 (Exception ex) { - String msg = String.format("send command %s to address %s encounter error", command, address); - throw new RemotingException(msg, ex); + }); + } 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); } } - public Command sendSync(final Address address, final Command command, final long timeoutMillis) 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(String.format("connect to : %s fail", address)); } final long opaque = command.getOpaque(); - try { - final ResponseFuture responseFuture = new ResponseFuture(opaque, timeoutMillis, null); - channel.writeAndFlush(command).addListener(new ChannelFutureListener() { - @Override - public void operationComplete(ChannelFuture channelFuture) throws Exception { - if(channelFuture.isSuccess()){ - responseFuture.setSendOk(true); - return; - } else{ - responseFuture.setSendOk(false); - responseFuture.setCause(channelFuture.cause()); - responseFuture.putResponse(null); - logger.error("send command {} to address {} failed", command, address); - } - } - }); - Command result = responseFuture.waitResponse(); - if(result == null){ - if(responseFuture.isSendOK()){ - throw new RemotingTimeoutException(address.toString(), timeoutMillis, responseFuture.getCause()); - } else{ - throw new RemoteException(address.toString(), responseFuture.getCause()); + 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()){ + responseFuture.setSendOk(true); + return; + } else { + responseFuture.setSendOk(false); } + responseFuture.setCause(future.cause()); + responseFuture.putResponse(null); + logger.error("send command {} to address {} failed", command, address); + } + }); + 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; - } catch (Exception ex) { - String msg = String.format("send command %s to address %s error", command, address); - throw new RemotingException(msg, ex); } + return result; } public Channel getChannel(Address address) { @@ -192,10 +211,6 @@ public class NettyRemotingClient { return null; } - public ExecutorService getDefaultExecutor() { - return defaultExecutor; - } - public void close() { if(isStarted.compareAndSet(true, false)){ try { @@ -203,8 +218,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); diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingTimeoutException.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingTimeoutException.java index c0d986b063..aaf9170781 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingTimeoutException.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingTimeoutException.java @@ -1,8 +1,23 @@ +/* + * 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.exceptions; -/** - * @Author: Tboy - */ + public class RemotingTimeoutException extends RemotingException{ public RemotingTimeoutException(String message) { diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingTooMuchRequestException.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingTooMuchRequestException.java new file mode 100644 index 0000000000..5ee11a04a7 --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingTooMuchRequestException.java @@ -0,0 +1,24 @@ +/* + * 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.exceptions; + +public class RemotingTooMuchRequestException extends RemotingException{ + + public RemotingTooMuchRequestException(String message) { + super(message); + } +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/InvokeCallback.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/InvokeCallback.java index 6ad6a7cbee..7cf875b002 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/InvokeCallback.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/InvokeCallback.java @@ -1,7 +1,23 @@ +/* + * 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.future; /** - * @Author: Tboy + * invoke callback */ public interface InvokeCallback { diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/ReleaseSemaphore.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/ReleaseSemaphore.java new file mode 100644 index 0000000000..95a04b1f1a --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/ReleaseSemaphore.java @@ -0,0 +1,41 @@ +/* + * 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.future; + +import java.util.concurrent.Semaphore; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * release semaphore + */ +public class ReleaseSemaphore { + + private final Semaphore semaphore; + + private final AtomicBoolean released; + + public ReleaseSemaphore(Semaphore semaphore){ + this.semaphore = semaphore; + this.released = new AtomicBoolean(false); + } + + public void release(){ + if(this.released.compareAndSet(false, true)){ + this.semaphore.release(); + } + } +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/ResponseFuture.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/ResponseFuture.java index ba2c7e36b8..a9bdb39adf 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/ResponseFuture.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/ResponseFuture.java @@ -1,13 +1,28 @@ +/* + * 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.future; import org.apache.dolphinscheduler.remote.command.Command; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; +import java.util.concurrent.*; /** - * @Author: Tboy + * response future */ public class ResponseFuture { @@ -19,6 +34,8 @@ public class ResponseFuture { private final InvokeCallback invokeCallback; + private final ReleaseSemaphore releaseSemaphore; + private final CountDownLatch latch = new CountDownLatch(1); private final long beginTimestamp = System.currentTimeMillis(); @@ -29,11 +46,11 @@ public class ResponseFuture { private volatile Throwable cause; - - public ResponseFuture(long opaque, long timeoutMillis, InvokeCallback invokeCallback) { + public ResponseFuture(long opaque, long timeoutMillis, InvokeCallback invokeCallback, ReleaseSemaphore releaseSemaphore) { this.opaque = opaque; this.timeoutMillis = timeoutMillis; this.invokeCallback = invokeCallback; + this.releaseSemaphore = releaseSemaphore; FUTURE_TABLE.put(opaque, this); } @@ -95,7 +112,17 @@ public class ResponseFuture { return responseCommand; } + public void setResponseCommand(Command responseCommand) { + this.responseCommand = responseCommand; + } + public InvokeCallback getInvokeCallback() { return invokeCallback; } + + public void release() { + if(this.releaseSemaphore != null){ + this.releaseSemaphore.release(); + } + } } diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java index 572957f9ab..97f6632fb3 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java @@ -24,6 +24,8 @@ import org.apache.dolphinscheduler.remote.utils.ChannelUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.concurrent.ExecutorService; + /** * netty client request handler */ @@ -34,8 +36,11 @@ public class NettyClientHandler extends ChannelInboundHandlerAdapter { private final NettyRemotingClient nettyRemotingClient; - public NettyClientHandler(NettyRemotingClient nettyRemotingClient){ + private final ExecutorService callbackExecutor; + + public NettyClientHandler(NettyRemotingClient nettyRemotingClient, ExecutorService callbackExecutor){ this.nettyRemotingClient = nettyRemotingClient; + this.callbackExecutor = callbackExecutor; } @Override @@ -52,8 +57,18 @@ public class NettyClientHandler extends ChannelInboundHandlerAdapter { private void processReceived(final Command responseCommand) { ResponseFuture future = ResponseFuture.getFuture(responseCommand.getOpaque()); if(future != null){ - future.putResponse(responseCommand); - future.executeInvokeCallback(); + future.setResponseCommand(responseCommand); + future.release(); + if(future.getInvokeCallback() != null){ + this.callbackExecutor.submit(new Runnable() { + @Override + public void run() { + future.executeInvokeCallback(); + } + }); + } else{ + future.putResponse(responseCommand); + } } else{ logger.warn("receive response {}, but not matched any request ", responseCommand); } diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/CallerThreadExecutePolicy.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/CallerThreadExecutePolicy.java new file mode 100644 index 0000000000..048ea86acb --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/CallerThreadExecutePolicy.java @@ -0,0 +1,38 @@ +/* + * 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.utils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.RejectedExecutionHandler; +import java.util.concurrent.ThreadPoolExecutor; + +/** + * caller thread execute + */ +public class CallerThreadExecutePolicy implements RejectedExecutionHandler { + + private final Logger logger = LoggerFactory.getLogger(CallerThreadExecutePolicy.class); + + @Override + public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) { + logger.warn("queue is full, trigger caller thread execute"); + r.run(); + } +} diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/NamedThreadFactory.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/NamedThreadFactory.java new file mode 100644 index 0000000000..bef64c7dc1 --- /dev/null +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/NamedThreadFactory.java @@ -0,0 +1,47 @@ +/* + * 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.utils; + +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.atomic.AtomicInteger; + +public class NamedThreadFactory implements ThreadFactory { + + private final AtomicInteger increment = new AtomicInteger(1); + + private final String name; + + private final int count; + + public NamedThreadFactory(String name){ + this(name, 0); + } + + public NamedThreadFactory(String name, int count){ + this.name = name; + this.count = count; + } + + @Override + public Thread newThread(Runnable r) { + final String threadName = count > 0 ? String.format(name + "_%d_%d", count, increment.getAndIncrement()) + : String.format(name + "_%d", increment.getAndIncrement()); + Thread t = new Thread(r, threadName); + t.setDaemon(true); + return t; + } +} diff --git a/dolphinscheduler-remote/src/test/java/org/apache/dolphinscheduler/remote/NettyRemotingClientTest.java b/dolphinscheduler-remote/src/test/java/org/apache/dolphinscheduler/remote/NettyRemotingClientTest.java index 15c556a2ca..732a995eb3 100644 --- a/dolphinscheduler-remote/src/test/java/org/apache/dolphinscheduler/remote/NettyRemotingClientTest.java +++ b/dolphinscheduler-remote/src/test/java/org/apache/dolphinscheduler/remote/NettyRemotingClientTest.java @@ -24,6 +24,8 @@ import org.apache.dolphinscheduler.remote.command.Ping; import org.apache.dolphinscheduler.remote.command.Pong; import org.apache.dolphinscheduler.remote.config.NettyClientConfig; import org.apache.dolphinscheduler.remote.config.NettyServerConfig; +import org.apache.dolphinscheduler.remote.future.InvokeCallback; +import org.apache.dolphinscheduler.remote.future.ResponseFuture; import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor; import org.apache.dolphinscheduler.remote.utils.Address; import org.junit.Assert; @@ -36,7 +38,7 @@ public class NettyRemotingClientTest { @Test - public void testSend(){ + public void testSendSync(){ NettyServerConfig serverConfig = new NettyServerConfig(); NettyRemotingServer server = new NettyRemotingServer(serverConfig); @@ -46,6 +48,8 @@ public class NettyRemotingClientTest { channel.writeAndFlush(Pong.create(command.getOpaque())); } }); + + server.start(); // final NettyClientConfig clientConfig = new NettyClientConfig(); @@ -58,4 +62,37 @@ public class NettyRemotingClientTest { e.printStackTrace(); } } + + @Test + public void testSendAsync(){ + NettyServerConfig serverConfig = new NettyServerConfig(); + + NettyRemotingServer server = new NettyRemotingServer(serverConfig); + server.registerProcessor(CommandType.PING, new NettyRequestProcessor() { + @Override + public void process(Channel channel, Command command) { + channel.writeAndFlush(Pong.create(command.getOpaque())); + } + }); + server.start(); + // + final NettyClientConfig clientConfig = new NettyClientConfig(); + NettyRemotingClient client = new NettyRemotingClient(clientConfig); + CountDownLatch latch = new CountDownLatch(1); + Command commandPing = Ping.create(); + try { + final AtomicLong opaque = new AtomicLong(0); + client.sendAsync(new Address("127.0.0.1", serverConfig.getListenPort()), commandPing, 2000, new InvokeCallback() { + @Override + public void operationComplete(ResponseFuture responseFuture) { + opaque.set(responseFuture.getOpaque()); + latch.countDown(); + } + }); + latch.await(); + Assert.assertEquals(commandPing.getOpaque(), opaque.get()); + } catch (Exception e) { + e.printStackTrace(); + } + } } From 99b9c10d78bef033a658a2389640ba471ae8da29 Mon Sep 17 00:00:00 2001 From: qiaozhanwei Date: Sun, 16 Feb 2020 18:12:44 +0800 Subject: [PATCH 022/221] =?UTF-8?q?1=EF=BC=8Cadd=20sendAsync=20method=202?= =?UTF-8?q?=EF=BC=8Crefactor=20LoggerClient?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../remote/NettyRemotingClient.java | 98 ++++++++++++++++++- .../remote/NettyRemotingServer.java | 2 +- .../command/ExecuteTaskRequestCommand.java | 2 +- .../command/ExecuteTaskResponseCommand.java | 2 +- .../dolphinscheduler/remote/command/Ping.java | 19 +++- .../log/GetLogBytesRequestCommand.java | 6 +- .../log/RollViewLogRequestCommand.java | 14 +++ .../command/log/ViewLogRequestCommand.java | 8 ++ .../exceptions/RemotingTimeoutException.java | 3 + .../RemotingTooMuchRequestException.java | 3 + .../remote/future/InvokeCallback.java | 5 + .../remote/future/ResponseFuture.java | 36 +++++++ .../remote/handler/NettyClientHandler.java | 37 +++++++ .../remote/utils/NamedThreadFactory.java | 14 +++ .../remote/NettyRemotingClientTest.java | 5 +- .../service/log/LogPromise.java | 28 ++++++ 16 files changed, 273 insertions(+), 9 deletions(-) diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java index e07cfd6404..357fd6d19d 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingClient.java @@ -51,24 +51,55 @@ public class NettyRemotingClient { private final Logger logger = LoggerFactory.getLogger(NettyRemotingClient.class); + /** + * client bootstrap + */ private final Bootstrap bootstrap = new Bootstrap(); + /** + * encoder + */ private final NettyEncoder encoder = new NettyEncoder(); + /** + * channels + */ private final ConcurrentHashMap channels = new ConcurrentHashMap(128); + /** + * started flag + */ private final AtomicBoolean isStarted = new AtomicBoolean(false); + /** + * worker group + */ private final NioEventLoopGroup workerGroup; + /** + * client config + */ private final NettyClientConfig clientConfig; + /** + * saync semaphore + */ private final Semaphore asyncSemaphore = new Semaphore(200, true); + /** + * callback thread executor + */ private final ExecutorService callbackExecutor; + /** + * client handler + */ private final NettyClientHandler clientHandler; + /** + * client init + * @param clientConfig client config + */ public NettyRemotingClient(final NettyClientConfig clientConfig){ this.clientConfig = clientConfig; this.workerGroup = new NioEventLoopGroup(clientConfig.getWorkerThreads(), new ThreadFactory() { @@ -80,12 +111,16 @@ public class NettyRemotingClient { } }); this.callbackExecutor = new ThreadPoolExecutor(5, 10, 1, TimeUnit.MINUTES, - new LinkedBlockingQueue<>(1000), new NamedThreadFactory("CallbackExecutor", 10), new CallerThreadExecutePolicy()); + new LinkedBlockingQueue<>(1000), new NamedThreadFactory("CallbackExecutor", 10), + new CallerThreadExecutePolicy()); this.clientHandler = new NettyClientHandler(this, callbackExecutor); this.start(); } + /** + * start + */ private void start(){ this.bootstrap @@ -108,16 +143,40 @@ public class NettyRemotingClient { isStarted.compareAndSet(false, true); } - public void sendAsync(final Address address, final Command command, final long timeoutMillis, final InvokeCallback invokeCallback) throws InterruptedException, RemotingException { + /** + * async send + * @param address address + * @param command command + * @param timeoutMillis timeoutMillis + * @param invokeCallback callback function + * @throws InterruptedException + * @throws RemotingException + */ + 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); - final ResponseFuture responseFuture = new ResponseFuture(opaque, timeoutMillis, invokeCallback, releaseSemaphore); + + /** + * response future + */ + final ResponseFuture responseFuture = new ResponseFuture(opaque, + timeoutMillis, + invokeCallback, + releaseSemaphore); try { channel.writeAndFlush(command).addListener(new ChannelFutureListener(){ @@ -151,6 +210,15 @@ public class NettyRemotingClient { } } + /** + * sync send + * @param address address + * @param command command + * @param timeoutMillis timeoutMillis + * @return command + * @throws InterruptedException + * @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) { @@ -172,6 +240,9 @@ public class NettyRemotingClient { logger.error("send command {} to address {} failed", command, address); } }); + /** + * sync wait for result + */ Command result = responseFuture.waitResponse(); if(result == null){ if(responseFuture.isSendOK()){ @@ -183,6 +254,11 @@ public class NettyRemotingClient { return result; } + /** + * get channel + * @param address + * @return + */ public Channel getChannel(Address address) { Channel channel = channels.get(address); if(channel != null && channel.isActive()){ @@ -191,6 +267,12 @@ public class NettyRemotingClient { return createChannel(address, true); } + /** + * create channel + * @param address address + * @param isSync sync flag + * @return channel + */ public Channel createChannel(Address address, boolean isSync) { ChannelFuture future; try { @@ -211,6 +293,9 @@ public class NettyRemotingClient { return null; } + /** + * close + */ public void close() { if(isStarted.compareAndSet(true, false)){ try { @@ -228,6 +313,9 @@ public class NettyRemotingClient { } } + /** + * close channels + */ private void closeChannels(){ for (Channel channel : this.channels.values()) { channel.close(); @@ -235,6 +323,10 @@ public class NettyRemotingClient { this.channels.clear(); } + /** + * close channel + * @param address address + */ public void closeChannel(Address address){ Channel channel = this.channels.remove(address); if(channel != null){ diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingServer.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingServer.java index c69bf09540..29b2317633 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingServer.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingServer.java @@ -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(); diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskRequestCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskRequestCommand.java index b3801aceb0..beec055403 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskRequestCommand.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskRequestCommand.java @@ -1 +1 @@ -/* * 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; public class ExecuteTaskRequestCommand implements Serializable { private String taskId; private String attemptId; private String applicationName; private String groupName; private String taskName; private int connectorPort; private String description; private String className; private String methodName; private String params; private List shardItems; public List getShardItems() { return shardItems; } public void setShardItems(List 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; } 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 +/* * 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 shardItems; public List getShardItems() { return shardItems; } public void setShardItems(List 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 diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskResponseCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskResponseCommand.java index aeb5f7d858..7e35fa6e75 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskResponseCommand.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskResponseCommand.java @@ -1 +1 @@ -/* * 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.concurrent.atomic.AtomicLong; public class ExecuteTaskResponseCommand implements Serializable { private String taskId; private String attemptId; private Object result; private long receivedTime; private int executeCount; private long executeTime; public String getAttemptId() { return attemptId; } public void setAttemptId(String attemptId) { this.attemptId = attemptId; } public String getTaskId() { return taskId; } public void setTaskId(String taskId) { this.taskId = taskId; } public Object getResult() { return result; } public void setResult(Object result) { this.result = result; } public long getReceivedTime() { return receivedTime; } public void setReceivedTime(long receivedTime) { this.receivedTime = receivedTime; } public int getExecuteCount() { return executeCount; } public void setExecuteCount(int executeCount) { this.executeCount = executeCount; } public long getExecuteTime() { return executeTime; } public void setExecuteTime(long executeTime) { this.executeTime = executeTime; } public Command convert2Command(long opaque){ Command command = new Command(); command.setType(CommandType.EXECUTE_TASK_RESPONSE); 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.concurrent.atomic.AtomicLong; /** * execute task response command */ public class ExecuteTaskResponseCommand implements Serializable { /** * task id */ private String taskId; /** * attempt id */ private String attemptId; /** * return result */ private Object result; /** * received time */ private long receivedTime; /** * execute count */ private int executeCount; /** * execute time */ private long executeTime; public String getAttemptId() { return attemptId; } public void setAttemptId(String attemptId) { this.attemptId = attemptId; } public String getTaskId() { return taskId; } public void setTaskId(String taskId) { this.taskId = taskId; } public Object getResult() { return result; } public void setResult(Object result) { this.result = result; } public long getReceivedTime() { return receivedTime; } public void setReceivedTime(long receivedTime) { this.receivedTime = receivedTime; } public int getExecuteCount() { return executeCount; } public void setExecuteCount(int executeCount) { this.executeCount = executeCount; } public long getExecuteTime() { return executeTime; } public void setExecuteTime(long executeTime) { this.executeTime = executeTime; } public Command convert2Command(long opaque){ Command command = new Command(); command.setType(CommandType.EXECUTE_TASK_RESPONSE); byte[] body = FastJsonSerializer.serialize(this); command.setBody(body); return command; } } \ No newline at end of file diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Ping.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Ping.java index 38a8b14871..c50413e98a 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Ping.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Ping.java @@ -23,11 +23,19 @@ import io.netty.buffer.Unpooled; import java.io.Serializable; import java.util.concurrent.atomic.AtomicLong; - +/** + * ping machine + */ public class Ping implements Serializable { + /** + * ping body + */ protected static ByteBuf EMPTY_BODY = Unpooled.EMPTY_BUFFER; + /** + * request command body + */ private static byte[] EMPTY_BODY_ARRAY = new byte[0]; private static final ByteBuf PING_BUF; @@ -42,10 +50,19 @@ public class Ping implements Serializable { PING_BUF = Unpooled.unreleasableBuffer(ping).asReadOnly(); } + /** + * ping content + * @return result + */ public static ByteBuf pingContent(){ return PING_BUF.duplicate(); } + /** + * create ping command + * + * @return command + */ public static Command create(){ Command command = new Command(); command.setType(CommandType.PING); diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesRequestCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesRequestCommand.java index 088bdd674e..4cc32ed42a 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesRequestCommand.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesRequestCommand.java @@ -29,6 +29,9 @@ import java.util.concurrent.atomic.AtomicLong; */ public class GetLogBytesRequestCommand implements Serializable { + /** + * log path + */ private String path; public GetLogBytesRequestCommand() { @@ -47,8 +50,9 @@ public class GetLogBytesRequestCommand implements Serializable { } /** + * package request command * - * @return + * @return command */ public Command convert2Command(){ Command command = new Command(); diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogRequestCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogRequestCommand.java index 339c1a7f95..621d35a804 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogRequestCommand.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogRequestCommand.java @@ -29,10 +29,19 @@ import java.util.concurrent.atomic.AtomicLong; */ public class RollViewLogRequestCommand implements Serializable { + /** + * log path + */ private String path; + /** + * skip line number + */ private int skipLineNum; + /** + * query line number + */ private int limit; public RollViewLogRequestCommand() { @@ -68,6 +77,11 @@ public class RollViewLogRequestCommand implements Serializable { this.limit = limit; } + /** + * package request command + * + * @return command + */ public Command convert2Command(){ Command command = new Command(); command.setType(CommandType.ROLL_VIEW_LOG_REQUEST); diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogRequestCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogRequestCommand.java index 69f6009256..8835348ee3 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogRequestCommand.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogRequestCommand.java @@ -29,6 +29,9 @@ import java.util.concurrent.atomic.AtomicLong; */ public class ViewLogRequestCommand implements Serializable { + /** + * log path + */ private String path; public ViewLogRequestCommand() { @@ -46,6 +49,11 @@ public class ViewLogRequestCommand implements Serializable { this.path = path; } + /** + * package request command + * + * @return command + */ public Command convert2Command(){ Command command = new Command(); command.setType(CommandType.VIEW_WHOLE_LOG_REQUEST); diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingTimeoutException.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingTimeoutException.java index aaf9170781..3d91ba57f6 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingTimeoutException.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingTimeoutException.java @@ -18,6 +18,9 @@ package org.apache.dolphinscheduler.remote.exceptions; +/** + * timeout exception + */ public class RemotingTimeoutException extends RemotingException{ public RemotingTimeoutException(String message) { diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingTooMuchRequestException.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingTooMuchRequestException.java index 5ee11a04a7..82cc3f4dbf 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingTooMuchRequestException.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/exceptions/RemotingTooMuchRequestException.java @@ -16,6 +16,9 @@ */ package org.apache.dolphinscheduler.remote.exceptions; +/** + * too much request exception + */ public class RemotingTooMuchRequestException extends RemotingException{ public RemotingTooMuchRequestException(String message) { diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/InvokeCallback.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/InvokeCallback.java index 7cf875b002..84cdae867b 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/InvokeCallback.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/InvokeCallback.java @@ -21,6 +21,11 @@ package org.apache.dolphinscheduler.remote.future; */ public interface InvokeCallback { + /** + * operation + * + * @param responseFuture responseFuture + */ void operationComplete(final ResponseFuture responseFuture); } diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/ResponseFuture.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/ResponseFuture.java index a9bdb39adf..caff34236e 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/ResponseFuture.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/future/ResponseFuture.java @@ -28,18 +28,33 @@ public class ResponseFuture { private final static ConcurrentHashMap FUTURE_TABLE = new ConcurrentHashMap<>(256); + /** + * request unique identification + */ private final long opaque; + /** + * timeout + */ private final long timeoutMillis; + /** + * invokeCallback function + */ private final InvokeCallback invokeCallback; + /** + * releaseSemaphore + */ private final ReleaseSemaphore releaseSemaphore; private final CountDownLatch latch = new CountDownLatch(1); private final long beginTimestamp = System.currentTimeMillis(); + /** + * response command + */ private volatile Command responseCommand; private volatile boolean sendOk = true; @@ -54,11 +69,22 @@ public class ResponseFuture { FUTURE_TABLE.put(opaque, this); } + /** + * wait for response + * + * @return command + * @throws InterruptedException + */ public Command waitResponse() throws InterruptedException { this.latch.await(timeoutMillis, TimeUnit.MILLISECONDS); return this.responseCommand; } + /** + * put response + * + * @param responseCommand responseCommand + */ public void putResponse(final Command responseCommand) { this.responseCommand = responseCommand; this.latch.countDown(); @@ -69,11 +95,18 @@ public class ResponseFuture { return FUTURE_TABLE.get(opaque); } + /** + * whether timeout + * @return timeout + */ public boolean isTimeout() { long diff = System.currentTimeMillis() - this.beginTimestamp; return diff > this.timeoutMillis; } + /** + * execute invoke callback + */ public void executeInvokeCallback() { if (invokeCallback != null) { invokeCallback.operationComplete(this); @@ -120,6 +153,9 @@ public class ResponseFuture { return invokeCallback; } + /** + * release + */ public void release() { if(this.releaseSemaphore != null){ this.releaseSemaphore.release(); diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java index 97f6632fb3..d5d0d4df83 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java @@ -34,8 +34,14 @@ public class NettyClientHandler extends ChannelInboundHandlerAdapter { private final Logger logger = LoggerFactory.getLogger(NettyClientHandler.class); + /** + * netty client + */ private final NettyRemotingClient nettyRemotingClient; + /** + * callback thread executor + */ private final ExecutorService callbackExecutor; public NettyClientHandler(NettyRemotingClient nettyRemotingClient, ExecutorService callbackExecutor){ @@ -43,17 +49,36 @@ public class NettyClientHandler extends ChannelInboundHandlerAdapter { this.callbackExecutor = callbackExecutor; } + /** + * When the current channel is not active, + * the current channel has reached the end of its life cycle + * + * @param ctx channel handler context + * @throws Exception + */ @Override public void channelInactive(ChannelHandlerContext ctx) throws Exception { nettyRemotingClient.closeChannel(ChannelUtils.toAddress(ctx.channel())); ctx.channel().close(); } + /** + * The current channel reads data from the remote + * + * @param ctx channel handler context + * @param msg message + * @throws Exception + */ @Override public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { processReceived((Command)msg); } + /** + * process received logic + * + * @param responseCommand responseCommand + */ private void processReceived(final Command responseCommand) { ResponseFuture future = ResponseFuture.getFuture(responseCommand.getOpaque()); if(future != null){ @@ -74,6 +99,12 @@ public class NettyClientHandler extends ChannelInboundHandlerAdapter { } } + /** + * caught exception + * @param ctx channel handler context + * @param cause cause + * @throws Exception + */ @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { logger.error("exceptionCaught : {}", cause); @@ -81,6 +112,12 @@ public class NettyClientHandler extends ChannelInboundHandlerAdapter { ctx.channel().close(); } + /** + * channel write changed + * + * @param ctx channel handler context + * @throws Exception + */ @Override public void channelWritabilityChanged(ChannelHandlerContext ctx) throws Exception { Channel ch = ctx.channel(); diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/NamedThreadFactory.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/NamedThreadFactory.java index bef64c7dc1..2f0d05ebd4 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/NamedThreadFactory.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/NamedThreadFactory.java @@ -19,12 +19,21 @@ package org.apache.dolphinscheduler.remote.utils; import java.util.concurrent.ThreadFactory; import java.util.concurrent.atomic.AtomicInteger; +/** + * thread factory + */ public class NamedThreadFactory implements ThreadFactory { private final AtomicInteger increment = new AtomicInteger(1); + /** + * name + */ private final String name; + /** + * count + */ private final int count; public NamedThreadFactory(String name){ @@ -36,6 +45,11 @@ public class NamedThreadFactory implements ThreadFactory { this.count = count; } + /** + * create thread + * @param r runnable + * @return thread + */ @Override public Thread newThread(Runnable r) { final String threadName = count > 0 ? String.format(name + "_%d_%d", count, increment.getAndIncrement()) diff --git a/dolphinscheduler-remote/src/test/java/org/apache/dolphinscheduler/remote/NettyRemotingClientTest.java b/dolphinscheduler-remote/src/test/java/org/apache/dolphinscheduler/remote/NettyRemotingClientTest.java index 1ad6734b49..b6f8e2a8de 100644 --- a/dolphinscheduler-remote/src/test/java/org/apache/dolphinscheduler/remote/NettyRemotingClientTest.java +++ b/dolphinscheduler-remote/src/test/java/org/apache/dolphinscheduler/remote/NettyRemotingClientTest.java @@ -41,7 +41,7 @@ public class NettyRemotingClientTest { /** - * test ping + * test sned sync */ @Test public void testSendSync(){ @@ -69,6 +69,9 @@ public class NettyRemotingClientTest { } } + /** + * test sned async + */ @Test public void testSendAsync(){ NettyServerConfig serverConfig = new NettyServerConfig(); diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogPromise.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogPromise.java index 8920b8a527..98ee3fdbbf 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogPromise.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogPromise.java @@ -28,14 +28,29 @@ public class LogPromise { private static final ConcurrentHashMap PROMISES = new ConcurrentHashMap<>(); + /** + * request unique identification + */ private long opaque; + /** + * start timemillis + */ private final long start; + /** + * timeout + */ private final long timeout; + /** + * latch + */ private final CountDownLatch latch; + /** + * result + */ private Object result; public LogPromise(long opaque, long timeout){ @@ -59,15 +74,28 @@ public class LogPromise { } } + /** + * countdown + * + * @param result result + */ private void doCountDown(Object result){ this.result = result; this.latch.countDown(); } + /** + * whether timeout + * @return timeout + */ public boolean isTimeout(){ return System.currentTimeMillis() - start > timeout; } + /** + * get result + * @return + */ public Object getResult(){ try { latch.await(timeout, TimeUnit.MILLISECONDS); From 371a9d192afe4adb4fbac40bb6a54edaca2c1f6c Mon Sep 17 00:00:00 2001 From: qiaozhanwei Date: Sun, 16 Feb 2020 18:45:01 +0800 Subject: [PATCH 023/221] LogClientService modify --- .../apache/dolphinscheduler/service/log/LogClientService.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java index 3ca2d29782..5daf535625 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java @@ -16,6 +16,7 @@ */ package org.apache.dolphinscheduler.service.log; +import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.remote.NettyRemotingClient; import org.apache.dolphinscheduler.remote.command.Command; import org.apache.dolphinscheduler.remote.command.log.*; @@ -78,7 +79,7 @@ public class LogClientService { Command response = this.client.sendSync(address, command, logRequestTimeout); if(response != null){ RollViewLogResponseCommand rollReviewLog = FastJsonSerializer.deserialize( - command.getBody(), RollViewLogResponseCommand.class); + response.getBody(), RollViewLogResponseCommand.class); return rollReviewLog.getMsg(); } } catch (Exception e) { From 619fb30c503ef5039657a898c42b63ebc5af851e Mon Sep 17 00:00:00 2001 From: Yelli Date: Thu, 20 Feb 2020 13:25:41 +0800 Subject: [PATCH 024/221] #1310 add executor user to the process instance & task instance pages (#1973) * add executor name in processDefinitionInstance & taskInstance * modify processInstanceTest * modify processInstanceTest * #1310 add executor user to the process instance & task instance pages * add processInstanceUT & taskInstanceUT * add processInstanceUT & taskInstanceUT * modify processInstanceTest & taskInstanceTest * Remove duplicate code * add userServiceUT --- .../dao/mapper/ProcessInstanceMapper.xml | 3 + .../dao/mapper/TaskInstanceMapper.xml | 3 + .../controller/ProcessInstanceController.java | 8 +- .../controller/TaskInstanceController.java | 8 +- .../api/service/ProcessInstanceService.java | 17 +- .../api/service/TaskInstanceService.java | 29 ++- .../api/service/UsersService.java | 20 +++ .../service/ProcessInstanceServiceTest.java | 169 +++++++++++++++--- .../api/service/TaskInstanceServiceTest.java | 156 ++++++++++++++-- .../api/service/UsersServiceTest.java | 22 +++ .../dao/entity/ProcessInstance.java | 14 ++ .../dao/entity/TaskInstance.java | 51 ++++-- .../dao/mapper/ProcessInstanceMapper.java | 1 + .../dao/mapper/TaskInstanceMapper.java | 6 +- .../dao/mapper/ProcessInstanceMapper.xml | 3 + .../dao/mapper/TaskInstanceMapper.xml | 3 + .../dao/mapper/ProcessInstanceMapperTest.java | 1 + .../dao/mapper/TaskInstanceMapperTest.java | 1 + .../service/process/ProcessService.java | 1 + .../_source/instanceConditions/index.vue | 7 +- .../instance/pages/list/_source/list.vue | 7 + .../pages/instance/pages/list/index.vue | 4 +- .../pages/taskInstance/_source/list.vue | 7 + .../projects/pages/taskInstance/index.vue | 6 +- .../src/js/module/i18n/locale/en_US.js | 1 + .../src/js/module/i18n/locale/zh_CN.js | 1 + pom.xml | 2 + sql/dolphinscheduler-postgre.sql | 1 + sql/dolphinscheduler_mysql.sql | 1 + .../mysql/dolphinscheduler_ddl.sql | 20 +++ .../postgresql/dolphinscheduler_ddl.sql | 17 ++ 31 files changed, 518 insertions(+), 72 deletions(-) diff --git a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapper.xml b/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapper.xml index 2e63867d33..ccd4463d16 100644 --- a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapper.xml +++ b/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapper.xml @@ -90,6 +90,9 @@ and instance.host like concat('%', #{host}, '%') + + and instance.executorId = #{executorId} + order by instance.start_time desc diff --git a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml b/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml index 3a1fddd288..143761bf8c 100644 --- a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml +++ b/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml @@ -124,6 +124,9 @@ and instance.host like concat('%', #{host}, '%') + + and instance.executor_id = #{executorId} + order by instance.start_time desc diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ProcessInstanceController.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ProcessInstanceController.java index 80db6c86af..102f116575 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ProcessInstanceController.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ProcessInstanceController.java @@ -75,6 +75,7 @@ public class ProcessInstanceController extends BaseController{ @ApiImplicitParams({ @ApiImplicitParam(name = "processDefinitionId", value = "PROCESS_DEFINITION_ID", dataType = "Int", example = "100"), @ApiImplicitParam(name = "searchVal", value = "SEARCH_VAL", type ="String"), + @ApiImplicitParam(name = "executorName", value = "EXECUTOR_NAME", type ="String"), @ApiImplicitParam(name = "stateType", value = "EXECUTION_STATUS", type ="ExecutionStatus"), @ApiImplicitParam(name = "host", value = "HOST", type ="String"), @ApiImplicitParam(name = "startDate", value = "START_DATE", type ="String"), @@ -88,6 +89,7 @@ public class ProcessInstanceController extends BaseController{ @ApiParam(name = "projectName", value = "PROJECT_NAME", required = true) @PathVariable String projectName, @RequestParam(value = "processDefinitionId", required = false, defaultValue = "0") Integer processDefinitionId, @RequestParam(value = "searchVal", required = false) String searchVal, + @RequestParam(value = "executorName", required = false) String executorName, @RequestParam(value = "stateType", required = false) ExecutionStatus stateType, @RequestParam(value = "host", required = false) String host, @RequestParam(value = "startDate", required = false) String startTime, @@ -96,12 +98,12 @@ public class ProcessInstanceController extends BaseController{ @RequestParam("pageSize") Integer pageSize){ try{ logger.info("query all process instance list, login user:{},project name:{}, define id:{}," + - "search value:{},state type:{},host:{},start time:{}, end time:{},page number:{}, page size:{}", - loginUser.getUserName(), projectName, processDefinitionId, searchVal, stateType,host, + "search value:{},executor name:{},state type:{},host:{},start time:{}, end time:{},page number:{}, page size:{}", + loginUser.getUserName(), projectName, processDefinitionId, searchVal, executorName,stateType,host, startTime, endTime, pageNo, pageSize); searchVal = ParameterUtils.handleEscapes(searchVal); Map result = processInstanceService.queryProcessInstanceList( - loginUser, projectName, processDefinitionId, startTime, endTime, searchVal, stateType, host, pageNo, pageSize); + loginUser, projectName, processDefinitionId, startTime, endTime, searchVal, executorName, stateType, host, pageNo, pageSize); return returnDataListPaging(result); }catch (Exception e){ logger.error(QUERY_PROCESS_INSTANCE_LIST_PAGING_ERROR.getMsg(),e); diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskInstanceController.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskInstanceController.java index 5f63d744cf..276d2ff7da 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskInstanceController.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/TaskInstanceController.java @@ -69,6 +69,7 @@ public class TaskInstanceController extends BaseController{ @ApiImplicitParam(name = "processInstanceId", value = "PROCESS_INSTANCE_ID",required = false, dataType = "Int", example = "100"), @ApiImplicitParam(name = "searchVal", value = "SEARCH_VAL", type ="String"), @ApiImplicitParam(name = "taskName", value = "TASK_NAME", type ="String"), + @ApiImplicitParam(name = "executorName", value = "EXECUTOR_NAME", type ="String"), @ApiImplicitParam(name = "stateType", value = "EXECUTION_STATUS", type ="ExecutionStatus"), @ApiImplicitParam(name = "host", value = "HOST", type ="String"), @ApiImplicitParam(name = "startDate", value = "START_DATE", type ="String"), @@ -83,6 +84,7 @@ public class TaskInstanceController extends BaseController{ @RequestParam(value = "processInstanceId", required = false, defaultValue = "0") Integer processInstanceId, @RequestParam(value = "searchVal", required = false) String searchVal, @RequestParam(value = "taskName", required = false) String taskName, + @RequestParam(value = "executorName", required = false) String executorName, @RequestParam(value = "stateType", required = false) ExecutionStatus stateType, @RequestParam(value = "host", required = false) String host, @RequestParam(value = "startDate", required = false) String startTime, @@ -91,11 +93,11 @@ public class TaskInstanceController extends BaseController{ @RequestParam("pageSize") Integer pageSize){ try{ - logger.info("query task instance list, project name:{},process instance:{}, search value:{},task name:{}, state type:{}, host:{}, start:{}, end:{}", - projectName, processInstanceId, searchVal, taskName, stateType, host, startTime, endTime); + logger.info("query task instance list, project name:{},process instance:{}, search value:{},task name:{}, executor name: {},state type:{}, host:{}, start:{}, end:{}", + projectName, processInstanceId, searchVal, taskName, executorName, stateType, host, startTime, endTime); searchVal = ParameterUtils.handleEscapes(searchVal); Map result = taskInstanceService.queryTaskListPaging( - loginUser, projectName, processInstanceId, taskName, startTime, endTime, searchVal, stateType, host, pageNo, pageSize); + loginUser, projectName, processInstanceId, taskName, executorName, startTime, endTime, searchVal, stateType, host, pageNo, pageSize); return returnDataListPaging(result); }catch (Exception e){ logger.error(Status.QUERY_TASK_LIST_PAGING_ERROR.getMsg(),e); diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessInstanceService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessInstanceService.java index 2b1f04e6ce..4f81d89505 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessInstanceService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessInstanceService.java @@ -95,6 +95,9 @@ public class ProcessInstanceService extends BaseDAGService { @Autowired WorkerGroupMapper workerGroupMapper; + @Autowired + UsersService usersService; + /** * query process instance by id * @@ -151,7 +154,7 @@ public class ProcessInstanceService extends BaseDAGService { */ public Map queryProcessInstanceList(User loginUser, String projectName, Integer processDefineId, String startDate, String endDate, - String searchVal, ExecutionStatus stateType, String host, + String searchVal, String executorName,ExecutionStatus stateType, String host, Integer pageNo, Integer pageSize) { Map result = new HashMap<>(5); @@ -182,25 +185,31 @@ public class ProcessInstanceService extends BaseDAGService { putMsg(result, Status.REQUEST_PARAMS_NOT_VALID_ERROR, "startDate,endDate"); return result; } + Page page = new Page(pageNo, pageSize); + PageInfo pageInfo = new PageInfo(pageNo, pageSize); + int executorId = usersService.getUserIdByName(executorName); IPage processInstanceList = processInstanceMapper.queryProcessInstanceListPaging(page, - project.getId(), processDefineId, searchVal, statusArray, host, start, end); + project.getId(), processDefineId, searchVal, executorId,statusArray, host, start, end); List processInstances = processInstanceList.getRecords(); for(ProcessInstance processInstance: processInstances){ processInstance.setDuration(DateUtils.differSec(processInstance.getStartTime(),processInstance.getEndTime())); + User executor = usersService.queryUser(processInstance.getExecutorId()); + if (null != executor) { + processInstance.setExecutorName(executor.getUserName()); + } } - Set exclusionSet = new HashSet(); + Set exclusionSet = new HashSet<>(); exclusionSet.add(Constants.CLASS); exclusionSet.add("locations"); exclusionSet.add("connects"); exclusionSet.add("processInstanceJson"); - PageInfo pageInfo = new PageInfo(pageNo, pageSize); pageInfo.setTotalCount((int) processInstanceList.getTotal()); pageInfo.setLists(CollectionUtils.getListByExclusion(processInstances, exclusionSet)); result.put(Constants.DATA_LIST, pageInfo); diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskInstanceService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskInstanceService.java index 9690f5c69f..e4fec54395 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskInstanceService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/TaskInstanceService.java @@ -17,6 +17,8 @@ package org.apache.dolphinscheduler.api.service; +import com.baomidou.mybatisplus.core.metadata.IPage; +import com.baomidou.mybatisplus.extension.plugins.pagination.Page; import org.apache.dolphinscheduler.api.enums.Status; import org.apache.dolphinscheduler.api.utils.PageInfo; import org.apache.dolphinscheduler.common.Constants; @@ -24,14 +26,11 @@ import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.utils.CollectionUtils; import org.apache.dolphinscheduler.common.utils.DateUtils; import org.apache.dolphinscheduler.common.utils.StringUtils; -import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.Project; import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.dao.entity.User; import org.apache.dolphinscheduler.dao.mapper.ProjectMapper; import org.apache.dolphinscheduler.dao.mapper.TaskInstanceMapper; -import com.baomidou.mybatisplus.core.metadata.IPage; -import com.baomidou.mybatisplus.extension.plugins.pagination.Page; import org.apache.dolphinscheduler.service.process.ProcessService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -61,6 +60,12 @@ public class TaskInstanceService extends BaseService { @Autowired TaskInstanceMapper taskInstanceMapper; + @Autowired + ProcessInstanceService processInstanceService; + + @Autowired + UsersService usersService; + /** * query task list by project, process instance, task name, task start time, task end time, task status, keyword paging @@ -79,8 +84,8 @@ public class TaskInstanceService extends BaseService { * @return task list page */ public Map queryTaskListPaging(User loginUser, String projectName, - Integer processInstanceId, String taskName, String startDate, String endDate, - String searchVal, ExecutionStatus stateType,String host, + Integer processInstanceId, String taskName, String executorName, String startDate, + String endDate, String searchVal, ExecutionStatus stateType,String host, Integer pageNo, Integer pageSize) { Map result = new HashMap<>(5); Project project = projectMapper.queryByName(projectName); @@ -112,17 +117,23 @@ public class TaskInstanceService extends BaseService { } Page page = new Page(pageNo, pageSize); + PageInfo pageInfo = new PageInfo(pageNo, pageSize); + int executorId = usersService.getUserIdByName(executorName); + IPage taskInstanceIPage = taskInstanceMapper.queryTaskInstanceListPaging( - page, project.getId(), processInstanceId, searchVal, taskName, statusArray, host, start, end + page, project.getId(), processInstanceId, searchVal, taskName, executorId, statusArray, host, start, end ); - PageInfo pageInfo = new PageInfo(pageNo, pageSize); Set exclusionSet = new HashSet<>(); exclusionSet.add(Constants.CLASS); exclusionSet.add("taskJson"); List taskInstanceList = taskInstanceIPage.getRecords(); + for(TaskInstance taskInstance : taskInstanceList){ - taskInstance.setDuration(DateUtils.differSec(taskInstance.getStartTime(), - taskInstance.getEndTime())); + taskInstance.setDuration(DateUtils.differSec(taskInstance.getStartTime(), taskInstance.getEndTime())); + User executor = usersService.queryUser(taskInstance.getExecutorId()); + if (null != executor) { + taskInstance.setExecutorName(executor.getUserName()); + } } pageInfo.setTotalCount((int)taskInstanceIPage.getTotal()); pageInfo.setLists(CollectionUtils.getListByExclusion(taskInstanceIPage.getRecords(),exclusionSet)); diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/UsersService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/UsersService.java index fb8dcc97ab..1e5ec9e369 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/UsersService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/UsersService.java @@ -191,6 +191,26 @@ public class UsersService extends BaseService { return userMapper.queryUserByNamePassword(name, md5); } + /** + * get user id by user name + * @param name user name + * @return if name empty 0, user not exists -1, user exist user id + */ + public int getUserIdByName(String name) { + //executor name query + int executorId = 0; + if (StringUtils.isNotEmpty(name)) { + User executor = queryUser(name); + if (null != executor) { + executorId = executor.getId(); + } else { + executorId = -1; + } + } + + return executorId; + } + /** * query user list * diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessInstanceServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessInstanceServiceTest.java index e0a52bb3a7..18f1d6947f 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessInstanceServiceTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessInstanceServiceTest.java @@ -16,43 +16,131 @@ */ package org.apache.dolphinscheduler.api.service; +import com.baomidou.mybatisplus.extension.plugins.pagination.Page; import org.apache.dolphinscheduler.api.ApiApplicationServer; import org.apache.dolphinscheduler.api.enums.Status; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.DependResult; import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.enums.UserType; +import org.apache.dolphinscheduler.common.utils.DateUtils; +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; +import org.apache.dolphinscheduler.dao.entity.Project; import org.apache.dolphinscheduler.dao.entity.User; -import com.alibaba.fastjson.JSON; +import org.apache.dolphinscheduler.dao.mapper.*; +import org.apache.dolphinscheduler.service.process.ProcessService; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; +import org.mockito.InjectMocks; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.MockitoJUnitRunner; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.springframework.beans.factory.annotation.Autowired; import org.springframework.boot.test.context.SpringBootTest; -import org.springframework.test.context.junit4.SpringRunner; import java.io.IOException; -import java.util.Map; +import java.text.MessageFormat; +import java.util.*; -@RunWith(SpringRunner.class) +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.Silent.class) @SpringBootTest(classes = ApiApplicationServer.class) public class ProcessInstanceServiceTest { private static final Logger logger = LoggerFactory.getLogger(ProcessInstanceServiceTest.class); - @Autowired + @InjectMocks ProcessInstanceService processInstanceService; + @Mock + ProjectMapper projectMapper; + + @Mock + ProjectService projectService; + + @Mock + ProcessService processService; + + @Mock + ProcessInstanceMapper processInstanceMapper; + + @Mock + ProcessDefinitionMapper processDefineMapper; + + @Mock + ProcessDefinitionService processDefinitionService; + + @Mock + ExecutorService execService; + + @Mock + TaskInstanceMapper taskInstanceMapper; + + @Mock + LoggerService loggerService; + + @Mock + WorkerGroupMapper workerGroupMapper; + + @Mock + UsersService usersService; + @Test - public void viewVariables() { - try { - Map map = processInstanceService.viewVariables(-1); - Assert.assertEquals(Status.SUCCESS, map.get(Constants.STATUS)); - logger.info(JSON.toJSONString(map)); - }catch (Exception e){ - logger.error(e.getMessage(), e); - } + public void testQueryProcessInstanceList() { + String projectName = "project_test1"; + User loginUser = getAdminUser(); + Map result = new HashMap<>(5); + putMsg(result, Status.PROJECT_NOT_FOUNT, projectName); + + //project auth fail + when(projectMapper.queryByName(projectName)).thenReturn(null); + when(projectService.checkProjectAndAuth(loginUser,null,projectName)).thenReturn(result); + Map proejctAuthFailRes = processInstanceService.queryProcessInstanceList(loginUser, projectName, 46, "2020-01-01 00:00:00", + "2020-01-02 00:00:00", "", "test_user", ExecutionStatus.SUBMITTED_SUCCESS, + "192.168.xx.xx", 1, 10); + Assert.assertEquals(Status.PROJECT_NOT_FOUNT, proejctAuthFailRes.get(Constants.STATUS)); + + //project auth success + putMsg(result, Status.SUCCESS, projectName); + Project project = getProject(projectName); + Date start = DateUtils.getScheduleDate("2020-01-01 00:00:00"); + Date end = DateUtils.getScheduleDate("2020-01-02 00:00:00"); + ProcessInstance processInstance = getProcessInstance(); + List processInstanceList = new ArrayList<>(); + Page pageReturn = new Page<>(1, 10); + processInstanceList.add(processInstance); + pageReturn.setRecords(processInstanceList); + when(projectMapper.queryByName(projectName)).thenReturn(project); + when(projectService.checkProjectAndAuth(loginUser,project,projectName)).thenReturn(result); + when(usersService.queryUser(loginUser.getId())).thenReturn(loginUser); + when(usersService.getUserIdByName(loginUser.getUserName())).thenReturn(loginUser.getId()); + when(processInstanceMapper.queryProcessInstanceListPaging(Mockito.any(Page.class), eq(project.getId()), eq(1), eq(""), eq(-1), Mockito.any(), + eq("192.168.xx.xx"), eq(start), eq(end))).thenReturn(pageReturn); + when(usersService.queryUser(processInstance.getExecutorId())).thenReturn(loginUser); + Map successRes = processInstanceService.queryProcessInstanceList(loginUser, projectName, 1, "2020-01-01 00:00:00", + "2020-01-02 00:00:00", "", loginUser.getUserName(), ExecutionStatus.SUBMITTED_SUCCESS, + "192.168.xx.xx", 1, 10); + Assert.assertEquals(Status.SUCCESS, successRes.get(Constants.STATUS)); + + //executor null + when(usersService.queryUser(loginUser.getId())).thenReturn(null); + when(usersService.getUserIdByName(loginUser.getUserName())).thenReturn(-1); + Map executorExistRes = processInstanceService.queryProcessInstanceList(loginUser, projectName, 1, "2020-01-01 00:00:00", + "2020-01-02 00:00:00", "", "admin", ExecutionStatus.SUBMITTED_SUCCESS, + "192.168.xx.xx", 1, 10); + Assert.assertEquals(Status.SUCCESS, executorExistRes.get(Constants.STATUS)); + + //executor name empty + when(processInstanceMapper.queryProcessInstanceListPaging(Mockito.any(Page.class), eq(project.getId()), eq(1), eq(""), eq(0), Mockito.any(), + eq("192.168.xx.xx"), eq(start), eq(end))).thenReturn(pageReturn); + Map executorEmptyRes = processInstanceService.queryProcessInstanceList(loginUser, projectName, 1, "2020-01-01 00:00:00", + "2020-01-02 00:00:00", "", "", ExecutionStatus.SUBMITTED_SUCCESS, + "192.168.xx.xx", 1, 10); + Assert.assertEquals(Status.SUCCESS, executorEmptyRes.get(Constants.STATUS)); + } @Test @@ -63,21 +151,58 @@ public class ProcessInstanceServiceTest { try { Map resultMap = processInstanceService.parseLogForDependentResult(logString); - Assert.assertEquals(resultMap.size() , 1); + Assert.assertEquals(1 , resultMap.size()); } catch (IOException e) { } } - @Test - public void queryProcessInstanceList() throws Exception { - + /** + * get Mock Admin User + * @return admin user + */ + private User getAdminUser() { User loginUser = new User(); - loginUser.setId(27); + loginUser.setId(-1); + loginUser.setUserName("admin"); loginUser.setUserType(UserType.GENERAL_USER); - Map map = processInstanceService.queryProcessInstanceList(loginUser, "project_test1", 0, "", "", "", ExecutionStatus.FAILURE, "", 1, 5); + return loginUser; + } - Assert.assertEquals(Status.PROJECT_NOT_FOUNT, map.get(Constants.STATUS)); - logger.info(JSON.toJSONString(map)); + /** + * get mock Project + * @param projectName projectName + * @return Project + */ + private Project getProject(String projectName){ + Project project = new Project(); + project.setId(1); + project.setName(projectName); + project.setUserId(1); + return project; } + + /** + * get Mock process instance + * @return process instance + */ + private ProcessInstance getProcessInstance() { + ProcessInstance processInstance = new ProcessInstance(); + processInstance.setId(1); + processInstance.setName("test_process_instance"); + processInstance.setStartTime(new Date()); + processInstance.setEndTime(new Date()); + return processInstance; + } + + private void putMsg(Map result, Status status, Object... statusParams) { + result.put(Constants.STATUS, status); + if (statusParams != null && statusParams.length > 0) { + result.put(Constants.MSG, MessageFormat.format(status.getMsg(), statusParams)); + } else { + result.put(Constants.MSG, status.getMsg()); + } + } + + } \ No newline at end of file diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskInstanceServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskInstanceServiceTest.java index 931f2cea37..ebb6139577 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskInstanceServiceTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TaskInstanceServiceTest.java @@ -16,47 +16,177 @@ */ package org.apache.dolphinscheduler.api.service; +import com.baomidou.mybatisplus.extension.plugins.pagination.Page; import org.apache.dolphinscheduler.api.ApiApplicationServer; import org.apache.dolphinscheduler.api.enums.Status; -import org.apache.dolphinscheduler.api.utils.PageInfo; import org.apache.dolphinscheduler.common.Constants; +import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.enums.UserType; +import org.apache.dolphinscheduler.common.utils.DateUtils; +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; +import org.apache.dolphinscheduler.dao.entity.Project; +import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.dolphinscheduler.dao.entity.User; +import org.apache.dolphinscheduler.dao.mapper.ProjectMapper; +import org.apache.dolphinscheduler.dao.mapper.TaskInstanceMapper; +import org.apache.dolphinscheduler.service.process.ProcessService; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; +import org.mockito.InjectMocks; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.MockitoJUnitRunner; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.springframework.beans.factory.annotation.Autowired; import org.springframework.boot.test.context.SpringBootTest; -import org.springframework.test.context.junit4.SpringRunner; -import java.util.Map; +import java.text.MessageFormat; +import java.util.*; -@RunWith(SpringRunner.class) +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.Silent.class) @SpringBootTest(classes = ApiApplicationServer.class) public class TaskInstanceServiceTest { private static final Logger logger = LoggerFactory.getLogger(TaskInstanceServiceTest.class); - @Autowired + @InjectMocks private TaskInstanceService taskInstanceService; + @Mock + ProjectMapper projectMapper; + + @Mock + ProjectService projectService; + + @Mock + ProcessService processService; + + @Mock + TaskInstanceMapper taskInstanceMapper; + + @Mock + ProcessInstanceService processInstanceService; + + @Mock + UsersService usersService; + @Test public void queryTaskListPaging(){ + String projectName = "project_test1"; + User loginUser = getAdminUser(); + Map result = new HashMap<>(5); + putMsg(result, Status.PROJECT_NOT_FOUNT, projectName); + + //project auth fail + when(projectMapper.queryByName(projectName)).thenReturn(null); + when(projectService.checkProjectAndAuth(loginUser,null,projectName)).thenReturn(result); + Map proejctAuthFailRes = taskInstanceService.queryTaskListPaging(loginUser, "project_test1", 0, "", + "test_user", "2019-02-26 19:48:00", "2019-02-26 19:48:22", "", null, "", 1, 20); + Assert.assertEquals(Status.PROJECT_NOT_FOUNT, proejctAuthFailRes.get(Constants.STATUS)); + + + //project + putMsg(result, Status.SUCCESS, projectName); + Project project = getProject(projectName); + Date start = DateUtils.getScheduleDate("2020-01-01 00:00:00"); + Date end = DateUtils.getScheduleDate("2020-01-02 00:00:00"); + ProcessInstance processInstance = getProcessInstance(); + TaskInstance taskInstance = getTaskInstance(); + List taskInstanceList = new ArrayList<>(); + Page pageReturn = new Page<>(1, 10); + taskInstanceList.add(taskInstance); + pageReturn.setRecords(taskInstanceList); + when(projectMapper.queryByName(Mockito.anyString())).thenReturn(project); + when(projectService.checkProjectAndAuth(loginUser,project,projectName)).thenReturn(result); + when(usersService.queryUser(loginUser.getId())).thenReturn(loginUser); + when(usersService.getUserIdByName(loginUser.getUserName())).thenReturn(loginUser.getId()); + when(taskInstanceMapper.queryTaskInstanceListPaging(Mockito.any(Page.class), eq(project.getId()), eq(1), eq(""), eq(""), + eq(0), Mockito.any(), eq("192.168.xx.xx"), eq(start), eq(end))).thenReturn(pageReturn); + when(usersService.queryUser(processInstance.getExecutorId())).thenReturn(loginUser); + when(processService.findProcessInstanceDetailById(taskInstance.getProcessInstanceId())).thenReturn(processInstance); + + Map successRes = taskInstanceService.queryTaskListPaging(loginUser, projectName, 1, "", + "test_user", "2020-01-01 00:00:00", "2020-01-02 00:00:00", "", ExecutionStatus.SUCCESS, "192.168.xx.xx", 1, 20); + Assert.assertEquals(Status.SUCCESS, successRes.get(Constants.STATUS)); + + //executor name empty + when(taskInstanceMapper.queryTaskInstanceListPaging(Mockito.any(Page.class), eq(project.getId()), eq(1), eq(""), eq(""), + eq(0), Mockito.any(), eq("192.168.xx.xx"), eq(start), eq(end))).thenReturn(pageReturn); + Map executorEmptyRes = taskInstanceService.queryTaskListPaging(loginUser, projectName, 1, "", + "", "2020-01-01 00:00:00", "2020-01-02 00:00:00", "", ExecutionStatus.SUCCESS, "192.168.xx.xx", 1, 20); + Assert.assertEquals(Status.SUCCESS, executorEmptyRes.get(Constants.STATUS)); + + //executor null + when(usersService.queryUser(loginUser.getId())).thenReturn(null); + when(usersService.getUserIdByName(loginUser.getUserName())).thenReturn(-1); + Map executorNullRes = taskInstanceService.queryTaskListPaging(loginUser, projectName, 1, "", + "test_user", "2020-01-01 00:00:00", "2020-01-02 00:00:00", "", ExecutionStatus.SUCCESS, "192.168.xx.xx", 1, 20); + Assert.assertEquals(Status.SUCCESS, executorNullRes.get(Constants.STATUS)); + } + + /** + * get Mock Admin User + * @return admin user + */ + private User getAdminUser() { User loginUser = new User(); loginUser.setId(-1); + loginUser.setUserName("admin"); loginUser.setUserType(UserType.GENERAL_USER); + return loginUser; + } - Map map = taskInstanceService.queryTaskListPaging(loginUser, "project_test1", 0, "", - "2019-02-26 19:48:00", "2019-02-26 19:48:22", "", null, "", 1, 20); - Assert.assertEquals(Status.PROJECT_NOT_FOUNT, map.get(Constants.STATUS)); - PageInfo pageInfo = (PageInfo) map.get("data"); + /** + * get mock Project + * @param projectName projectName + * @return Project + */ + private Project getProject(String projectName){ + Project project = new Project(); + project.setId(1); + project.setName(projectName); + project.setUserId(1); + return project; + } - if(pageInfo != null){ - logger.info(pageInfo.getLists().toString()); - } + /** + * get Mock process instance + * @return process instance + */ + private ProcessInstance getProcessInstance() { + ProcessInstance processInstance = new ProcessInstance(); + processInstance.setId(1); + processInstance.setName("test_process_instance"); + processInstance.setStartTime(new Date()); + processInstance.setEndTime(new Date()); + processInstance.setExecutorId(-1); + return processInstance; + } + /** + * get Mock task instance + * @return task instance + */ + private TaskInstance getTaskInstance() { + TaskInstance taskInstance = new TaskInstance(); + taskInstance.setId(1); + taskInstance.setName("test_task_instance"); + taskInstance.setStartTime(new Date()); + taskInstance.setEndTime(new Date()); + taskInstance.setExecutorId(-1); + return taskInstance; + } + private void putMsg(Map result, Status status, Object... statusParams) { + result.put(Constants.STATUS, status); + if (statusParams != null && statusParams.length > 0) { + result.put(Constants.MSG, MessageFormat.format(status.getMsg(), statusParams)); + } else { + result.put(Constants.MSG, status.getMsg()); + } } } \ No newline at end of file diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/UsersServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/UsersServiceTest.java index 30aabe93f2..efe9022ad7 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/UsersServiceTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/UsersServiceTest.java @@ -148,6 +148,28 @@ public class UsersServiceTest { Assert.assertTrue(queryUser!=null); } + @Test + public void testGetUserIdByName() { + User user = new User(); + user.setId(1); + user.setUserType(UserType.ADMIN_USER); + user.setUserName("test_user"); + + //user name null + int userId = usersService.getUserIdByName(""); + Assert.assertEquals(0, userId); + + //user not exist + when(usersService.queryUser(user.getUserName())).thenReturn(null); + int userNotExistId = usersService.getUserIdByName(user.getUserName()); + Assert.assertEquals(-1, userNotExistId); + + //user exist + when(usersService.queryUser(user.getUserName())).thenReturn(user); + int userExistId = usersService.getUserIdByName(user.getUserName()); + Assert.assertEquals(user.getId(), userExistId); + } + @Test diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessInstance.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessInstance.java index 54c96e932d..fb5d0cda42 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessInstance.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessInstance.java @@ -139,6 +139,12 @@ public class ProcessInstance { */ private int executorId; + /** + * executor name + */ + @TableField(exist = false) + private String executorName; + /** * tenant code */ @@ -472,6 +478,14 @@ public class ProcessInstance { return historyCmd; } + public String getExecutorName() { + return executorName; + } + + public void setExecutorName(String executorName) { + this.executorName = executorName; + } + public void setHistoryCmd(String historyCmd) { this.historyCmd = historyCmd; } diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java index 2db1eda8f4..785b67eb28 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java @@ -196,6 +196,17 @@ public class TaskInstance { */ private int workerGroupId; + /** + * executor id + */ + private int executorId; + + /** + * executor name + */ + @TableField(exist = false) + private String executorName; + public void init(String host,Date startTime,String executePath){ @@ -415,6 +426,22 @@ public class TaskInstance { this.retryInterval = retryInterval; } + public int getExecutorId() { + return executorId; + } + + public void setExecutorId(int executorId) { + this.executorId = executorId; + } + + public String getExecutorName() { + return executorName; + } + + public void setExecutorName(String executorName) { + this.executorName = executorName; + } + public Boolean isTaskComplete() { return this.getState().typeIsPause() @@ -466,6 +493,14 @@ public class TaskInstance { this.workerGroupId = workerGroupId; } + public String getDependentResult() { + return dependentResult; + } + + public void setDependentResult(String dependentResult) { + this.dependentResult = dependentResult; + } + @Override public String toString() { return "TaskInstance{" + @@ -485,27 +520,21 @@ public class TaskInstance { ", logPath='" + logPath + '\'' + ", retryTimes=" + retryTimes + ", alertFlag=" + alertFlag + - ", flag=" + flag + ", processInstance=" + processInstance + ", processDefine=" + processDefine + ", pid=" + pid + ", appLink='" + appLink + '\'' + ", flag=" + flag + - ", dependency=" + dependency + + ", dependency='" + dependency + '\'' + ", duration=" + duration + ", maxRetryTimes=" + maxRetryTimes + ", retryInterval=" + retryInterval + ", taskInstancePriority=" + taskInstancePriority + ", processInstancePriority=" + processInstancePriority + - ", workGroupId=" + workerGroupId + + ", dependentResult='" + dependentResult + '\'' + + ", workerGroupId=" + workerGroupId + + ", executorId=" + executorId + + ", executorName='" + executorName + '\'' + '}'; } - - public String getDependentResult() { - return dependentResult; - } - - public void setDependentResult(String dependentResult) { - this.dependentResult = dependentResult; - } } diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapper.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapper.java index 167928cb4b..9a5f261254 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapper.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapper.java @@ -82,6 +82,7 @@ public interface ProcessInstanceMapper extends BaseMapper { @Param("projectId") int projectId, @Param("processDefinitionId") Integer processDefinitionId, @Param("searchVal") String searchVal, + @Param("executorId") Integer executorId, @Param("states") int[] statusArray, @Param("host") String host, @Param("startTime") Date startTime, diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java index 12ce29faf4..ac23b25c9c 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java @@ -16,13 +16,12 @@ */ package org.apache.dolphinscheduler.dao.mapper; -import org.apache.dolphinscheduler.dao.entity.ExecuteStatusCount; -import org.apache.dolphinscheduler.dao.entity.TaskInstance; import com.baomidou.mybatisplus.core.mapper.BaseMapper; import com.baomidou.mybatisplus.core.metadata.IPage; import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.enums.Flag; -import org.apache.dolphinscheduler.common.enums.UserType; +import org.apache.dolphinscheduler.dao.entity.ExecuteStatusCount; +import org.apache.dolphinscheduler.dao.entity.TaskInstance; import org.apache.ibatis.annotations.Param; import java.util.Date; @@ -64,6 +63,7 @@ public interface TaskInstanceMapper extends BaseMapper { @Param("processInstanceId") Integer processInstanceId, @Param("searchVal") String searchVal, @Param("taskName") String taskName, + @Param("executorId") int executorId, @Param("states") int[] statusArray, @Param("host") String host, @Param("startTime") Date startTime, diff --git a/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapper.xml b/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapper.xml index d37ea92140..3559ca9c85 100644 --- a/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapper.xml +++ b/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapper.xml @@ -90,6 +90,9 @@ and instance.host like concat('%', #{host}, '%') + + and instance.executor_id = #{executorId} + order by instance.start_time desc diff --git a/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml b/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml index 3a1fddd288..143761bf8c 100644 --- a/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml +++ b/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml @@ -124,6 +124,9 @@ and instance.host like concat('%', #{host}, '%') + + and instance.executor_id = #{executorId} + order by instance.start_time desc diff --git a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapperTest.java b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapperTest.java index 951b3f0e38..3b307cc2ad 100644 --- a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapperTest.java +++ b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapperTest.java @@ -165,6 +165,7 @@ public class ProcessInstanceMapperTest { processDefinition.getProjectId(), processInstance.getProcessDefinitionId(), processInstance.getName(), + 0, stateArray, processInstance.getHost(), null, diff --git a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapperTest.java b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapperTest.java index 16ba4b06c4..51a22b756e 100644 --- a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapperTest.java +++ b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapperTest.java @@ -282,6 +282,7 @@ public class TaskInstanceMapperTest { task.getProcessInstanceId(), "", "", + 0, new int[0], "", null,null diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java index a26044e417..34b1e48d63 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java @@ -950,6 +950,7 @@ public class ProcessService { } } } + taskInstance.setExecutorId(processInstance.getExecutorId()); taskInstance.setProcessInstancePriority(processInstance.getProcessInstancePriority()); taskInstance.setState(getSubmitTaskState(taskInstance, processInstanceState)); taskInstance.setSubmitTime(new Date()); diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/_source/instanceConditions/index.vue b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/_source/instanceConditions/index.vue index 2f30f0bea4..1ef2e1f3e4 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/_source/instanceConditions/index.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/_source/instanceConditions/index.vue @@ -54,6 +54,9 @@
+
+ +
@@ -80,7 +83,9 @@ // search value searchVal: '', // host - host: '' + host: '', + // executor name + executorName: '' } } }, diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/_source/list.vue b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/_source/list.vue index 1410a67f90..57ae6bd685 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/_source/list.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/_source/list.vue @@ -28,6 +28,9 @@ {{$t('Process Name')}} + + {{$t('Executor')}} + {{$t('Run Type')}} @@ -67,6 +70,10 @@ {{item.name}} + + {{item.executorName}} + - + {{_rtRunningType(item.commandType)}} {{item.scheduleTime | formatDate}} diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/index.vue b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/index.vue index 7bcf9ac26b..b95d4ed720 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/index.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/instance/pages/list/index.vue @@ -71,7 +71,9 @@ // Start Time startDate: '', // End Time - endDate: '' + endDate: '', + // Exectuor Name + executorName: '' } } }, diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/taskInstance/_source/list.vue b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/taskInstance/_source/list.vue index bb972c3f3b..f7be553568 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/taskInstance/_source/list.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/taskInstance/_source/list.vue @@ -28,6 +28,9 @@ {{$t('Process Instance')}} + + {{$t('Executor')}} + {{$t('Node Type')}} @@ -64,6 +67,10 @@ {{item.name}} {{item.processInstanceName}} + + {{item.executorName}} + - + {{item.taskType}} diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/taskInstance/index.vue b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/taskInstance/index.vue index a23eee5fa0..4cb166647e 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/taskInstance/index.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/projects/pages/taskInstance/index.vue @@ -68,7 +68,9 @@ // start date startDate: '', // end date - endDate: '' + endDate: '', + // Exectuor Name + executorName: '' } } }, @@ -147,4 +149,4 @@ }, components: { mList, mInstanceConditions, mSpin, mListConstruction, mSecondaryMenu, mNoData } } - \ No newline at end of file + diff --git a/dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js b/dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js index 0402d7e398..b87c07a7e4 100644 --- a/dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js +++ b/dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js @@ -240,6 +240,7 @@ export default { 'Service-Master': 'Service-Master', 'Service-Worker': 'Service-Worker', 'Process Name': 'Process Name', + 'Executor': 'Executor', 'Run Type': 'Run Type', 'Scheduling Time': 'Scheduling Time', 'Run Times': 'Run Times', diff --git a/dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js b/dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js index 95eb4a1081..745df2175d 100644 --- a/dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js +++ b/dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js @@ -239,6 +239,7 @@ export default { 'Service-Master': '服务管理-Master', 'Service-Worker': '服务管理-Worker', 'Process Name': '工作流名称', + 'Executor': '执行用户', 'Run Type': '运行类型', 'Scheduling Time': '调度时间', 'Run Times': '运行次数', diff --git a/pom.xml b/pom.xml index e23a1625b0..f4150fb730 100644 --- a/pom.xml +++ b/pom.xml @@ -708,6 +708,8 @@ **/api/service/BaseDAGServiceTest.java **/api/service/LoggerServiceTest.java **/api/service/DataAnalysisServiceTest.java + **/api/service/ProcessInstanceServiceTest.java + **/api/service/TaskInstanceServiceTest.java **/alert/utils/ExcelUtilsTest.java **/alert/utils/FuncUtilsTest.java **/alert/utils/JSONUtilsTest.java diff --git a/sql/dolphinscheduler-postgre.sql b/sql/dolphinscheduler-postgre.sql index c68fd17be1..88d62737de 100644 --- a/sql/dolphinscheduler-postgre.sql +++ b/sql/dolphinscheduler-postgre.sql @@ -574,6 +574,7 @@ CREATE TABLE t_ds_task_instance ( max_retry_times int DEFAULT NULL , task_instance_priority int DEFAULT NULL , worker_group_id int DEFAULT '-1' , + executor_id int DEFAULT NULL , PRIMARY KEY (id) ) ; diff --git a/sql/dolphinscheduler_mysql.sql b/sql/dolphinscheduler_mysql.sql index ea0f9cb022..68393702fc 100644 --- a/sql/dolphinscheduler_mysql.sql +++ b/sql/dolphinscheduler_mysql.sql @@ -616,6 +616,7 @@ CREATE TABLE `t_ds_task_instance` ( `max_retry_times` int(2) DEFAULT NULL COMMENT 'max retry times', `task_instance_priority` int(11) DEFAULT NULL COMMENT 'task instance priority:0 Highest,1 High,2 Medium,3 Low,4 Lowest', `worker_group_id` int(11) DEFAULT '-1' COMMENT 'worker group id', + `executor_id` int(11) DEFAULT NULL COMMENT 'executor id', PRIMARY KEY (`id`), KEY `process_instance_id` (`process_instance_id`) USING BTREE, KEY `task_instance_index` (`process_definition_id`,`process_instance_id`) USING BTREE, diff --git a/sql/upgrade/1.2.2_schema/mysql/dolphinscheduler_ddl.sql b/sql/upgrade/1.2.2_schema/mysql/dolphinscheduler_ddl.sql index 9fe246a8c2..790a9a8ada 100644 --- a/sql/upgrade/1.2.2_schema/mysql/dolphinscheduler_ddl.sql +++ b/sql/upgrade/1.2.2_schema/mysql/dolphinscheduler_ddl.sql @@ -35,3 +35,23 @@ d// delimiter ; CALL uc_dolphin_T_t_ds_process_definition_A_modify_by; DROP PROCEDURE uc_dolphin_T_t_ds_process_definition_A_modify_by; + +-- uc_dolphin_T_t_ds_process_definition_A_modify_by +drop PROCEDURE if EXISTS uc_dolphin_T_t_ds_task_instance_A_executor_id; +delimiter d// +CREATE PROCEDURE uc_dolphin_T_t_ds_task_instance_A_executor_id() + BEGIN + IF NOT EXISTS (SELECT 1 FROM information_schema.COLUMNS + WHERE TABLE_NAME='t_ds_task_instance' + AND TABLE_SCHEMA=(SELECT DATABASE()) + AND COLUMN_NAME ='executor_id') + THEN + ALTER TABLE t_ds_task_instance ADD `executor_id` int(11) DEFAULT NULL COMMENT 'executor id'; + END IF; + END; + +d// + +delimiter ; +CALL uc_dolphin_T_t_ds_task_instance_A_executor_id; +DROP PROCEDURE uc_dolphin_T_t_ds_task_instance_A_executor_id; \ No newline at end of file diff --git a/sql/upgrade/1.2.2_schema/postgresql/dolphinscheduler_ddl.sql b/sql/upgrade/1.2.2_schema/postgresql/dolphinscheduler_ddl.sql index 7fc12900e4..cbe7c22bbe 100644 --- a/sql/upgrade/1.2.2_schema/postgresql/dolphinscheduler_ddl.sql +++ b/sql/upgrade/1.2.2_schema/postgresql/dolphinscheduler_ddl.sql @@ -32,3 +32,20 @@ delimiter ; SELECT uc_dolphin_T_t_ds_process_definition_A_modify_by(); DROP FUNCTION IF EXISTS uc_dolphin_T_t_ds_process_definition_A_modify_by(); +-- uc_dolphin_T_t_ds_process_definition_A_modify_by +delimiter d// +CREATE OR REPLACE FUNCTION uc_dolphin_T_t_ds_task_instance_A_executor_id() RETURNS void AS $$ +BEGIN + IF NOT EXISTS (SELECT 1 FROM information_schema.COLUMNS + WHERE TABLE_NAME='t_ds_task_instance' + AND COLUMN_NAME ='executor_id') + THEN + ALTER TABLE t_ds_task_instance ADD COLUMN executor_id int DEFAULT NULL; + END IF; +END; +$$ LANGUAGE plpgsql; +d// + +delimiter ; +SELECT uc_dolphin_T_t_ds_task_instance_A_executor_id(); +DROP FUNCTION IF EXISTS uc_dolphin_T_t_ds_task_instance_A_executor_id(); From 34e0a663edbd01995c1e594b3b23a856db7b38a4 Mon Sep 17 00:00:00 2001 From: liwenhe1993 <32166572+liwenhe1993@users.noreply.github.com> Date: Thu, 20 Feb 2020 20:49:38 +0800 Subject: [PATCH 025/221] Refactor dockerfile (#1978) * Support DS to create user and group in windows environment * Add unit test * delete combined-server config in dolphinscheduler-daemon.sh file * refactor dockerfile * modify dockerfile * modify dockerfile --- dockerfile/Dockerfile | 180 +++++------ dockerfile/README.md | 289 +++++++++++++++++- .../alert.properties => alert.properties.tpl} | 32 +- ...perties => application-api.properties.tpl} | 17 +- ....properties => application.properties.tpl} | 60 ++-- ...ommon.properties => common.properties.tpl} | 69 +++-- .../dolphinscheduler/conf/alert_logback.xml | 49 --- .../conf/apiserver_logback.xml | 60 ---- .../conf/combined_logback.xml | 80 ----- .../conf/common/hadoop/hadoop.properties | 35 --- .../conf/config/install_config.conf | 20 -- .../conf/env/.dolphinscheduler_env.sh | 21 -- .../conf/env/.escheduler_env.sh | 20 -- .../conf/i18n/messages.properties | 252 --------------- .../conf/i18n/messages_en_US.properties | 252 --------------- .../conf/i18n/messages_zh_CN.properties | 250 --------------- .../mail_templates/alert_mail_template.ftl | 17 -- .../dolphinscheduler/conf/master_logback.xml | 52 ---- .../dao/mapper/AccessTokenMapper.xml | 33 -- .../dao/mapper/AlertGroupMapper.xml | 47 --- .../dao/mapper/AlertMapper.xml | 26 -- .../dao/mapper/CommandMapper.xml | 43 --- .../dao/mapper/DataSourceMapper.xml | 79 ----- .../dao/mapper/DataSourceUserMapper.xml | 30 -- .../dao/mapper/ErrorCommandMapper.xml | 36 --- .../dao/mapper/ProcessDefinitionMapper.xml | 96 ------ .../dao/mapper/ProcessInstanceMapMapper.xml | 43 --- .../dao/mapper/ProcessInstanceMapper.xml | 185 ----------- .../dao/mapper/ProjectMapper.xml | 68 ----- .../dao/mapper/ProjectUserMapper.xml | 36 --- .../dao/mapper/QueueMapper.xml | 42 --- .../dao/mapper/ResourceMapper.xml | 74 ----- .../dao/mapper/ResourceUserMapper.xml | 32 -- .../dao/mapper/ScheduleMapper.xml | 58 ---- .../dao/mapper/SessionMapper.xml | 32 -- .../dao/mapper/TaskInstanceMapper.xml | 132 -------- .../dao/mapper/TenantMapper.xml | 41 --- .../dao/mapper/UDFUserMapper.xml | 29 -- .../dao/mapper/UdfFuncMapper.xml | 71 ----- .../dao/mapper/UserAlertGroupMapper.xml | 31 -- .../dao/mapper/UserMapper.xml | 72 ----- .../dao/mapper/WorkerGroupMapper.xml | 40 --- .../dolphinscheduler/conf/worker_logback.xml | 79 ----- .../conf/zookeeper.properties | 42 --- .../dolphinscheduler_env} | 13 +- ...uartz.properties => quartz.properties.tpl} | 21 +- dockerfile/conf/maven/settings.xml | 263 ---------------- dockerfile/conf/nginx/dolphinscheduler.conf | 4 +- dockerfile/hooks/build | 15 +- dockerfile/hooks/build.bat | 33 ++ dockerfile/hooks/push | 2 +- dockerfile/hooks/push.bat | 22 ++ dockerfile/startup-init-conf.sh | 100 ++++++ dockerfile/startup.sh | 223 ++++++++++---- dolphinscheduler-ui/package.json | 2 +- script/dolphinscheduler-daemon.sh | 2 +- 56 files changed, 817 insertions(+), 3135 deletions(-) rename dockerfile/conf/dolphinscheduler/{conf/alert.properties => alert.properties.tpl} (68%) rename dockerfile/conf/dolphinscheduler/{conf/application-api.properties => application-api.properties.tpl} (90%) rename dockerfile/conf/dolphinscheduler/{conf/application-dao.properties => application.properties.tpl} (72%) rename dockerfile/conf/dolphinscheduler/{conf/common/common.properties => common.properties.tpl} (55%) delete mode 100644 dockerfile/conf/dolphinscheduler/conf/alert_logback.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/apiserver_logback.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/combined_logback.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/common/hadoop/hadoop.properties delete mode 100644 dockerfile/conf/dolphinscheduler/conf/config/install_config.conf delete mode 100644 dockerfile/conf/dolphinscheduler/conf/env/.dolphinscheduler_env.sh delete mode 100644 dockerfile/conf/dolphinscheduler/conf/env/.escheduler_env.sh delete mode 100644 dockerfile/conf/dolphinscheduler/conf/i18n/messages.properties delete mode 100644 dockerfile/conf/dolphinscheduler/conf/i18n/messages_en_US.properties delete mode 100644 dockerfile/conf/dolphinscheduler/conf/i18n/messages_zh_CN.properties delete mode 100644 dockerfile/conf/dolphinscheduler/conf/mail_templates/alert_mail_template.ftl delete mode 100644 dockerfile/conf/dolphinscheduler/conf/master_logback.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/AccessTokenMapper.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/AlertGroupMapper.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/AlertMapper.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/CommandMapper.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/DataSourceMapper.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/DataSourceUserMapper.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ErrorCommandMapper.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ProcessDefinitionMapper.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapMapper.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapper.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ProjectMapper.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ProjectUserMapper.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/QueueMapper.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ResourceMapper.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ResourceUserMapper.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ScheduleMapper.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/SessionMapper.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/TenantMapper.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/UDFUserMapper.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/UdfFuncMapper.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/UserAlertGroupMapper.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/UserMapper.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/WorkerGroupMapper.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/worker_logback.xml delete mode 100644 dockerfile/conf/dolphinscheduler/conf/zookeeper.properties rename dockerfile/conf/dolphinscheduler/{conf/config/run_config.conf => env/dolphinscheduler_env} (65%) rename dockerfile/conf/dolphinscheduler/{conf/quartz.properties => quartz.properties.tpl} (81%) delete mode 100644 dockerfile/conf/maven/settings.xml create mode 100644 dockerfile/hooks/build.bat create mode 100644 dockerfile/hooks/push.bat create mode 100644 dockerfile/startup-init-conf.sh diff --git a/dockerfile/Dockerfile b/dockerfile/Dockerfile index 217b2c052f..1fc064c489 100644 --- a/dockerfile/Dockerfile +++ b/dockerfile/Dockerfile @@ -15,122 +15,78 @@ # limitations under the License. # -FROM ubuntu:18.04 - -ENV LANG=C.UTF-8 -ENV DEBIAN_FRONTEND=noninteractive - -ARG version -ARG tar_version - -#1,install jdk - -RUN apt-get update \ - && apt-get -y install openjdk-8-jdk \ - && rm -rf /var/lib/apt/lists/* - -ENV JAVA_HOME /usr/lib/jvm/java-8-openjdk-amd64 +FROM nginx:alpine + +ARG VERSION + +ENV TZ Asia/Shanghai +ENV LANG C.UTF-8 +ENV DEBIAN_FRONTEND noninteractive + +#1. install dos2unix shadow bash openrc python sudo vim wget iputils net-tools ssh pip kazoo. +#If install slowly, you can replcae alpine's mirror with aliyun's mirror, Example: +#RUN sed -i "s/dl-cdn.alpinelinux.org/mirrors.aliyun.com/g" /etc/apk/repositories +RUN apk update && \ + apk add dos2unix shadow bash openrc python sudo vim wget iputils net-tools openssh-server py2-pip && \ + apk add --update procps && \ + openrc boot && \ + pip install kazoo + +#2. install jdk +RUN apk add openjdk8 +ENV JAVA_HOME /usr/lib/jvm/java-1.8-openjdk ENV PATH $JAVA_HOME/bin:$PATH - -#install wget -RUN apt-get update && \ - apt-get -y install wget -#2,install ZK - +#3. install zk RUN cd /opt && \ - wget https://www-us.apache.org/dist/zookeeper/zookeeper-3.4.14/zookeeper-3.4.14.tar.gz && \ - tar -zxvf zookeeper-3.4.14.tar.gz && \ - mv zookeeper-3.4.14 zookeeper && \ - rm -rf ./zookeeper-*tar.gz && \ + wget https://downloads.apache.org/zookeeper/zookeeper-3.5.7/apache-zookeeper-3.5.7-bin.tar.gz && \ + tar -zxvf apache-zookeeper-3.5.7-bin.tar.gz && \ + mv apache-zookeeper-3.5.7-bin zookeeper && \ mkdir -p /tmp/zookeeper && \ + rm -rf ./zookeeper-*tar.gz && \ rm -rf /opt/zookeeper/conf/zoo_sample.cfg - -ADD ./dockerfile/conf/zookeeper/zoo.cfg /opt/zookeeper/conf -ENV ZK_HOME=/opt/zookeeper -ENV PATH $PATH:$ZK_HOME/bin - -#3,install maven -RUN cd /opt && \ - wget http://apache-mirror.rbc.ru/pub/apache/maven/maven-3/3.3.9/binaries/apache-maven-3.3.9-bin.tar.gz && \ - tar -zxvf apache-maven-3.3.9-bin.tar.gz && \ - mv apache-maven-3.3.9 maven && \ - rm -rf ./apache-maven-*tar.gz && \ - rm -rf /opt/maven/conf/settings.xml -ADD ./dockerfile/conf/maven/settings.xml /opt/maven/conf -ENV MAVEN_HOME=/opt/maven -ENV PATH $PATH:$MAVEN_HOME/bin - -#4,install node -RUN cd /opt && \ - wget https://nodejs.org/download/release/v8.9.4/node-v8.9.4-linux-x64.tar.gz && \ - tar -zxvf node-v8.9.4-linux-x64.tar.gz && \ - mv node-v8.9.4-linux-x64 node && \ - rm -rf ./node-v8.9.4-*tar.gz -ENV NODE_HOME=/opt/node -ENV PATH $PATH:$NODE_HOME/bin - -#5,install postgresql -RUN apt-get update && \ - apt-get install -y postgresql postgresql-contrib sudo && \ - sed -i 's/localhost/*/g' /etc/postgresql/10/main/postgresql.conf - -#6,install nginx -RUN apt-get update && \ - apt-get install -y nginx && \ - rm -rf /var/lib/apt/lists/* && \ - echo "\ndaemon off;" >> /etc/nginx/nginx.conf && \ - chown -R www-data:www-data /var/lib/nginx - -#7,install sudo,python,vim,ping and ssh command -RUN apt-get update && \ - apt-get -y install sudo && \ - apt-get -y install python && \ - apt-get -y install vim && \ - apt-get -y install iputils-ping && \ - apt-get -y install net-tools && \ - apt-get -y install openssh-server && \ - apt-get -y install python-pip && \ - pip install kazoo - -#8,add dolphinscheduler source code to /opt/dolphinscheduler_source -ADD . /opt/dolphinscheduler_source - - -#9,backend compilation -RUN cd /opt/dolphinscheduler_source && \ - mvn clean package -Prelease -Dmaven.test.skip=true - -#10,frontend compilation -RUN chmod -R 777 /opt/dolphinscheduler_source/dolphinscheduler-ui && \ - cd /opt/dolphinscheduler_source/dolphinscheduler-ui && \ - rm -rf /opt/dolphinscheduler_source/dolphinscheduler-ui/node_modules && \ - npm install node-sass --unsafe-perm && \ - npm install && \ - npm run build - -#11,modify dolphinscheduler configuration file -#backend configuration -RUN tar -zxvf /opt/dolphinscheduler_source/dolphinscheduler-dist/dolphinscheduler-backend/target/apache-dolphinscheduler-incubating-${tar_version}-dolphinscheduler-backend-bin.tar.gz -C /opt && \ - mv /opt/apache-dolphinscheduler-incubating-${tar_version}-dolphinscheduler-backend-bin /opt/dolphinscheduler && \ - rm -rf /opt/dolphinscheduler/conf - -ADD ./dockerfile/conf/dolphinscheduler/conf /opt/dolphinscheduler/conf -#frontend nginx configuration -ADD ./dockerfile/conf/nginx/dolphinscheduler.conf /etc/nginx/conf.d - -#12,open port -EXPOSE 2181 2888 3888 3306 80 12345 8888 - -COPY ./dockerfile/startup.sh /root/startup.sh -#13,modify permissions and set soft links -RUN chmod +x /root/startup.sh && \ - chmod +x /opt/dolphinscheduler/script/create-dolphinscheduler.sh && \ - chmod +x /opt/zookeeper/bin/zkServer.sh && \ - chmod +x /opt/dolphinscheduler/bin/dolphinscheduler-daemon.sh && \ - rm -rf /bin/sh && \ - ln -s /bin/bash /bin/sh && \ - mkdir -p /tmp/xls - +ADD ./conf/zookeeper/zoo.cfg /opt/zookeeper/conf +ENV ZK_HOME /opt/zookeeper +ENV PATH $ZK_HOME/bin:$PATH + +#4. install pg +RUN apk add postgresql postgresql-contrib + +#5. add dolphinscheduler +ADD ./apache-dolphinscheduler-incubating-${VERSION}-SNAPSHOT-dolphinscheduler-bin.tar.gz /opt/ +RUN mv /opt/apache-dolphinscheduler-incubating-${VERSION}-SNAPSHOT-dolphinscheduler-bin/ /opt/dolphinscheduler/ +ENV DOLPHINSCHEDULER_HOME /opt/dolphinscheduler + +#6. modify nginx +RUN echo "daemon off;" >> /etc/nginx/nginx.conf && \ + rm -rf /etc/nginx/conf.d/* +ADD ./conf/nginx/dolphinscheduler.conf /etc/nginx/conf.d + +#7. add configuration and modify permissions and set soft links +ADD ./startup-init-conf.sh /root/startup-init-conf.sh +ADD ./startup.sh /root/startup.sh +ADD ./conf/dolphinscheduler/*.tpl /opt/dolphinscheduler/conf/ +ADD ./conf/dolphinscheduler/env/dolphinscheduler_env /opt/dolphinscheduler/conf/env/ +RUN chmod +x /root/startup-init-conf.sh && \ + chmod +x /root/startup.sh && \ + chmod +x /opt/dolphinscheduler/conf/env/dolphinscheduler_env && \ + chmod +x /opt/dolphinscheduler/script/*.sh && \ + chmod +x /opt/dolphinscheduler/bin/*.sh && \ + chmod +x /opt/zookeeper/bin/*.sh && \ + dos2unix /root/startup-init-conf.sh && \ + dos2unix /root/startup.sh && \ + dos2unix /opt/dolphinscheduler/conf/env/dolphinscheduler_env && \ + dos2unix /opt/dolphinscheduler/script/*.sh && \ + dos2unix /opt/dolphinscheduler/bin/*.sh && \ + dos2unix /opt/zookeeper/bin/*.sh && \ + rm -rf /bin/sh && \ + ln -s /bin/bash /bin/sh && \ + mkdir -p /tmp/xls + +#8. remove apk index cache +RUN rm -rf /var/cache/apk/* + +#9. expose port +EXPOSE 2181 2888 3888 5432 12345 8888 ENTRYPOINT ["/root/startup.sh"] \ No newline at end of file diff --git a/dockerfile/README.md b/dockerfile/README.md index 33b58cacde..fb1500b495 100644 --- a/dockerfile/README.md +++ b/dockerfile/README.md @@ -1,11 +1,284 @@ -## Build Image +## What is Dolphin Scheduler? + +Dolphin Scheduler is a distributed and easy-to-expand visual DAG workflow scheduling system, dedicated to solving the complex dependencies in data processing, making the scheduling system out of the box for data processing. + +Github URL: https://github.com/apache/incubator-dolphinscheduler + +Official Website: https://dolphinscheduler.apache.org + +![Dolphin Scheduler](https://dolphinscheduler.apache.org/img/hlogo_colorful.svg) + +## How to use this image + +#### You can start a dolphinscheduler instance +``` +$ docker run -d --name dolphinscheduler \ +-e POSTGRESQL_USERNAME=test -e POSTGRESQL_PASSWORD=test \ +-p 8888:8888 \ +dolphinscheduler all +``` + +The default postgres user `root`, postgres password `root` and database `dolphinscheduler` are created in the `startup.sh`. + +The default zookeeper is created in the `startup.sh`. + +#### Or via Environment Variables **`POSTGRESQL_HOST`** **`POSTGRESQL_PORT`** **`ZOOKEEPER_QUORUM`** + +You can specify **existing postgres service**. Example: + +``` +$ docker run -d --name dolphinscheduler \ +-e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" \ +-e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ +-p 8888:8888 \ +dolphinscheduler all +``` + +You can specify **existing zookeeper service**. Example: + +``` +$ docker run -d --name dolphinscheduler \ +-e ZOOKEEPER_QUORUM="l92.168.x.x:2181" +-e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ +-p 8888:8888 \ +dolphinscheduler all +``` + +#### Or start a standalone dolphinscheduler server + +You can start a standalone dolphinscheduler server. + +* Start a **master server**, For example: + +``` +$ docker run -d --name dolphinscheduler \ +-e ZOOKEEPER_QUORUM="l92.168.x.x:2181" +-e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" \ +-e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ +dolphinscheduler master-server +``` + +* Start a **worker server**, For example: + +``` +$ docker run -d --name dolphinscheduler \ +-e ZOOKEEPER_QUORUM="l92.168.x.x:2181" +-e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" \ +-e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ +dolphinscheduler worker-server +``` + +* Start a **api server**, For example: + +``` +$ docker run -d --name dolphinscheduler \ +-e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" \ +-e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ +-p 12345:12345 \ +dolphinscheduler api-server +``` + +* Start a **alert server**, For example: + +``` +$ docker run -d --name dolphinscheduler \ +-e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" \ +-e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ +dolphinscheduler alert-server +``` + +* Start a **frontend**, For example: + +``` +$ docker run -d --name dolphinscheduler \ +-e FRONTEND_API_SERVER_HOST="192.168.x.x" -e FRONTEND_API_SERVER_PORT="12345" \ +-p 8888:8888 \ +dolphinscheduler frontend +``` + +**Note**: You must be specify `POSTGRESQL_HOST` `POSTGRESQL_PORT` `ZOOKEEPER_QUORUM` when start a standalone dolphinscheduler server. + +## Environment Variables + +The Dolphin Scheduler image uses several environment variables which are easy to miss. While none of the variables are required, they may significantly aid you in using the image. + +**`POSTGRESQL_HOST`** + +This environment variable sets the host for PostgreSQL. The default value is `127.0.0.1`. + +**Note**: You must be specify it when start a standalone dolphinscheduler server. Like `master-server`, `worker-server`, `api-server`, `alert-server`. + +**`POSTGRESQL_PORT`** + +This environment variable sets the port for PostgreSQL. The default value is `5432`. + +**Note**: You must be specify it when start a standalone dolphinscheduler server. Like `master-server`, `worker-server`, `api-server`, `alert-server`. + +**`POSTGRESQL_USERNAME`** + +This environment variable sets the username for PostgreSQL. The default value is `root`. + +**`POSTGRESQL_PASSWORD`** + +This environment variable sets the password for PostgreSQL. The default value is `root`. + +**`DOLPHINSCHEDULER_ENV_PATH`** + +This environment variable sets the runtime environment for task. The default value is `/opt/dolphinscheduler/conf/env/dolphinscheduler_env.sh`. + +**`TASK_QUEUE`** + +This environment variable sets the task queue for `master-server` and `worker-serverr`. The default value is `zookeeper`. + +**`ZOOKEEPER_QUORUM`** + +This environment variable sets zookeeper quorum for `master-server` and `worker-serverr`. The default value is `127.0.0.1:2181`. + +**Note**: You must be specify it when start a standalone dolphinscheduler server. Like `master-server`, `worker-server`. + +**`MASTER_EXEC_THREADS`** + +This environment variable sets exec thread num for `master-server`. The default value is `100`. + +**`MASTER_EXEC_TASK_NUM`** + +This environment variable sets exec task num for `master-server`. The default value is `20`. + +**`MASTER_HEARTBEAT_INTERVAL`** + +This environment variable sets heartbeat interval for `master-server`. The default value is `10`. + +**`MASTER_TASK_COMMIT_RETRYTIMES`** + +This environment variable sets task commit retry times for `master-server`. The default value is `5`. + +**`MASTER_TASK_COMMIT_INTERVAL`** + +This environment variable sets task commit interval for `master-server`. The default value is `1000`. + +**`MASTER_MAX_CPULOAD_AVG`** + +This environment variable sets max cpu load avg for `master-server`. The default value is `100`. + +**`MASTER_RESERVED_MEMORY`** + +This environment variable sets reserved memory for `master-server`. The default value is `0.1`. + +**`WORKER_EXEC_THREADS`** + +This environment variable sets exec thread num for `worker-server`. The default value is `100`. + +**`WORKER_HEARTBEAT_INTERVAL`** + +This environment variable sets heartbeat interval for `worker-server`. The default value is `10`. + +**`WORKER_FETCH_TASK_NUM`** + +This environment variable sets fetch task num for `worker-server`. The default value is `3`. + +**`WORKER_MAX_CPULOAD_AVG`** + +This environment variable sets max cpu load avg for `worker-server`. The default value is `100`. + +**`WORKER_RESERVED_MEMORY`** + +This environment variable sets reserved memory for `worker-server`. The default value is `0.1`. + +**`XLS_FILE_PATH`** + +This environment variable sets xls file path for `alert-server`. The default value is `/tmp/xls`. + +**`MAIL_SERVER_HOST`** + +This environment variable sets mail server host for `alert-server`. The default value is empty. + +**`MAIL_SERVER_PORT`** + +This environment variable sets mail server port for `alert-server`. The default value is empty. + +**`MAIL_SENDER`** + +This environment variable sets mail sender for `alert-server`. The default value is empty. + +**`MAIL_USER=`** + +This environment variable sets mail user for `alert-server`. The default value is empty. + +**`MAIL_PASSWD`** + +This environment variable sets mail password for `alert-server`. The default value is empty. + +**`MAIL_SMTP_STARTTLS_ENABLE`** + +This environment variable sets SMTP tls for `alert-server`. The default value is `true`. + +**`MAIL_SMTP_SSL_ENABLE`** + +This environment variable sets SMTP ssl for `alert-server`. The default value is `false`. + +**`MAIL_SMTP_SSL_TRUST`** + +This environment variable sets SMTP ssl truest for `alert-server`. The default value is empty. + +**`ENTERPRISE_WECHAT_ENABLE`** + +This environment variable sets enterprise wechat enable for `alert-server`. The default value is `false`. + +**`ENTERPRISE_WECHAT_CORP_ID`** + +This environment variable sets enterprise wechat corp id for `alert-server`. The default value is empty. + +**`ENTERPRISE_WECHAT_SECRET`** + +This environment variable sets enterprise wechat secret for `alert-server`. The default value is empty. + +**`ENTERPRISE_WECHAT_AGENT_ID`** + +This environment variable sets enterprise wechat agent id for `alert-server`. The default value is empty. + +**`ENTERPRISE_WECHAT_USERS`** + +This environment variable sets enterprise wechat users for `alert-server`. The default value is empty. + +**`FRONTEND_API_SERVER_HOST`** + +This environment variable sets api server host for `frontend`. The default value is `127.0.0.1`. + +**Note**: You must be specify it when start a standalone dolphinscheduler server. Like `api-server`. + +**`FRONTEND_API_SERVER_PORT`** + +This environment variable sets api server port for `frontend`. The default value is `123451`. + +**Note**: You must be specify it when start a standalone dolphinscheduler server. Like `api-server`. + +## Initialization scripts + +If you would like to do additional initialization in an image derived from this one, add one or more environment variable under `/root/start-init-conf.sh`, and modify template files in `/opt/dolphinscheduler/conf/*.tpl`. + +For example, to add an environment variable `API_SERVER_PORT` in `/root/start-init-conf.sh`: + +``` +export API_SERVER_PORT=5555 +``` + +and to modify `/opt/dolphinscheduler/conf/application-api.properties.tpl` template file, add server port: ``` - cd .. - docker build -t dolphinscheduler --build-arg version=1.1.0 --build-arg tar_version=1.1.0-SNAPSHOT -f dockerfile/Dockerfile . - docker run -p 12345:12345 -p 8888:8888 --rm --name dolphinscheduler -d dolphinscheduler +server.port=${API_SERVER_PORT} ``` -* Visit the url: http://127.0.0.1:8888 -* UserName:admin Password:dolphinscheduler123 -## Note -* MacOS: The memory of docker needs to be set to 4G, default 2G. Steps: Preferences -> Advanced -> adjust resources -> Apply & Restart +`/root/start-init-conf.sh` will dynamically generate config file: + +```sh +echo "generate app config" +ls ${DOLPHINSCHEDULER_HOME}/conf/ | grep ".tpl" | while read line; do +eval "cat << EOF +$(cat ${DOLPHINSCHEDULER_HOME}/conf/${line}) +EOF +" > ${DOLPHINSCHEDULER_HOME}/conf/${line%.*} +done + +echo "generate nginx config" +sed -i "s/FRONTEND_API_SERVER_HOST/${FRONTEND_API_SERVER_HOST}/g" /etc/nginx/conf.d/dolphinscheduler.conf +sed -i "s/FRONTEND_API_SERVER_PORT/${FRONTEND_API_SERVER_PORT}/g" /etc/nginx/conf.d/dolphinscheduler.conf +``` diff --git a/dockerfile/conf/dolphinscheduler/conf/alert.properties b/dockerfile/conf/dolphinscheduler/alert.properties.tpl similarity index 68% rename from dockerfile/conf/dolphinscheduler/conf/alert.properties rename to dockerfile/conf/dolphinscheduler/alert.properties.tpl index 276ef3132a..b940ecd203 100644 --- a/dockerfile/conf/dolphinscheduler/conf/alert.properties +++ b/dockerfile/conf/dolphinscheduler/alert.properties.tpl @@ -14,33 +14,33 @@ # See the License for the specific language governing permissions and # limitations under the License. # - #alert type is EMAIL/SMS alert.type=EMAIL +# alter msg template, default is html template +#alert.template=html # mail server configuration mail.protocol=SMTP -mail.server.host=smtp.126.com -mail.server.port= -mail.sender=dolphinscheduler@126.com -mail.user=dolphinscheduler@126.com -mail.passwd=escheduler123 - +mail.server.host=${MAIL_SERVER_HOST} +mail.server.port=${MAIL_SERVER_PORT} +mail.sender=${MAIL_SENDER} +mail.user=${MAIL_USER} +mail.passwd=${MAIL_PASSWD} # TLS -mail.smtp.starttls.enable=false +mail.smtp.starttls.enable=${MAIL_SMTP_STARTTLS_ENABLE} # SSL -mail.smtp.ssl.enable=true -mail.smtp.ssl.trust=smtp.126.com +mail.smtp.ssl.enable=${MAIL_SMTP_SSL_ENABLE} +mail.smtp.ssl.trust=${MAIL_SMTP_SSL_TRUST} #xls file path,need create if not exist -xls.file.path=/tmp/xls +xls.file.path=${XLS_FILE_PATH} # Enterprise WeChat configuration -enterprise.wechat.enable=false -enterprise.wechat.corp.id=xxxxxxx -enterprise.wechat.secret=xxxxxxx -enterprise.wechat.agent.id=xxxxxxx -enterprise.wechat.users=xxxxxxx +enterprise.wechat.enable=${ENTERPRISE_WECHAT_ENABLE} +enterprise.wechat.corp.id=${ENTERPRISE_WECHAT_CORP_ID} +enterprise.wechat.secret=${ENTERPRISE_WECHAT_SECRET} +enterprise.wechat.agent.id=${ENTERPRISE_WECHAT_AGENT_ID} +enterprise.wechat.users=${ENTERPRISE_WECHAT_USERS} enterprise.wechat.token.url=https://qyapi.weixin.qq.com/cgi-bin/gettoken?corpid=$corpId&corpsecret=$secret enterprise.wechat.push.url=https://qyapi.weixin.qq.com/cgi-bin/message/send?access_token=$token enterprise.wechat.team.send.msg={\"toparty\":\"$toParty\",\"agentid\":\"$agentId\",\"msgtype\":\"text\",\"text\":{\"content\":\"$msg\"},\"safe\":\"0\"} diff --git a/dockerfile/conf/dolphinscheduler/conf/application-api.properties b/dockerfile/conf/dolphinscheduler/application-api.properties.tpl similarity index 90% rename from dockerfile/conf/dolphinscheduler/conf/application-api.properties rename to dockerfile/conf/dolphinscheduler/application-api.properties.tpl index ead8dd872e..424ea55f7d 100644 --- a/dockerfile/conf/dolphinscheduler/conf/application-api.properties +++ b/dockerfile/conf/dolphinscheduler/application-api.properties.tpl @@ -14,27 +14,24 @@ # See the License for the specific language governing permissions and # limitations under the License. # - -logging.config=classpath:apiserver_logback.xml - # server port server.port=12345 - # session config server.servlet.session.timeout=7200 - +# servlet config server.servlet.context-path=/dolphinscheduler/ - # file size limit for upload spring.servlet.multipart.max-file-size=1024MB spring.servlet.multipart.max-request-size=1024MB - -#post content +# post content server.jetty.max-http-post-size=5000000 - +# i18n spring.messages.encoding=UTF-8 - #i18n classpath folder , file prefix messages, if have many files, use "," seperator spring.messages.basename=i18n/messages +# Authentication types (supported types: PASSWORD) +security.authentication.type=PASSWORD + + diff --git a/dockerfile/conf/dolphinscheduler/conf/application-dao.properties b/dockerfile/conf/dolphinscheduler/application.properties.tpl similarity index 72% rename from dockerfile/conf/dolphinscheduler/conf/application-dao.properties rename to dockerfile/conf/dolphinscheduler/application.properties.tpl index 166c36fbf0..c643c414cd 100644 --- a/dockerfile/conf/dolphinscheduler/conf/application-dao.properties +++ b/dockerfile/conf/dolphinscheduler/application.properties.tpl @@ -17,70 +17,57 @@ # base spring data source configuration spring.datasource.type=com.alibaba.druid.pool.DruidDataSource -# postgresql +# postgre spring.datasource.driver-class-name=org.postgresql.Driver -spring.datasource.url=jdbc:postgresql://127.0.0.1:5432/dolphinscheduler -spring.datasource.username=root -spring.datasource.password=root@123 - +spring.datasource.url=jdbc:postgresql://${POSTGRESQL_HOST}:${POSTGRESQL_PORT}/dolphinscheduler?characterEncoding=utf8 +# mysql +#spring.datasource.driver-class-name=com.mysql.jdbc.Driver +#spring.datasource.url=jdbc:mysql://192.168.xx.xx:3306/dolphinscheduler?useUnicode=true&characterEncoding=UTF-8 +spring.datasource.username=${POSTGRESQL_USERNAME} +spring.datasource.password=${POSTGRESQL_PASSWORD} # connection configuration spring.datasource.initialSize=5 # min connection number spring.datasource.minIdle=5 # max connection number spring.datasource.maxActive=50 - # max wait time for get a connection in milliseconds. if configuring maxWait, fair locks are enabled by default and concurrency efficiency decreases. # If necessary, unfair locks can be used by configuring the useUnfairLock attribute to true. spring.datasource.maxWait=60000 - # milliseconds for check to close free connections spring.datasource.timeBetweenEvictionRunsMillis=60000 - # the Destroy thread detects the connection interval and closes the physical connection in milliseconds if the connection idle time is greater than or equal to minEvictableIdleTimeMillis. spring.datasource.timeBetweenConnectErrorMillis=60000 - # the longest time a connection remains idle without being evicted, in milliseconds spring.datasource.minEvictableIdleTimeMillis=300000 - #the SQL used to check whether the connection is valid requires a query statement. If validation Query is null, testOnBorrow, testOnReturn, and testWhileIdle will not work. spring.datasource.validationQuery=SELECT 1 - #check whether the connection is valid for timeout, in seconds spring.datasource.validationQueryTimeout=3 - # when applying for a connection, if it is detected that the connection is idle longer than time Between Eviction Runs Millis, # validation Query is performed to check whether the connection is valid spring.datasource.testWhileIdle=true - #execute validation to check if the connection is valid when applying for a connection spring.datasource.testOnBorrow=true #execute validation to check if the connection is valid when the connection is returned spring.datasource.testOnReturn=false spring.datasource.defaultAutoCommit=true spring.datasource.keepAlive=true - # open PSCache, specify count PSCache for every connection spring.datasource.poolPreparedStatements=true spring.datasource.maxPoolPreparedStatementPerConnectionSize=20 - spring.datasource.spring.datasource.filters=stat,wall,log4j spring.datasource.connectionProperties=druid.stat.mergeSql=true;druid.stat.slowSqlMillis=5000 #mybatis mybatis-plus.mapper-locations=classpath*:/org.apache.dolphinscheduler.dao.mapper/*.xml - mybatis-plus.typeEnumsPackage=org.apache.dolphinscheduler.*.enums - #Entity scan, where multiple packages are separated by a comma or semicolon mybatis-plus.typeAliasesPackage=org.apache.dolphinscheduler.dao.entity - #Primary key type AUTO:" database ID AUTO ", INPUT:" user INPUT ID", ID_WORKER:" global unique ID (numeric type unique ID)", UUID:" global unique ID UUID"; mybatis-plus.global-config.db-config.id-type=AUTO - #Field policy IGNORED:" ignore judgment ",NOT_NULL:" not NULL judgment "),NOT_EMPTY:" not NULL judgment" mybatis-plus.global-config.db-config.field-strategy=NOT_NULL - #The hump underline is converted mybatis-plus.global-config.db-config.column-underline=true mybatis-plus.global-config.db-config.logic-delete-value=-1 @@ -92,12 +79,37 @@ mybatis-plus.configuration.cache-enabled=false mybatis-plus.configuration.call-setters-on-nulls=true mybatis-plus.configuration.jdbc-type-for-null=null +# master settings +# master execute thread num +master.exec.threads=${MASTER_EXEC_THREADS} +# master execute task number in parallel +master.exec.task.num=${MASTER_EXEC_TASK_NUM} +# master heartbeat interval +master.heartbeat.interval=${MASTER_HEARTBEAT_INTERVAL} +# master commit task retry times +master.task.commit.retryTimes=${MASTER_TASK_COMMIT_RETRYTIMES} +# master commit task interval +master.task.commit.interval=${MASTER_TASK_COMMIT_INTERVAL} +# only less than cpu avg load, master server can work. default value : the number of cpu cores * 2 +master.max.cpuload.avg=${MASTER_MAX_CPULOAD_AVG} +# only larger than reserved memory, master server can work. default value : physical memory * 1/10, unit is G. +master.reserved.memory=${MASTER_RESERVED_MEMORY} + +# worker settings +# worker execute thread num +worker.exec.threads=${WORKER_EXEC_THREADS} +# worker heartbeat interval +worker.heartbeat.interval=${WORKER_HEARTBEAT_INTERVAL} +# submit the number of tasks at a time +worker.fetch.task.num=${WORKER_FETCH_TASK_NUM} +# only less than cpu avg load, worker server can work. default value : the number of cpu cores * 2 +worker.max.cpuload.avg=${WORKER_MAX_CPULOAD_AVG} +# only larger than reserved memory, worker server can work. default value : physical memory * 1/6, unit is G. +worker.reserved.memory=${WORKER_RESERVED_MEMORY} + # data quality analysis is not currently in use. please ignore the following configuration -# task record flag +# task record task.record.flag=false task.record.datasource.url=jdbc:mysql://192.168.xx.xx:3306/etl?characterEncoding=UTF-8 task.record.datasource.username=xx task.record.datasource.password=xx - -# Logger Config -#logging.level.org.apache.dolphinscheduler.dao=debug diff --git a/dockerfile/conf/dolphinscheduler/conf/common/common.properties b/dockerfile/conf/dolphinscheduler/common.properties.tpl similarity index 55% rename from dockerfile/conf/dolphinscheduler/conf/common/common.properties rename to dockerfile/conf/dolphinscheduler/common.properties.tpl index 24844f693b..ea03e0b78e 100644 --- a/dockerfile/conf/dolphinscheduler/conf/common/common.properties +++ b/dockerfile/conf/dolphinscheduler/common.properties.tpl @@ -16,44 +16,69 @@ # #task queue implementation, default "zookeeper" -dolphinscheduler.queue.impl=zookeeper +dolphinscheduler.queue.impl=${TASK_QUEUE} -# user data directory path, self configuration, please make sure the directory exists and have read write permissions -data.basedir.path=/tmp/dolphinscheduler +#zookeeper cluster. multiple are separated by commas. eg. 192.168.xx.xx:2181,192.168.xx.xx:2181,192.168.xx.xx:2181 +zookeeper.quorum=${ZOOKEEPER_QUORUM} +#dolphinscheduler root directory +zookeeper.dolphinscheduler.root=/dolphinscheduler +#dolphinscheduler failover directory +zookeeper.session.timeout=300 +zookeeper.connection.timeout=300 +zookeeper.retry.base.sleep=100 +zookeeper.retry.max.sleep=30000 +zookeeper.retry.maxtime=5 -# directory path for user data download. self configuration, please make sure the directory exists and have read write permissions -data.download.basedir.path=/tmp/dolphinscheduler/download +#============================================================================ +# System +#============================================================================ +# system env path. self configuration, please make sure the directory and file exists and have read write execute permissions +dolphinscheduler.env.path=${DOLPHINSCHEDULER_ENV_PATH} +#resource.view.suffixs +resource.view.suffixs=txt,log,sh,conf,cfg,py,java,sql,hql,xml,properties +# is development state? default "false" +development.state=true -# process execute directory. self configuration, please make sure the directory exists and have read write permissions -process.exec.basepath=/tmp/dolphinscheduler/exec +# resource upload startup type : HDFS,S3,NONE +res.upload.startup.type=NONE +#============================================================================ +# HDFS +#============================================================================ # Users who have permission to create directories under the HDFS root path hdfs.root.user=hdfs - # data base dir, resource file will store to this hadoop hdfs path, self configuration, please make sure the directory exists on hdfs and have read write permissions。"/dolphinscheduler" is recommended data.store2hdfs.basepath=/dolphinscheduler - -# resource upload startup type : HDFS,S3,NONE -res.upload.startup.type=NONE - +# user data directory path, self configuration, please make sure the directory exists and have read write permissions +data.basedir.path=/tmp/dolphinscheduler +# directory path for user data download. self configuration, please make sure the directory exists and have read write permissions +data.download.basedir.path=/tmp/dolphinscheduler/download +# process execute directory. self configuration, please make sure the directory exists and have read write permissions +process.exec.basepath=/tmp/dolphinscheduler/exec # whether kerberos starts hadoop.security.authentication.startup.state=false - # java.security.krb5.conf path java.security.krb5.conf.path=/opt/krb5.conf - # loginUserFromKeytab user login.user.keytab.username=hdfs-mycluster@ESZ.COM - # loginUserFromKeytab path login.user.keytab.path=/opt/hdfs.headless.keytab -# system env path. self configuration, please make sure the directory and file exists and have read write execute permissions -dolphinscheduler.env.path=/opt/dolphinscheduler/conf/env/dolphinscheduler_env.sh +#============================================================================ +# S3 +#============================================================================ +# ha or single namenode,If namenode ha needs to copy core-site.xml and hdfs-site.xml +# to the conf directory,support s3,for example : s3a://dolphinscheduler +fs.defaultFS=hdfs://mycluster:8020 +# s3 need,s3 endpoint +fs.s3a.endpoint=http://192.168.199.91:9010 +# s3 need,s3 access key +fs.s3a.access.key=A3DXS30FO22544RE +# s3 need,s3 secret key +fs.s3a.secret.key=OloCLq3n+8+sdPHUhJ21XrSxTC+JK +#resourcemanager ha note this need ips , this empty if single +yarn.resourcemanager.ha.rm.ids=192.168.xx.xx,192.168.xx.xx +# If it is a single resourcemanager, you only need to configure one host name. If it is resourcemanager HA, the default configuration is fine +yarn.application.status.address=http://ark1:8088/ws/v1/cluster/apps/%s -#resource.view.suffixs -resource.view.suffixs=txt,log,sh,conf,cfg,py,java,sql,hql,xml - -# is development state? default "false" -development.state=true diff --git a/dockerfile/conf/dolphinscheduler/conf/alert_logback.xml b/dockerfile/conf/dolphinscheduler/conf/alert_logback.xml deleted file mode 100644 index 35e19865b9..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/alert_logback.xml +++ /dev/null @@ -1,49 +0,0 @@ - - - - - - - - - - [%level] %date{yyyy-MM-dd HH:mm:ss.SSS} %logger{96}:[%line] - %msg%n - - UTF-8 - - - - - ${log.base}/dolphinscheduler-alert.log - - ${log.base}/dolphinscheduler-alert.%d{yyyy-MM-dd_HH}.%i.log - 20 - 64MB - - - - [%level] %date{yyyy-MM-dd HH:mm:ss.SSS} %logger{96}:[%line] - %msg%n - - UTF-8 - - - - - - - \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/apiserver_logback.xml b/dockerfile/conf/dolphinscheduler/conf/apiserver_logback.xml deleted file mode 100644 index 36719671c9..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/apiserver_logback.xml +++ /dev/null @@ -1,60 +0,0 @@ - - - - - - - - - - - - - - - [%level] %date{yyyy-MM-dd HH:mm:ss.SSS} %logger{96}:[%line] - %msg%n - - UTF-8 - - - - - - - INFO - - ${log.base}/dolphinscheduler-api-server.log - - ${log.base}/dolphinscheduler-api-server.%d{yyyy-MM-dd_HH}.%i.log - 168 - 64MB - - - - - [%level] %date{yyyy-MM-dd HH:mm:ss.SSS} %logger{96}:[%line] - %msg%n - - UTF-8 - - - - - - - - \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/combined_logback.xml b/dockerfile/conf/dolphinscheduler/conf/combined_logback.xml deleted file mode 100644 index 7a9a5b4621..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/combined_logback.xml +++ /dev/null @@ -1,80 +0,0 @@ - - - - - - - - - - %highlight([%level]) %date{yyyy-MM-dd HH:mm:ss.SSS} %logger{10}:[%line] - %msg%n - - UTF-8 - - - - - INFO - - - - taskAppId - ${log.base} - - - - ${log.base}/${taskAppId}.log - - - [%level] %date{yyyy-MM-dd HH:mm:ss.SSS} %logger{96}:[%line] - %msg%n - - UTF-8 - - true - - - - - - ${log.base}/dolphinscheduler-combined.log - - INFO - - - - ${log.base}/dolphinscheduler-combined.%d{yyyy-MM-dd_HH}.%i.log - 168 - 200MB - -       - - - [%level] %date{yyyy-MM-dd HH:mm:ss.SSS} %logger{96}:[%line] - %msg%n - - UTF-8 - -    - - - - - - - - - \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/common/hadoop/hadoop.properties b/dockerfile/conf/dolphinscheduler/conf/common/hadoop/hadoop.properties deleted file mode 100644 index 2c19b4a52e..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/common/hadoop/hadoop.properties +++ /dev/null @@ -1,35 +0,0 @@ -# -# 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. -# - -# ha or single namenode,If namenode ha needs to copy core-site.xml and hdfs-site.xml -# to the conf directory,support s3,for example : s3a://dolphinscheduler -fs.defaultFS=hdfs://mycluster:8020 - -# s3 need,s3 endpoint -fs.s3a.endpoint=http://192.168.199.91:9010 - -# s3 need,s3 access key -fs.s3a.access.key=A3DXS30FO22544RE - -# s3 need,s3 secret key -fs.s3a.secret.key=OloCLq3n+8+sdPHUhJ21XrSxTC+JK - -#resourcemanager ha note this need ips , this empty if single -yarn.resourcemanager.ha.rm.ids=192.168.xx.xx,192.168.xx.xx - -# If it is a single resourcemanager, you only need to configure one host name. If it is resourcemanager HA, the default configuration is fine -yarn.application.status.address=http://ark1:8088/ws/v1/cluster/apps/%s \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/config/install_config.conf b/dockerfile/conf/dolphinscheduler/conf/config/install_config.conf deleted file mode 100644 index 196a78f49c..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/config/install_config.conf +++ /dev/null @@ -1,20 +0,0 @@ -# -# 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. -# - -installPath=/data1_1T/dolphinscheduler -deployUser=dolphinscheduler -ips=ark0,ark1,ark2,ark3,ark4 diff --git a/dockerfile/conf/dolphinscheduler/conf/env/.dolphinscheduler_env.sh b/dockerfile/conf/dolphinscheduler/conf/env/.dolphinscheduler_env.sh deleted file mode 100644 index 8e842fe28e..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/env/.dolphinscheduler_env.sh +++ /dev/null @@ -1,21 +0,0 @@ -# -# 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. -# - -export PYTHON_HOME=/usr/bin/python -export JAVA_HOME=/usr/lib/jvm/java-8-openjdk-amd64 -export PATH=$PYTHON_HOME:$JAVA_HOME/bin:$PATH -export DATAX_HOME=/opt/datax/bin/datax.py \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/env/.escheduler_env.sh b/dockerfile/conf/dolphinscheduler/conf/env/.escheduler_env.sh deleted file mode 100644 index 5b85917fc2..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/env/.escheduler_env.sh +++ /dev/null @@ -1,20 +0,0 @@ -# -# 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. -# - -export PYTHON_HOME=/usr/bin/python -export JAVA_HOME=/usr/lib/jvm/java-8-openjdk-amd64 -export PATH=$PYTHON_HOME:$JAVA_HOME/bin:$PATH \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/i18n/messages.properties b/dockerfile/conf/dolphinscheduler/conf/i18n/messages.properties deleted file mode 100644 index be880ba26d..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/i18n/messages.properties +++ /dev/null @@ -1,252 +0,0 @@ -# -# 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. -# - -QUERY_SCHEDULE_LIST_NOTES=query schedule list -EXECUTE_PROCESS_TAG=execute process related operation -PROCESS_INSTANCE_EXECUTOR_TAG=process instance executor related operation -RUN_PROCESS_INSTANCE_NOTES=run process instance -START_NODE_LIST=start node list(node name) -TASK_DEPEND_TYPE=task depend type -COMMAND_TYPE=command type -RUN_MODE=run mode -TIMEOUT=timeout -EXECUTE_ACTION_TO_PROCESS_INSTANCE_NOTES=execute action to process instance -EXECUTE_TYPE=execute type -START_CHECK_PROCESS_DEFINITION_NOTES=start check process definition -GET_RECEIVER_CC_NOTES=query receiver cc -DESC=description -GROUP_NAME=group name -GROUP_TYPE=group type -QUERY_ALERT_GROUP_LIST_NOTES=query alert group list -UPDATE_ALERT_GROUP_NOTES=update alert group -DELETE_ALERT_GROUP_BY_ID_NOTES=delete alert group by id -VERIFY_ALERT_GROUP_NAME_NOTES=verify alert group name, check alert group exist or not -GRANT_ALERT_GROUP_NOTES=grant alert group -USER_IDS=user id list -ALERT_GROUP_TAG=alert group related operation -CREATE_ALERT_GROUP_NOTES=create alert group -WORKER_GROUP_TAG=worker group related operation -SAVE_WORKER_GROUP_NOTES=create worker group -WORKER_GROUP_NAME=worker group name -WORKER_IP_LIST=worker ip list, eg. 192.168.1.1,192.168.1.2 -QUERY_WORKER_GROUP_PAGING_NOTES=query worker group paging -QUERY_WORKER_GROUP_LIST_NOTES=query worker group list -DELETE_WORKER_GROUP_BY_ID_NOTES=delete worker group by id -DATA_ANALYSIS_TAG=analysis related operation of task state -COUNT_TASK_STATE_NOTES=count task state -COUNT_PROCESS_INSTANCE_NOTES=count process instance state -COUNT_PROCESS_DEFINITION_BY_USER_NOTES=count process definition by user -COUNT_COMMAND_STATE_NOTES=count command state -COUNT_QUEUE_STATE_NOTES=count the running status of the task in the queue\ - -ACCESS_TOKEN_TAG=access token related operation -MONITOR_TAG=monitor related operation -MASTER_LIST_NOTES=master server list -WORKER_LIST_NOTES=worker server list -QUERY_DATABASE_STATE_NOTES=query database state -QUERY_ZOOKEEPER_STATE_NOTES=QUERY ZOOKEEPER STATE -TASK_STATE=task instance state -SOURCE_TABLE=SOURCE TABLE -DEST_TABLE=dest table -TASK_DATE=task date -QUERY_HISTORY_TASK_RECORD_LIST_PAGING_NOTES=query history task record list paging -DATA_SOURCE_TAG=data source related operation -CREATE_DATA_SOURCE_NOTES=create data source -DATA_SOURCE_NAME=data source name -DATA_SOURCE_NOTE=data source desc -DB_TYPE=database type -DATA_SOURCE_HOST=DATA SOURCE HOST -DATA_SOURCE_PORT=data source port -DATABASE_NAME=database name -QUEUE_TAG=queue related operation -QUERY_QUEUE_LIST_NOTES=query queue list -QUERY_QUEUE_LIST_PAGING_NOTES=query queue list paging -CREATE_QUEUE_NOTES=create queue -YARN_QUEUE_NAME=yarn(hadoop) queue name -QUEUE_ID=queue id -TENANT_DESC=tenant desc -QUERY_TENANT_LIST_PAGING_NOTES=query tenant list paging -QUERY_TENANT_LIST_NOTES=query tenant list -UPDATE_TENANT_NOTES=update tenant -DELETE_TENANT_NOTES=delete tenant -RESOURCES_TAG=resource center related operation -CREATE_RESOURCE_NOTES=create resource -RESOURCE_TYPE=resource file type -RESOURCE_NAME=resource name -RESOURCE_DESC=resource file desc -RESOURCE_FILE=resource file -RESOURCE_ID=resource id -QUERY_RESOURCE_LIST_NOTES=query resource list -DELETE_RESOURCE_BY_ID_NOTES=delete resource by id -VIEW_RESOURCE_BY_ID_NOTES=view resource by id -ONLINE_CREATE_RESOURCE_NOTES=online create resource -SUFFIX=resource file suffix -CONTENT=resource file content -UPDATE_RESOURCE_NOTES=edit resource file online -DOWNLOAD_RESOURCE_NOTES=download resource file -CREATE_UDF_FUNCTION_NOTES=create udf function -UDF_TYPE=UDF type -FUNC_NAME=function name -CLASS_NAME=package and class name -ARG_TYPES=arguments -UDF_DESC=udf desc -VIEW_UDF_FUNCTION_NOTES=view udf function -UPDATE_UDF_FUNCTION_NOTES=update udf function -QUERY_UDF_FUNCTION_LIST_PAGING_NOTES=query udf function list paging -VERIFY_UDF_FUNCTION_NAME_NOTES=verify udf function name -DELETE_UDF_FUNCTION_NOTES=delete udf function -AUTHORIZED_FILE_NOTES=authorized file -UNAUTHORIZED_FILE_NOTES=unauthorized file -AUTHORIZED_UDF_FUNC_NOTES=authorized udf func -UNAUTHORIZED_UDF_FUNC_NOTES=unauthorized udf func -VERIFY_QUEUE_NOTES=verify queue -TENANT_TAG=tenant related operation -CREATE_TENANT_NOTES=create tenant -TENANT_CODE=tenant code -TENANT_NAME=tenant name -QUEUE_NAME=queue name -PASSWORD=password -DATA_SOURCE_OTHER=jdbc connection params, format:{"key1":"value1",...} -PROJECT_TAG=project related operation -CREATE_PROJECT_NOTES=create project -PROJECT_DESC=project description -UPDATE_PROJECT_NOTES=update project -PROJECT_ID=project id -QUERY_PROJECT_BY_ID_NOTES=query project info by project id -QUERY_PROJECT_LIST_PAGING_NOTES=QUERY PROJECT LIST PAGING -DELETE_PROJECT_BY_ID_NOTES=delete project by id -QUERY_UNAUTHORIZED_PROJECT_NOTES=query unauthorized project -QUERY_ALL_PROJECT_LIST_NOTES=query all project list -QUERY_AUTHORIZED_PROJECT_NOTES=query authorized project -TASK_RECORD_TAG=task record related operation -QUERY_TASK_RECORD_LIST_PAGING_NOTES=query task record list paging -CREATE_TOKEN_NOTES=create token ,note: please login first -QUERY_ACCESS_TOKEN_LIST_NOTES=query access token list paging -SCHEDULE=schedule -WARNING_TYPE=warning type(sending strategy) -WARNING_GROUP_ID=warning group id -FAILURE_STRATEGY=failure strategy -RECEIVERS=receivers -RECEIVERS_CC=receivers cc -WORKER_GROUP_ID=worker server group id -PROCESS_INSTANCE_PRIORITY=process instance priority -UPDATE_SCHEDULE_NOTES=update schedule -SCHEDULE_ID=schedule id -ONLINE_SCHEDULE_NOTES=online schedule -OFFLINE_SCHEDULE_NOTES=offline schedule -QUERY_SCHEDULE_NOTES=query schedule -QUERY_SCHEDULE_LIST_PAGING_NOTES=query schedule list paging -LOGIN_TAG=User login related operations -USER_NAME=user name -PROJECT_NAME=project name -CREATE_PROCESS_DEFINITION_NOTES=create process definition -PROCESS_DEFINITION_NAME=process definition name -PROCESS_DEFINITION_JSON=process definition detail info (json format) -PROCESS_DEFINITION_LOCATIONS=process definition node locations info (json format) -PROCESS_INSTANCE_LOCATIONS=process instance node locations info (json format) -PROCESS_DEFINITION_CONNECTS=process definition node connects info (json format) -PROCESS_INSTANCE_CONNECTS=process instance node connects info (json format) -PROCESS_DEFINITION_DESC=process definition desc -PROCESS_DEFINITION_TAG=process definition related opertation -SIGNOUT_NOTES=logout -USER_PASSWORD=user password -UPDATE_PROCESS_INSTANCE_NOTES=update process instance -QUERY_PROCESS_INSTANCE_LIST_NOTES=query process instance list -VERIFY_PROCCESS_DEFINITION_NAME_NOTES=verify proccess definition name -LOGIN_NOTES=user login -UPDATE_PROCCESS_DEFINITION_NOTES=update proccess definition -PROCESS_DEFINITION_ID=process definition id -PROCESS_DEFINITION_IDS=process definition ids -RELEASE_PROCCESS_DEFINITION_NOTES=release proccess definition -QUERY_PROCCESS_DEFINITION_BY_ID_NOTES=query proccess definition by id -QUERY_PROCCESS_DEFINITION_LIST_NOTES=query proccess definition list -QUERY_PROCCESS_DEFINITION_LIST_PAGING_NOTES=query proccess definition list paging -QUERY_ALL_DEFINITION_LIST_NOTES=query all definition list -PAGE_NO=page no -PROCESS_INSTANCE_ID=process instance id -PROCESS_INSTANCE_JSON=process instance info(json format) -SCHEDULE_TIME=schedule time -SYNC_DEFINE=update the information of the process instance to the process definition\ - -RECOVERY_PROCESS_INSTANCE_FLAG=whether to recovery process instance -SEARCH_VAL=search val -USER_ID=user id -PAGE_SIZE=page size -LIMIT=limit -VIEW_TREE_NOTES=view tree -GET_NODE_LIST_BY_DEFINITION_ID_NOTES=get task node list by process definition id -PROCESS_DEFINITION_ID_LIST=process definition id list -QUERY_PROCCESS_DEFINITION_All_BY_PROJECT_ID_NOTES=query proccess definition all by project id -DELETE_PROCESS_DEFINITION_BY_ID_NOTES=delete process definition by process definition id -BATCH_DELETE_PROCESS_DEFINITION_BY_IDS_NOTES=batch delete process definition by process definition ids -QUERY_PROCESS_INSTANCE_BY_ID_NOTES=query process instance by process instance id -DELETE_PROCESS_INSTANCE_BY_ID_NOTES=delete process instance by process instance id -TASK_ID=task instance id -SKIP_LINE_NUM=skip line num -QUERY_TASK_INSTANCE_LOG_NOTES=query task instance log -DOWNLOAD_TASK_INSTANCE_LOG_NOTES=download task instance log -USERS_TAG=users related operation -SCHEDULER_TAG=scheduler related operation -CREATE_SCHEDULE_NOTES=create schedule -CREATE_USER_NOTES=create user -TENANT_ID=tenant id -QUEUE=queue -EMAIL=email -PHONE=phone -QUERY_USER_LIST_NOTES=query user list -UPDATE_USER_NOTES=update user -DELETE_USER_BY_ID_NOTES=delete user by id -GRANT_PROJECT_NOTES=GRANT PROJECT -PROJECT_IDS=project ids(string format, multiple projects separated by ",") -GRANT_RESOURCE_NOTES=grant resource file -RESOURCE_IDS=resource ids(string format, multiple resources separated by ",") -GET_USER_INFO_NOTES=get user info -LIST_USER_NOTES=list user -VERIFY_USER_NAME_NOTES=verify user name -UNAUTHORIZED_USER_NOTES=cancel authorization -ALERT_GROUP_ID=alert group id -AUTHORIZED_USER_NOTES=authorized user -GRANT_UDF_FUNC_NOTES=grant udf function -UDF_IDS=udf ids(string format, multiple udf functions separated by ",") -GRANT_DATASOURCE_NOTES=grant datasource -DATASOURCE_IDS=datasource ids(string format, multiple datasources separated by ",") -QUERY_SUBPROCESS_INSTANCE_BY_TASK_ID_NOTES=query subprocess instance by task instance id -QUERY_PARENT_PROCESS_INSTANCE_BY_SUB_PROCESS_INSTANCE_ID_NOTES=query parent process instance info by sub process instance id -QUERY_PROCESS_INSTANCE_GLOBAL_VARIABLES_AND_LOCAL_VARIABLES_NOTES=query process instance global variables and local variables -VIEW_GANTT_NOTES=view gantt -SUB_PROCESS_INSTANCE_ID=sub process instance id -TASK_NAME=task instance name -TASK_INSTANCE_TAG=task instance related operation -LOGGER_TAG=log related operation -PROCESS_INSTANCE_TAG=process instance related operation -EXECUTION_STATUS=runing status for workflow and task nodes -HOST=ip address of running task -START_DATE=start date -END_DATE=end date -QUERY_TASK_LIST_BY_PROCESS_INSTANCE_ID_NOTES=query task list by process instance id -UPDATE_DATA_SOURCE_NOTES=update data source -DATA_SOURCE_ID=DATA SOURCE ID -QUERY_DATA_SOURCE_NOTES=query data source by id -QUERY_DATA_SOURCE_LIST_BY_TYPE_NOTES=query data source list by database type -QUERY_DATA_SOURCE_LIST_PAGING_NOTES=query data source list paging -CONNECT_DATA_SOURCE_NOTES=CONNECT DATA SOURCE -CONNECT_DATA_SOURCE_TEST_NOTES=connect data source test -DELETE_DATA_SOURCE_NOTES=delete data source -VERIFY_DATA_SOURCE_NOTES=verify data source -UNAUTHORIZED_DATA_SOURCE_NOTES=unauthorized data source -AUTHORIZED_DATA_SOURCE_NOTES=authorized data source -DELETE_SCHEDULER_BY_ID_NOTES=delete scheduler by id diff --git a/dockerfile/conf/dolphinscheduler/conf/i18n/messages_en_US.properties b/dockerfile/conf/dolphinscheduler/conf/i18n/messages_en_US.properties deleted file mode 100644 index 24c0843c10..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/i18n/messages_en_US.properties +++ /dev/null @@ -1,252 +0,0 @@ -# -# 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. -# - -QUERY_SCHEDULE_LIST_NOTES=query schedule list -EXECUTE_PROCESS_TAG=execute process related operation -PROCESS_INSTANCE_EXECUTOR_TAG=process instance executor related operation -RUN_PROCESS_INSTANCE_NOTES=run process instance -START_NODE_LIST=start node list(node name) -TASK_DEPEND_TYPE=task depend type -COMMAND_TYPE=command type -RUN_MODE=run mode -TIMEOUT=timeout -EXECUTE_ACTION_TO_PROCESS_INSTANCE_NOTES=execute action to process instance -EXECUTE_TYPE=execute type -START_CHECK_PROCESS_DEFINITION_NOTES=start check process definition -GET_RECEIVER_CC_NOTES=query receiver cc -DESC=description -GROUP_NAME=group name -GROUP_TYPE=group type -QUERY_ALERT_GROUP_LIST_NOTES=query alert group list -UPDATE_ALERT_GROUP_NOTES=update alert group -DELETE_ALERT_GROUP_BY_ID_NOTES=delete alert group by id -VERIFY_ALERT_GROUP_NAME_NOTES=verify alert group name, check alert group exist or not -GRANT_ALERT_GROUP_NOTES=grant alert group -USER_IDS=user id list -ALERT_GROUP_TAG=alert group related operation -CREATE_ALERT_GROUP_NOTES=create alert group -WORKER_GROUP_TAG=worker group related operation -SAVE_WORKER_GROUP_NOTES=create worker group -WORKER_GROUP_NAME=worker group name -WORKER_IP_LIST=worker ip list, eg. 192.168.1.1,192.168.1.2 -QUERY_WORKER_GROUP_PAGING_NOTES=query worker group paging -QUERY_WORKER_GROUP_LIST_NOTES=query worker group list -DELETE_WORKER_GROUP_BY_ID_NOTES=delete worker group by id -DATA_ANALYSIS_TAG=analysis related operation of task state -COUNT_TASK_STATE_NOTES=count task state -COUNT_PROCESS_INSTANCE_NOTES=count process instance state -COUNT_PROCESS_DEFINITION_BY_USER_NOTES=count process definition by user -COUNT_COMMAND_STATE_NOTES=count command state -COUNT_QUEUE_STATE_NOTES=count the running status of the task in the queue\ - -ACCESS_TOKEN_TAG=access token related operation -MONITOR_TAG=monitor related operation -MASTER_LIST_NOTES=master server list -WORKER_LIST_NOTES=worker server list -QUERY_DATABASE_STATE_NOTES=query database state -QUERY_ZOOKEEPER_STATE_NOTES=QUERY ZOOKEEPER STATE -TASK_STATE=task instance state -SOURCE_TABLE=SOURCE TABLE -DEST_TABLE=dest table -TASK_DATE=task date -QUERY_HISTORY_TASK_RECORD_LIST_PAGING_NOTES=query history task record list paging -DATA_SOURCE_TAG=data source related operation -CREATE_DATA_SOURCE_NOTES=create data source -DATA_SOURCE_NAME=data source name -DATA_SOURCE_NOTE=data source desc -DB_TYPE=database type -DATA_SOURCE_HOST=DATA SOURCE HOST -DATA_SOURCE_PORT=data source port -DATABASE_NAME=database name -QUEUE_TAG=queue related operation -QUERY_QUEUE_LIST_NOTES=query queue list -QUERY_QUEUE_LIST_PAGING_NOTES=query queue list paging -CREATE_QUEUE_NOTES=create queue -YARN_QUEUE_NAME=yarn(hadoop) queue name -QUEUE_ID=queue id -TENANT_DESC=tenant desc -QUERY_TENANT_LIST_PAGING_NOTES=query tenant list paging -QUERY_TENANT_LIST_NOTES=query tenant list -UPDATE_TENANT_NOTES=update tenant -DELETE_TENANT_NOTES=delete tenant -RESOURCES_TAG=resource center related operation -CREATE_RESOURCE_NOTES=create resource -RESOURCE_TYPE=resource file type -RESOURCE_NAME=resource name -RESOURCE_DESC=resource file desc -RESOURCE_FILE=resource file -RESOURCE_ID=resource id -QUERY_RESOURCE_LIST_NOTES=query resource list -DELETE_RESOURCE_BY_ID_NOTES=delete resource by id -VIEW_RESOURCE_BY_ID_NOTES=view resource by id -ONLINE_CREATE_RESOURCE_NOTES=online create resource -SUFFIX=resource file suffix -CONTENT=resource file content -UPDATE_RESOURCE_NOTES=edit resource file online -DOWNLOAD_RESOURCE_NOTES=download resource file -CREATE_UDF_FUNCTION_NOTES=create udf function -UDF_TYPE=UDF type -FUNC_NAME=function name -CLASS_NAME=package and class name -ARG_TYPES=arguments -UDF_DESC=udf desc -VIEW_UDF_FUNCTION_NOTES=view udf function -UPDATE_UDF_FUNCTION_NOTES=update udf function -QUERY_UDF_FUNCTION_LIST_PAGING_NOTES=query udf function list paging -VERIFY_UDF_FUNCTION_NAME_NOTES=verify udf function name -DELETE_UDF_FUNCTION_NOTES=delete udf function -AUTHORIZED_FILE_NOTES=authorized file -UNAUTHORIZED_FILE_NOTES=unauthorized file -AUTHORIZED_UDF_FUNC_NOTES=authorized udf func -UNAUTHORIZED_UDF_FUNC_NOTES=unauthorized udf func -VERIFY_QUEUE_NOTES=verify queue -TENANT_TAG=tenant related operation -CREATE_TENANT_NOTES=create tenant -TENANT_CODE=tenant code -TENANT_NAME=tenant name -QUEUE_NAME=queue name -PASSWORD=password -DATA_SOURCE_OTHER=jdbc connection params, format:{"key1":"value1",...} -PROJECT_TAG=project related operation -CREATE_PROJECT_NOTES=create project -PROJECT_DESC=project description -UPDATE_PROJECT_NOTES=update project -PROJECT_ID=project id -QUERY_PROJECT_BY_ID_NOTES=query project info by project id -QUERY_PROJECT_LIST_PAGING_NOTES=QUERY PROJECT LIST PAGING -QUERY_ALL_PROJECT_LIST_NOTES=query all project list -DELETE_PROJECT_BY_ID_NOTES=delete project by id -QUERY_UNAUTHORIZED_PROJECT_NOTES=query unauthorized project -QUERY_AUTHORIZED_PROJECT_NOTES=query authorized project -TASK_RECORD_TAG=task record related operation -QUERY_TASK_RECORD_LIST_PAGING_NOTES=query task record list paging -CREATE_TOKEN_NOTES=create token ,note: please login first -QUERY_ACCESS_TOKEN_LIST_NOTES=query access token list paging -SCHEDULE=schedule -WARNING_TYPE=warning type(sending strategy) -WARNING_GROUP_ID=warning group id -FAILURE_STRATEGY=failure strategy -RECEIVERS=receivers -RECEIVERS_CC=receivers cc -WORKER_GROUP_ID=worker server group id -PROCESS_INSTANCE_PRIORITY=process instance priority -UPDATE_SCHEDULE_NOTES=update schedule -SCHEDULE_ID=schedule id -ONLINE_SCHEDULE_NOTES=online schedule -OFFLINE_SCHEDULE_NOTES=offline schedule -QUERY_SCHEDULE_NOTES=query schedule -QUERY_SCHEDULE_LIST_PAGING_NOTES=query schedule list paging -LOGIN_TAG=User login related operations -USER_NAME=user name -PROJECT_NAME=project name -CREATE_PROCESS_DEFINITION_NOTES=create process definition -PROCESS_DEFINITION_NAME=process definition name -PROCESS_DEFINITION_JSON=process definition detail info (json format) -PROCESS_DEFINITION_LOCATIONS=process definition node locations info (json format) -PROCESS_INSTANCE_LOCATIONS=process instance node locations info (json format) -PROCESS_DEFINITION_CONNECTS=process definition node connects info (json format) -PROCESS_INSTANCE_CONNECTS=process instance node connects info (json format) -PROCESS_DEFINITION_DESC=process definition desc -PROCESS_DEFINITION_TAG=process definition related opertation -SIGNOUT_NOTES=logout -USER_PASSWORD=user password -UPDATE_PROCESS_INSTANCE_NOTES=update process instance -QUERY_PROCESS_INSTANCE_LIST_NOTES=query process instance list -VERIFY_PROCCESS_DEFINITION_NAME_NOTES=verify proccess definition name -LOGIN_NOTES=user login -UPDATE_PROCCESS_DEFINITION_NOTES=update proccess definition -PROCESS_DEFINITION_ID=process definition id -PROCESS_DEFINITION_IDS=process definition ids -RELEASE_PROCCESS_DEFINITION_NOTES=release proccess definition -QUERY_PROCCESS_DEFINITION_BY_ID_NOTES=query proccess definition by id -QUERY_PROCCESS_DEFINITION_LIST_NOTES=query proccess definition list -QUERY_PROCCESS_DEFINITION_LIST_PAGING_NOTES=query proccess definition list paging -QUERY_ALL_DEFINITION_LIST_NOTES=query all definition list -PAGE_NO=page no -PROCESS_INSTANCE_ID=process instance id -PROCESS_INSTANCE_JSON=process instance info(json format) -SCHEDULE_TIME=schedule time -SYNC_DEFINE=update the information of the process instance to the process definition\ - -RECOVERY_PROCESS_INSTANCE_FLAG=whether to recovery process instance -SEARCH_VAL=search val -USER_ID=user id -PAGE_SIZE=page size -LIMIT=limit -VIEW_TREE_NOTES=view tree -GET_NODE_LIST_BY_DEFINITION_ID_NOTES=get task node list by process definition id -PROCESS_DEFINITION_ID_LIST=process definition id list -QUERY_PROCCESS_DEFINITION_All_BY_PROJECT_ID_NOTES=query proccess definition all by project id -DELETE_PROCESS_DEFINITION_BY_ID_NOTES=delete process definition by process definition id -BATCH_DELETE_PROCESS_DEFINITION_BY_IDS_NOTES=batch delete process definition by process definition ids -QUERY_PROCESS_INSTANCE_BY_ID_NOTES=query process instance by process instance id -DELETE_PROCESS_INSTANCE_BY_ID_NOTES=delete process instance by process instance id -TASK_ID=task instance id -SKIP_LINE_NUM=skip line num -QUERY_TASK_INSTANCE_LOG_NOTES=query task instance log -DOWNLOAD_TASK_INSTANCE_LOG_NOTES=download task instance log -USERS_TAG=users related operation -SCHEDULER_TAG=scheduler related operation -CREATE_SCHEDULE_NOTES=create schedule -CREATE_USER_NOTES=create user -TENANT_ID=tenant id -QUEUE=queue -EMAIL=email -PHONE=phone -QUERY_USER_LIST_NOTES=query user list -UPDATE_USER_NOTES=update user -DELETE_USER_BY_ID_NOTES=delete user by id -GRANT_PROJECT_NOTES=GRANT PROJECT -PROJECT_IDS=project ids(string format, multiple projects separated by ",") -GRANT_RESOURCE_NOTES=grant resource file -RESOURCE_IDS=resource ids(string format, multiple resources separated by ",") -GET_USER_INFO_NOTES=get user info -LIST_USER_NOTES=list user -VERIFY_USER_NAME_NOTES=verify user name -UNAUTHORIZED_USER_NOTES=cancel authorization -ALERT_GROUP_ID=alert group id -AUTHORIZED_USER_NOTES=authorized user -GRANT_UDF_FUNC_NOTES=grant udf function -UDF_IDS=udf ids(string format, multiple udf functions separated by ",") -GRANT_DATASOURCE_NOTES=grant datasource -DATASOURCE_IDS=datasource ids(string format, multiple datasources separated by ",") -QUERY_SUBPROCESS_INSTANCE_BY_TASK_ID_NOTES=query subprocess instance by task instance id -QUERY_PARENT_PROCESS_INSTANCE_BY_SUB_PROCESS_INSTANCE_ID_NOTES=query parent process instance info by sub process instance id -QUERY_PROCESS_INSTANCE_GLOBAL_VARIABLES_AND_LOCAL_VARIABLES_NOTES=query process instance global variables and local variables -VIEW_GANTT_NOTES=view gantt -SUB_PROCESS_INSTANCE_ID=sub process instance id -TASK_NAME=task instance name -TASK_INSTANCE_TAG=task instance related operation -LOGGER_TAG=log related operation -PROCESS_INSTANCE_TAG=process instance related operation -EXECUTION_STATUS=runing status for workflow and task nodes -HOST=ip address of running task -START_DATE=start date -END_DATE=end date -QUERY_TASK_LIST_BY_PROCESS_INSTANCE_ID_NOTES=query task list by process instance id -UPDATE_DATA_SOURCE_NOTES=update data source -DATA_SOURCE_ID=DATA SOURCE ID -QUERY_DATA_SOURCE_NOTES=query data source by id -QUERY_DATA_SOURCE_LIST_BY_TYPE_NOTES=query data source list by database type -QUERY_DATA_SOURCE_LIST_PAGING_NOTES=query data source list paging -CONNECT_DATA_SOURCE_NOTES=CONNECT DATA SOURCE -CONNECT_DATA_SOURCE_TEST_NOTES=connect data source test -DELETE_DATA_SOURCE_NOTES=delete data source -VERIFY_DATA_SOURCE_NOTES=verify data source -UNAUTHORIZED_DATA_SOURCE_NOTES=unauthorized data source -AUTHORIZED_DATA_SOURCE_NOTES=authorized data source -DELETE_SCHEDULER_BY_ID_NOTES=delete scheduler by id diff --git a/dockerfile/conf/dolphinscheduler/conf/i18n/messages_zh_CN.properties b/dockerfile/conf/dolphinscheduler/conf/i18n/messages_zh_CN.properties deleted file mode 100644 index 5f24a6fedd..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/i18n/messages_zh_CN.properties +++ /dev/null @@ -1,250 +0,0 @@ -# -# 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. -# - -QUERY_SCHEDULE_LIST_NOTES=查询定时列表 -PROCESS_INSTANCE_EXECUTOR_TAG=流程实例执行相关操作 -RUN_PROCESS_INSTANCE_NOTES=运行流程实例 -START_NODE_LIST=开始节点列表(节点name) -TASK_DEPEND_TYPE=任务依赖类型 -COMMAND_TYPE=指令类型 -RUN_MODE=运行模式 -TIMEOUT=超时时间 -EXECUTE_ACTION_TO_PROCESS_INSTANCE_NOTES=执行流程实例的各种操作(暂停、停止、重跑、恢复等) -EXECUTE_TYPE=执行类型 -START_CHECK_PROCESS_DEFINITION_NOTES=检查流程定义 -DESC=备注(描述) -GROUP_NAME=组名称 -GROUP_TYPE=组类型 -QUERY_ALERT_GROUP_LIST_NOTES=告警组列表\ - -UPDATE_ALERT_GROUP_NOTES=编辑(更新)告警组 -DELETE_ALERT_GROUP_BY_ID_NOTES=删除告警组通过ID -VERIFY_ALERT_GROUP_NAME_NOTES=检查告警组是否存在 -GRANT_ALERT_GROUP_NOTES=授权告警组 -USER_IDS=用户ID列表 -ALERT_GROUP_TAG=告警组相关操作 -WORKER_GROUP_TAG=Worker分组管理 -SAVE_WORKER_GROUP_NOTES=创建Worker分组\ - -WORKER_GROUP_NAME=Worker分组名称 -WORKER_IP_LIST=Worker ip列表,注意:多个IP地址以逗号分割\ - -QUERY_WORKER_GROUP_PAGING_NOTES=Worker分组管理 -QUERY_WORKER_GROUP_LIST_NOTES=查询worker group分组 -DELETE_WORKER_GROUP_BY_ID_NOTES=删除worker group通过ID -DATA_ANALYSIS_TAG=任务状态分析相关操作 -COUNT_TASK_STATE_NOTES=任务状态统计 -COUNT_PROCESS_INSTANCE_NOTES=统计流程实例状态 -COUNT_PROCESS_DEFINITION_BY_USER_NOTES=统计用户创建的流程定义 -COUNT_COMMAND_STATE_NOTES=统计命令状态 -COUNT_QUEUE_STATE_NOTES=统计队列里任务状态 -ACCESS_TOKEN_TAG=access token相关操作,需要先登录 -MONITOR_TAG=监控相关操作 -MASTER_LIST_NOTES=master服务列表 -WORKER_LIST_NOTES=worker服务列表 -QUERY_DATABASE_STATE_NOTES=查询数据库状态 -QUERY_ZOOKEEPER_STATE_NOTES=查询Zookeeper状态 -TASK_STATE=任务实例状态 -SOURCE_TABLE=源表 -DEST_TABLE=目标表 -TASK_DATE=任务时间 -QUERY_HISTORY_TASK_RECORD_LIST_PAGING_NOTES=分页查询历史任务记录列表 -DATA_SOURCE_TAG=数据源相关操作 -CREATE_DATA_SOURCE_NOTES=创建数据源 -DATA_SOURCE_NAME=数据源名称 -DATA_SOURCE_NOTE=数据源描述 -DB_TYPE=数据源类型 -DATA_SOURCE_HOST=IP主机名 -DATA_SOURCE_PORT=数据源端口 -DATABASE_NAME=数据库名 -QUEUE_TAG=队列相关操作 -QUERY_QUEUE_LIST_NOTES=查询队列列表 -QUERY_QUEUE_LIST_PAGING_NOTES=分页查询队列列表 -CREATE_QUEUE_NOTES=创建队列 -YARN_QUEUE_NAME=hadoop yarn队列名 -QUEUE_ID=队列ID -TENANT_DESC=租户描述 -QUERY_TENANT_LIST_PAGING_NOTES=分页查询租户列表 -QUERY_TENANT_LIST_NOTES=查询租户列表 -UPDATE_TENANT_NOTES=更新租户 -DELETE_TENANT_NOTES=删除租户 -RESOURCES_TAG=资源中心相关操作 -CREATE_RESOURCE_NOTES=创建资源 -RESOURCE_TYPE=资源文件类型 -RESOURCE_NAME=资源文件名称 -RESOURCE_DESC=资源文件描述 -RESOURCE_FILE=资源文件 -RESOURCE_ID=资源ID -QUERY_RESOURCE_LIST_NOTES=查询资源列表 -DELETE_RESOURCE_BY_ID_NOTES=删除资源通过ID -VIEW_RESOURCE_BY_ID_NOTES=浏览资源通通过ID -ONLINE_CREATE_RESOURCE_NOTES=在线创建资源 -SUFFIX=资源文件后缀 -CONTENT=资源文件内容 -UPDATE_RESOURCE_NOTES=在线更新资源文件 -DOWNLOAD_RESOURCE_NOTES=下载资源文件 -CREATE_UDF_FUNCTION_NOTES=创建UDF函数 -UDF_TYPE=UDF类型 -FUNC_NAME=函数名称 -CLASS_NAME=包名类名 -ARG_TYPES=参数 -UDF_DESC=udf描述,使用说明 -VIEW_UDF_FUNCTION_NOTES=查看udf函数 -UPDATE_UDF_FUNCTION_NOTES=更新udf函数 -QUERY_UDF_FUNCTION_LIST_PAGING_NOTES=分页查询udf函数列表 -VERIFY_UDF_FUNCTION_NAME_NOTES=验证udf函数名 -DELETE_UDF_FUNCTION_NOTES=删除UDF函数 -AUTHORIZED_FILE_NOTES=授权文件 -UNAUTHORIZED_FILE_NOTES=取消授权文件 -AUTHORIZED_UDF_FUNC_NOTES=授权udf函数 -UNAUTHORIZED_UDF_FUNC_NOTES=取消udf函数授权 -VERIFY_QUEUE_NOTES=验证队列 -TENANT_TAG=租户相关操作 -CREATE_TENANT_NOTES=创建租户 -TENANT_CODE=租户编码 -TENANT_NAME=租户名称 -QUEUE_NAME=队列名 -PASSWORD=密码 -DATA_SOURCE_OTHER=jdbc连接参数,格式为:{"key1":"value1",...} -PROJECT_TAG=项目相关操作 -CREATE_PROJECT_NOTES=创建项目 -PROJECT_DESC=项目描述 -UPDATE_PROJECT_NOTES=更新项目 -PROJECT_ID=项目ID -QUERY_PROJECT_BY_ID_NOTES=通过项目ID查询项目信息 -QUERY_PROJECT_LIST_PAGING_NOTES=分页查询项目列表 -QUERY_ALL_PROJECT_LIST_NOTES=查询所有项目 -DELETE_PROJECT_BY_ID_NOTES=删除项目通过ID -QUERY_UNAUTHORIZED_PROJECT_NOTES=查询未授权的项目 -QUERY_AUTHORIZED_PROJECT_NOTES=查询授权项目 -TASK_RECORD_TAG=任务记录相关操作 -QUERY_TASK_RECORD_LIST_PAGING_NOTES=分页查询任务记录列表 -CREATE_TOKEN_NOTES=创建token,注意需要先登录 -QUERY_ACCESS_TOKEN_LIST_NOTES=分页查询access token列表 -SCHEDULE=定时 -WARNING_TYPE=发送策略 -WARNING_GROUP_ID=发送组ID -FAILURE_STRATEGY=失败策略 -RECEIVERS=收件人 -RECEIVERS_CC=收件人(抄送) -WORKER_GROUP_ID=Worker Server分组ID -PROCESS_INSTANCE_PRIORITY=流程实例优先级 -UPDATE_SCHEDULE_NOTES=更新定时 -SCHEDULE_ID=定时ID -ONLINE_SCHEDULE_NOTES=定时上线 -OFFLINE_SCHEDULE_NOTES=定时下线 -QUERY_SCHEDULE_NOTES=查询定时 -QUERY_SCHEDULE_LIST_PAGING_NOTES=分页查询定时 -LOGIN_TAG=用户登录相关操作 -USER_NAME=用户名 -PROJECT_NAME=项目名称 -CREATE_PROCESS_DEFINITION_NOTES=创建流程定义 -PROCESS_DEFINITION_NAME=流程定义名称 -PROCESS_DEFINITION_JSON=流程定义详细信息(json格式) -PROCESS_DEFINITION_LOCATIONS=流程定义节点坐标位置信息(json格式) -PROCESS_INSTANCE_LOCATIONS=流程实例节点坐标位置信息(json格式) -PROCESS_DEFINITION_CONNECTS=流程定义节点图标连接信息(json格式) -PROCESS_INSTANCE_CONNECTS=流程实例节点图标连接信息(json格式) -PROCESS_DEFINITION_DESC=流程定义描述信息 -PROCESS_DEFINITION_TAG=流程定义相关操作 -SIGNOUT_NOTES=退出登录 -USER_PASSWORD=用户密码 -UPDATE_PROCESS_INSTANCE_NOTES=更新流程实例 -QUERY_PROCESS_INSTANCE_LIST_NOTES=查询流程实例列表 -VERIFY_PROCCESS_DEFINITION_NAME_NOTES=验证流程定义名字 -LOGIN_NOTES=用户登录 -UPDATE_PROCCESS_DEFINITION_NOTES=更新流程定义 -PROCESS_DEFINITION_ID=流程定义ID -RELEASE_PROCCESS_DEFINITION_NOTES=发布流程定义 -QUERY_PROCCESS_DEFINITION_BY_ID_NOTES=查询流程定义通过流程定义ID -QUERY_PROCCESS_DEFINITION_LIST_NOTES=查询流程定义列表 -QUERY_PROCCESS_DEFINITION_LIST_PAGING_NOTES=分页查询流程定义列表 -QUERY_ALL_DEFINITION_LIST_NOTES=查询所有流程定义 -PAGE_NO=页码号 -PROCESS_INSTANCE_ID=流程实例ID -PROCESS_INSTANCE_IDS=流程实例ID集合 -PROCESS_INSTANCE_JSON=流程实例信息(json格式) -SCHEDULE_TIME=定时时间 -SYNC_DEFINE=更新流程实例的信息是否同步到流程定义 -RECOVERY_PROCESS_INSTANCE_FLAG=是否恢复流程实例 -SEARCH_VAL=搜索值 -USER_ID=用户ID -PAGE_SIZE=页大小 -LIMIT=显示多少条 -VIEW_TREE_NOTES=树状图 -GET_NODE_LIST_BY_DEFINITION_ID_NOTES=获得任务节点列表通过流程定义ID -PROCESS_DEFINITION_ID_LIST=流程定义id列表 -QUERY_PROCCESS_DEFINITION_All_BY_PROJECT_ID_NOTES=查询流程定义通过项目ID -BATCH_DELETE_PROCESS_DEFINITION_BY_IDS_NOTES=批量删除流程定义通过流程定义ID集合 -DELETE_PROCESS_DEFINITION_BY_ID_NOTES=删除流程定义通过流程定义ID -QUERY_PROCESS_INSTANCE_BY_ID_NOTES=查询流程实例通过流程实例ID -DELETE_PROCESS_INSTANCE_BY_ID_NOTES=删除流程实例通过流程实例ID -TASK_ID=任务实例ID -SKIP_LINE_NUM=忽略行数 -QUERY_TASK_INSTANCE_LOG_NOTES=查询任务实例日志 -DOWNLOAD_TASK_INSTANCE_LOG_NOTES=下载任务实例日志 -USERS_TAG=用户相关操作 -SCHEDULER_TAG=定时相关操作 -CREATE_SCHEDULE_NOTES=创建定时 -CREATE_USER_NOTES=创建用户 -TENANT_ID=租户ID -QUEUE=使用的队列 -EMAIL=邮箱 -PHONE=手机号 -QUERY_USER_LIST_NOTES=查询用户列表 -UPDATE_USER_NOTES=更新用户 -DELETE_USER_BY_ID_NOTES=删除用户通过ID -GRANT_PROJECT_NOTES=授权项目 -PROJECT_IDS=项目IDS(字符串格式,多个项目以","分割) -GRANT_RESOURCE_NOTES=授权资源文件 -RESOURCE_IDS=资源ID列表(字符串格式,多个资源ID以","分割) -GET_USER_INFO_NOTES=获取用户信息 -LIST_USER_NOTES=用户列表 -VERIFY_USER_NAME_NOTES=验证用户名 -UNAUTHORIZED_USER_NOTES=取消授权 -ALERT_GROUP_ID=报警组ID -AUTHORIZED_USER_NOTES=授权用户 -GRANT_UDF_FUNC_NOTES=授权udf函数 -UDF_IDS=udf函数id列表(字符串格式,多个udf函数ID以","分割) -GRANT_DATASOURCE_NOTES=授权数据源 -DATASOURCE_IDS=数据源ID列表(字符串格式,多个数据源ID以","分割) -QUERY_SUBPROCESS_INSTANCE_BY_TASK_ID_NOTES=查询子流程实例通过任务实例ID -QUERY_PARENT_PROCESS_INSTANCE_BY_SUB_PROCESS_INSTANCE_ID_NOTES=查询父流程实例信息通过子流程实例ID -QUERY_PROCESS_INSTANCE_GLOBAL_VARIABLES_AND_LOCAL_VARIABLES_NOTES=查询流程实例全局变量和局部变量 -VIEW_GANTT_NOTES=浏览Gantt图 -SUB_PROCESS_INSTANCE_ID=子流程是咧ID -TASK_NAME=任务实例名 -TASK_INSTANCE_TAG=任务实例相关操作 -LOGGER_TAG=日志相关操作 -PROCESS_INSTANCE_TAG=流程实例相关操作 -EXECUTION_STATUS=工作流和任务节点的运行状态 -HOST=运行任务的主机IP地址 -START_DATE=开始时间 -END_DATE=结束时间 -QUERY_TASK_LIST_BY_PROCESS_INSTANCE_ID_NOTES=通过流程实例ID查询任务列表 -UPDATE_DATA_SOURCE_NOTES=更新数据源 -DATA_SOURCE_ID=数据源ID -QUERY_DATA_SOURCE_NOTES=查询数据源通过ID -QUERY_DATA_SOURCE_LIST_BY_TYPE_NOTES=查询数据源列表通过数据源类型 -QUERY_DATA_SOURCE_LIST_PAGING_NOTES=分页查询数据源列表 -CONNECT_DATA_SOURCE_NOTES=连接数据源 -CONNECT_DATA_SOURCE_TEST_NOTES=连接数据源测试 -DELETE_DATA_SOURCE_NOTES=删除数据源 -VERIFY_DATA_SOURCE_NOTES=验证数据源 -UNAUTHORIZED_DATA_SOURCE_NOTES=未授权的数据源 -AUTHORIZED_DATA_SOURCE_NOTES=授权的数据源 -DELETE_SCHEDULER_BY_ID_NOTES=根据定时id删除定时数据 diff --git a/dockerfile/conf/dolphinscheduler/conf/mail_templates/alert_mail_template.ftl b/dockerfile/conf/dolphinscheduler/conf/mail_templates/alert_mail_template.ftl deleted file mode 100644 index c638609090..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/mail_templates/alert_mail_template.ftl +++ /dev/null @@ -1,17 +0,0 @@ -<#-- - ~ 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. ---> - dolphinscheduler<#if title??> ${title}<#if content??> ${content}
\ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/master_logback.xml b/dockerfile/conf/dolphinscheduler/conf/master_logback.xml deleted file mode 100644 index 12bcd658e1..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/master_logback.xml +++ /dev/null @@ -1,52 +0,0 @@ - - - - - - - - - - [%level] %date{yyyy-MM-dd HH:mm:ss.SSS} %logger{96}:[%line] - %msg%n - - UTF-8 - - - - - ${log.base}/dolphinscheduler-master.log - - INFO - - - ${log.base}/dolphinscheduler-master.%d{yyyy-MM-dd_HH}.%i.log - 168 - 200MB - - - - [%level] %date{yyyy-MM-dd HH:mm:ss.SSS} %logger{96}:[%line] - %msg%n - - UTF-8 - - - - - - - \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/AccessTokenMapper.xml b/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/AccessTokenMapper.xml deleted file mode 100644 index 29c8dfa5a3..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/AccessTokenMapper.xml +++ /dev/null @@ -1,33 +0,0 @@ - - - - - - - \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/AlertGroupMapper.xml b/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/AlertGroupMapper.xml deleted file mode 100644 index 8ee335b6ff..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/AlertGroupMapper.xml +++ /dev/null @@ -1,47 +0,0 @@ - - - - - - - - - - - \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/AlertMapper.xml b/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/AlertMapper.xml deleted file mode 100644 index 703b685157..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/AlertMapper.xml +++ /dev/null @@ -1,26 +0,0 @@ - - - - - - - \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/CommandMapper.xml b/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/CommandMapper.xml deleted file mode 100644 index 66e6c3edd3..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/CommandMapper.xml +++ /dev/null @@ -1,43 +0,0 @@ - - - - - - - - \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/DataSourceMapper.xml b/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/DataSourceMapper.xml deleted file mode 100644 index b296d5fc3e..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/DataSourceMapper.xml +++ /dev/null @@ -1,79 +0,0 @@ - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/DataSourceUserMapper.xml b/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/DataSourceUserMapper.xml deleted file mode 100644 index a43cbeca91..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/DataSourceUserMapper.xml +++ /dev/null @@ -1,30 +0,0 @@ - - - - - - - delete from t_ds_relation_datasource_user - where user_id = #{userId} - - - - delete from t_ds_relation_datasource_user - where datasource_id = #{datasourceId} - - \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ErrorCommandMapper.xml b/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ErrorCommandMapper.xml deleted file mode 100644 index 2f5ae7104a..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ErrorCommandMapper.xml +++ /dev/null @@ -1,36 +0,0 @@ - - - - - - - \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ProcessDefinitionMapper.xml b/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ProcessDefinitionMapper.xml deleted file mode 100644 index 1b97c07676..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ProcessDefinitionMapper.xml +++ /dev/null @@ -1,96 +0,0 @@ - - - - - - - - - - - - - - \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapMapper.xml b/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapMapper.xml deleted file mode 100644 index d217665eab..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapMapper.xml +++ /dev/null @@ -1,43 +0,0 @@ - - - - - - - delete - from t_ds_relation_process_instance - where parent_process_instance_id=#{parentProcessId} - - - - - - \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapper.xml b/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapper.xml deleted file mode 100644 index ccd4463d16..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ProcessInstanceMapper.xml +++ /dev/null @@ -1,185 +0,0 @@ - - - - - - - - - - - - - - - update t_ds_process_instance - set host=null - where host =#{host} and state in - - #{i} - - - - update t_ds_process_instance - set state = #{destState} - where state = #{originState} - - - - update t_ds_process_instance - set tenant_id = #{destTenantId} - where tenant_id = #{originTenantId} - - - - update t_ds_process_instance - set worker_group_id = #{destWorkerGroupId} - where worker_group_id = #{originWorkerGroupId} - - - - - - - - - - \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ProjectMapper.xml b/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ProjectMapper.xml deleted file mode 100644 index 5ab0756250..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ProjectMapper.xml +++ /dev/null @@ -1,68 +0,0 @@ - - - - - - - - - - - - - \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ProjectUserMapper.xml b/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ProjectUserMapper.xml deleted file mode 100644 index 006cf080eb..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ProjectUserMapper.xml +++ /dev/null @@ -1,36 +0,0 @@ - - - - - - - delete from t_ds_relation_project_user - where 1=1 - and user_id = #{userId} - - and project_id = #{projectId} - - - - \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/QueueMapper.xml b/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/QueueMapper.xml deleted file mode 100644 index 423b0dd04d..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/QueueMapper.xml +++ /dev/null @@ -1,42 +0,0 @@ - - - - - - - - - \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ResourceMapper.xml b/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ResourceMapper.xml deleted file mode 100644 index 146daa0632..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ResourceMapper.xml +++ /dev/null @@ -1,74 +0,0 @@ - - - - - - - - - - - - diff --git a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ResourceUserMapper.xml b/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ResourceUserMapper.xml deleted file mode 100644 index 6a89e47c2f..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ResourceUserMapper.xml +++ /dev/null @@ -1,32 +0,0 @@ - - - - - - - delete - from t_ds_relation_resources_user - where 1 = 1 - - and user_id = #{userId} - - - and resources_id = #{resourceId} - - - \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ScheduleMapper.xml b/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ScheduleMapper.xml deleted file mode 100644 index 402c864251..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/ScheduleMapper.xml +++ /dev/null @@ -1,58 +0,0 @@ - - - - - - - - - - \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/SessionMapper.xml b/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/SessionMapper.xml deleted file mode 100644 index 4fa7f309dc..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/SessionMapper.xml +++ /dev/null @@ -1,32 +0,0 @@ - - - - - - - - - \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml b/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml deleted file mode 100644 index 143761bf8c..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml +++ /dev/null @@ -1,132 +0,0 @@ - - - - - - - update t_ds_task_instance - set state = #{destStatus} - where host = #{host} - and state in - - #{i} - - - - - - - - - - diff --git a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/TenantMapper.xml b/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/TenantMapper.xml deleted file mode 100644 index fc9219ce86..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/TenantMapper.xml +++ /dev/null @@ -1,41 +0,0 @@ - - - - - - - - - \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/UDFUserMapper.xml b/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/UDFUserMapper.xml deleted file mode 100644 index 61b4e2c372..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/UDFUserMapper.xml +++ /dev/null @@ -1,29 +0,0 @@ - - - - - - - delete from t_ds_relation_udfs_user - where user_id = #{userId} - - - delete from t_ds_relation_udfs_user - where udf_id = #{udfFuncId} - - \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/UdfFuncMapper.xml b/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/UdfFuncMapper.xml deleted file mode 100644 index 04926d132e..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/UdfFuncMapper.xml +++ /dev/null @@ -1,71 +0,0 @@ - - - - - - - - - - - \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/UserAlertGroupMapper.xml b/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/UserAlertGroupMapper.xml deleted file mode 100644 index cbb448275c..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/UserAlertGroupMapper.xml +++ /dev/null @@ -1,31 +0,0 @@ - - - - - - - delete from t_ds_relation_user_alertgroup - where alertgroup_id = #{alertgroupId} - - - \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/UserMapper.xml b/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/UserMapper.xml deleted file mode 100644 index 6046ad22eb..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/UserMapper.xml +++ /dev/null @@ -1,72 +0,0 @@ - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/WorkerGroupMapper.xml b/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/WorkerGroupMapper.xml deleted file mode 100644 index 84dd4db88d..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/org/apache/dolphinscheduler/dao/mapper/WorkerGroupMapper.xml +++ /dev/null @@ -1,40 +0,0 @@ - - - - - - - - - \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/worker_logback.xml b/dockerfile/conf/dolphinscheduler/conf/worker_logback.xml deleted file mode 100644 index bf4a651e7c..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/worker_logback.xml +++ /dev/null @@ -1,79 +0,0 @@ - - - - - - - - - - [%level] %date{yyyy-MM-dd HH:mm:ss.SSS} %logger{96}:[%line] - %msg%n - - UTF-8 - - - - - INFO - - - - taskAppId - ${log.base} - - - - ${log.base}/${taskAppId}.log - - - [%level] %date{yyyy-MM-dd HH:mm:ss.SSS} %logger{96}:[%line] - %msg%n - - UTF-8 - - true - - - - - - ${log.base}/dolphinscheduler-worker.log - - INFO - - - - ${log.base}/dolphinscheduler-worker.%d{yyyy-MM-dd_HH}.%i.log - 168 - 200MB - -       - - - [%level] %date{yyyy-MM-dd HH:mm:ss.SSS} %logger{96}:[%line] - %msg%n - - UTF-8 - -    - - - - - - - - \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/zookeeper.properties b/dockerfile/conf/dolphinscheduler/conf/zookeeper.properties deleted file mode 100644 index 5e9df1c863..0000000000 --- a/dockerfile/conf/dolphinscheduler/conf/zookeeper.properties +++ /dev/null @@ -1,42 +0,0 @@ -# -# 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. -# - -#zookeeper cluster -zookeeper.quorum=127.0.0.1:2181 - -#dolphinscheduler root directory -zookeeper.dolphinscheduler.root=/dolphinscheduler - -#zookeeper server dirctory -zookeeper.dolphinscheduler.dead.servers=/dolphinscheduler/dead-servers -zookeeper.dolphinscheduler.masters=/dolphinscheduler/masters -zookeeper.dolphinscheduler.workers=/dolphinscheduler/workers - -#zookeeper lock dirctory -zookeeper.dolphinscheduler.lock.masters=/dolphinscheduler/lock/masters -zookeeper.dolphinscheduler.lock.workers=/dolphinscheduler/lock/workers - -#dolphinscheduler failover directory -zookeeper.dolphinscheduler.lock.failover.masters=/dolphinscheduler/lock/failover/masters -zookeeper.dolphinscheduler.lock.failover.workers=/dolphinscheduler/lock/failover/workers -zookeeper.dolphinscheduler.lock.failover.startup.masters=/dolphinscheduler/lock/failover/startup-masters - -#dolphinscheduler failover directory -zookeeper.session.timeout=300 -zookeeper.connection.timeout=300 -zookeeper.retry.sleep=1000 -zookeeper.retry.maxtime=5 \ No newline at end of file diff --git a/dockerfile/conf/dolphinscheduler/conf/config/run_config.conf b/dockerfile/conf/dolphinscheduler/env/dolphinscheduler_env similarity index 65% rename from dockerfile/conf/dolphinscheduler/conf/config/run_config.conf rename to dockerfile/conf/dolphinscheduler/env/dolphinscheduler_env index 69a28db458..070c438bb6 100644 --- a/dockerfile/conf/dolphinscheduler/conf/config/run_config.conf +++ b/dockerfile/conf/dolphinscheduler/env/dolphinscheduler_env @@ -15,7 +15,12 @@ # limitations under the License. # -masters=ark0,ark1 -workers=ark2,ark3,ark4 -alertServer=ark3 -apiServers=ark1 \ No newline at end of file +export HADOOP_HOME=/opt/soft/hadoop +export HADOOP_CONF_DIR=/opt/soft/hadoop/etc/hadoop +export SPARK_HOME1=/opt/soft/spark1 +export SPARK_HOME2=/opt/soft/spark2 +export PYTHON_HOME=/opt/soft/python +export JAVA_HOME=/opt/soft/java +export HIVE_HOME=/opt/soft/hive +export FLINK_HOME=/opt/soft/flink +export PATH=$HADOOP_HOME/bin:$SPARK_HOME1/bin:$SPARK_HOME2/bin:$PYTHON_HOME:$JAVA_HOME/bin:$HIVE_HOME/bin:$FLINK_HOME/bin:$PATH diff --git a/dockerfile/conf/dolphinscheduler/conf/quartz.properties b/dockerfile/conf/dolphinscheduler/quartz.properties.tpl similarity index 81% rename from dockerfile/conf/dolphinscheduler/conf/quartz.properties rename to dockerfile/conf/dolphinscheduler/quartz.properties.tpl index a83abad5bc..de5496bc3d 100644 --- a/dockerfile/conf/dolphinscheduler/conf/quartz.properties +++ b/dockerfile/conf/dolphinscheduler/quartz.properties.tpl @@ -18,6 +18,13 @@ #============================================================================ # Configure Main Scheduler Properties #============================================================================ +#org.quartz.jobStore.driverDelegateClass = org.quartz.impl.jdbcjobstore.StdJDBCDelegate +org.quartz.jobStore.driverDelegateClass = org.quartz.impl.jdbcjobstore.PostgreSQLDelegate +# postgre +org.quartz.dataSource.myDs.driver = org.postgresql.Driver +org.quartz.dataSource.myDs.URL = jdbc:postgresql://${POSTGRESQL_HOST}:${POSTGRESQL_PORT}/dolphinscheduler?characterEncoding=utf8 +org.quartz.dataSource.myDs.user = ${POSTGRESQL_USERNAME} +org.quartz.dataSource.myDs.password = ${POSTGRESQL_PASSWORD} org.quartz.scheduler.instanceName = DolphinScheduler org.quartz.scheduler.instanceId = AUTO org.quartz.scheduler.makeSchedulerThreadDaemon = true @@ -26,7 +33,6 @@ org.quartz.jobStore.useProperties = false #============================================================================ # Configure ThreadPool #============================================================================ - org.quartz.threadPool.class = org.quartz.simpl.SimpleThreadPool org.quartz.threadPool.makeThreadsDaemons = true org.quartz.threadPool.threadCount = 25 @@ -35,22 +41,17 @@ org.quartz.threadPool.threadPriority = 5 #============================================================================ # Configure JobStore #============================================================================ - org.quartz.jobStore.class = org.quartz.impl.jdbcjobstore.JobStoreTX -org.quartz.jobStore.driverDelegateClass = org.quartz.impl.jdbcjobstore.PostgreSQLDelegate org.quartz.jobStore.tablePrefix = QRTZ_ org.quartz.jobStore.isClustered = true org.quartz.jobStore.misfireThreshold = 60000 org.quartz.jobStore.clusterCheckinInterval = 5000 +org.quartz.jobStore.acquireTriggersWithinLock=true org.quartz.jobStore.dataSource = myDs #============================================================================ -# Configure Datasources +# Configure Datasources #============================================================================ -org.quartz.dataSource.myDs.connectionProvider.class = org.apache.dolphinscheduler.dao.quartz.DruidConnectionProvider -org.quartz.dataSource.myDs.driver = org.postgresql.Driver -org.quartz.dataSource.myDs.URL=jdbc:postgresql://127.0.0.1:5432/dolphinscheduler -org.quartz.dataSource.myDs.user=root -org.quartz.dataSource.myDs.password=root@123 +org.quartz.dataSource.myDs.connectionProvider.class = org.apache.dolphinscheduler.service.quartz.DruidConnectionProvider org.quartz.dataSource.myDs.maxConnections = 10 -org.quartz.dataSource.myDs.validationQuery = select 1 +org.quartz.dataSource.myDs.validationQuery = select 1 \ No newline at end of file diff --git a/dockerfile/conf/maven/settings.xml b/dockerfile/conf/maven/settings.xml deleted file mode 100644 index 6bdea4a1bf..0000000000 --- a/dockerfile/conf/maven/settings.xml +++ /dev/null @@ -1,263 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - nexus-aliyun - central - Nexus aliyun - http://maven.aliyun.com/nexus/content/groups/public - - - - - - - - - - - - diff --git a/dockerfile/conf/nginx/dolphinscheduler.conf b/dockerfile/conf/nginx/dolphinscheduler.conf index 03f87e6b52..9c2c3913dc 100644 --- a/dockerfile/conf/nginx/dolphinscheduler.conf +++ b/dockerfile/conf/nginx/dolphinscheduler.conf @@ -21,11 +21,11 @@ server { #charset koi8-r; #access_log /var/log/nginx/host.access.log main; location / { - root /opt/dolphinscheduler_source/dolphinscheduler-ui/dist; + root /opt/dolphinscheduler/ui; index index.html index.html; } location /dolphinscheduler { - proxy_pass http://127.0.0.1:12345; + proxy_pass http://FRONTEND_API_SERVER_HOST:FRONTEND_API_SERVER_PORT; proxy_set_header Host $host; proxy_set_header X-Real-IP $remote_addr; proxy_set_header x_real_ipP $remote_addr; diff --git a/dockerfile/hooks/build b/dockerfile/hooks/build index 8b7d5329dc..2394a8cd9c 100644 --- a/dockerfile/hooks/build +++ b/dockerfile/hooks/build @@ -15,10 +15,21 @@ # See the License for the specific language governing permissions and # limitations under the License. # - echo "------ dolphinscheduler start - build -------" printenv -docker build --build-arg version=$version --build-arg tar_version=$tar_version -t $DOCKER_REPO:$version . +echo -e "Current Directory is $(pwd)\n" + +# maven package(Project Directory) +echo -e "mvn clean compile package -Prelease" +mvn clean compile package -Prelease + +# mv dolphinscheduler-bin.tar.gz file to dockerfile directory +echo -e "mv $(pwd)/dolphinscheduler-dist/target/apache-dolphinscheduler-incubating-${VERSION}-SNAPSHOT-dolphinscheduler-bin.tar.gz $(pwd)/dockerfile/\n" +mv $(pwd)/dolphinscheduler-dist/target/apache-dolphinscheduler-incubating-${VERSION}-SNAPSHOT-dolphinscheduler-bin.tar.gz $(pwd)/dockerfile/ + +# docker build +echo -e "docker build --build-arg VERSION=${VERSION} -t $DOCKER_REPO:${VERSION} $(pwd)/dockerfile/\n" +docker build --build-arg VERSION=${VERSION} -t $DOCKER_REPO:${VERSION} $(pwd)/dockerfile/ echo "------ dolphinscheduler end - build -------" diff --git a/dockerfile/hooks/build.bat b/dockerfile/hooks/build.bat new file mode 100644 index 0000000000..33a6c23ecf --- /dev/null +++ b/dockerfile/hooks/build.bat @@ -0,0 +1,33 @@ +:: 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. +:: +echo "------ dolphinscheduler start - build -------" +set + +echo "Current Directory is %cd%" + +:: maven package(Project Directory) +echo "call mvn clean compile package -Prelease" +call mvn clean compile package -Prelease -DskipTests=true + +:: move dolphinscheduler-bin.tar.gz file to dockerfile directory +echo "move %cd%\dolphinscheduler-dist\target\apache-dolphinscheduler-incubating-%VERSION%-SNAPSHOT-dolphinscheduler-bin.tar.gz %cd%\dockerfile\" +move %cd%\dolphinscheduler-dist\target\apache-dolphinscheduler-incubating-%VERSION%-SNAPSHOT-dolphinscheduler-bin.tar.gz %cd%\dockerfile\ + +:: docker build +echo "docker build --build-arg VERSION=%VERSION% -t %DOCKER_REPO%:%VERSION% %cd%\dockerfile\" +docker build --build-arg VERSION=%VERSION% -t %DOCKER_REPO%:%VERSION% %cd%\dockerfile\ + +echo "------ dolphinscheduler end - build -------" \ No newline at end of file diff --git a/dockerfile/hooks/push b/dockerfile/hooks/push index 6146727d45..41a25c54fe 100644 --- a/dockerfile/hooks/push +++ b/dockerfile/hooks/push @@ -19,6 +19,6 @@ echo "------ push start -------" printenv -docker push $DOCKER_REPO:$version +docker push $DOCKER_REPO:${VERSION} echo "------ push end -------" diff --git a/dockerfile/hooks/push.bat b/dockerfile/hooks/push.bat new file mode 100644 index 0000000000..ee0c37bd37 --- /dev/null +++ b/dockerfile/hooks/push.bat @@ -0,0 +1,22 @@ +:: 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. +:: + +echo "------ push start -------" +set + +docker push %DOCKER_REPO%:%VERSION% + +echo "------ push end -------" diff --git a/dockerfile/startup-init-conf.sh b/dockerfile/startup-init-conf.sh new file mode 100644 index 0000000000..db37976168 --- /dev/null +++ b/dockerfile/startup-init-conf.sh @@ -0,0 +1,100 @@ +#!/bin/bash +# +# 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. +# + +set -e + +echo "init env variables" + +# Define parameters default value. +#============================================================================ +# Database Source +#============================================================================ +export POSTGRESQL_HOST=${POSTGRESQL_HOST:-"127.0.0.1"} +export POSTGRESQL_PORT=${POSTGRESQL_PORT:-"5432"} +export POSTGRESQL_USERNAME=${POSTGRESQL_USERNAME:-"root"} +export POSTGRESQL_PASSWORD=${POSTGRESQL_PASSWORD:-"root"} + +#============================================================================ +# System +#============================================================================ +export DOLPHINSCHEDULER_ENV_PATH=${DOLPHINSCHEDULER_ENV_PATH:-"/opt/dolphinscheduler/conf/env/dolphinscheduler_env.sh"} + +#============================================================================ +# Zookeeper +#============================================================================ +export TASK_QUEUE=${TASK_QUEUE:-"zookeeper"} +export ZOOKEEPER_QUORUM=${ZOOKEEPER_QUORUM:-"127.0.0.1:2181"} + +#============================================================================ +# Master Server +#============================================================================ +export MASTER_EXEC_THREADS=${MASTER_EXEC_THREADS:-"100"} +export MASTER_EXEC_TASK_NUM=${MASTER_EXEC_TASK_NUM:-"20"} +export MASTER_HEARTBEAT_INTERVAL=${MASTER_HEARTBEAT_INTERVAL:-"10"} +export MASTER_TASK_COMMIT_RETRYTIMES=${MASTER_TASK_COMMIT_RETRYTIMES:-"5"} +export MASTER_TASK_COMMIT_INTERVAL=${MASTER_TASK_COMMIT_INTERVAL:-"1000"} +export MASTER_MAX_CPULOAD_AVG=${MASTER_MAX_CPULOAD_AVG:-"100"} +export MASTER_RESERVED_MEMORY=${MASTER_RESERVED_MEMORY:-"0.1"} + +#============================================================================ +# Worker Server +#============================================================================ +export WORKER_EXEC_THREADS=${WORKER_EXEC_THREADS:-"100"} +export WORKER_HEARTBEAT_INTERVAL=${WORKER_HEARTBEAT_INTERVAL:-"10"} +export WORKER_FETCH_TASK_NUM=${WORKER_FETCH_TASK_NUM:-"3"} +export WORKER_MAX_CPULOAD_AVG=${WORKER_MAX_CPULOAD_AVG:-"100"} +export WORKER_RESERVED_MEMORY=${WORKER_RESERVED_MEMORY:-"0.1"} + +#============================================================================ +# Alert Server +#============================================================================ +# XLS FILE +export XLS_FILE_PATH=${XLS_FILE_PATH:-"/tmp/xls"} +# mail +export MAIL_SERVER_HOST=${MAIL_SERVER_HOST:-""} +export MAIL_SERVER_PORT=${MAIL_SERVER_PORT:-""} +export MAIL_SENDER=${MAIL_SENDER:-""} +export MAIL_USER=${MAIL_USER:-""} +export MAIL_PASSWD=${MAIL_PASSWD:-""} +export MAIL_SMTP_STARTTLS_ENABLE=${MAIL_SMTP_STARTTLS_ENABLE:-"true"} +export MAIL_SMTP_SSL_ENABLE=${MAIL_SMTP_SSL_ENABLE:-"false"} +export MAIL_SMTP_SSL_TRUST=${MAIL_SMTP_SSL_TRUST:-""} +# wechat +export ENTERPRISE_WECHAT_ENABLE=${ENTERPRISE_WECHAT_ENABLE:-"false"} +export ENTERPRISE_WECHAT_CORP_ID=${ENTERPRISE_WECHAT_CORP_ID:-""} +export ENTERPRISE_WECHAT_SECRET=${ENTERPRISE_WECHAT_SECRET:-""} +export ENTERPRISE_WECHAT_AGENT_ID=${ENTERPRISE_WECHAT_AGENT_ID:-""} +export ENTERPRISE_WECHAT_USERS=${ENTERPRISE_WECHAT_USERS:-""} + +#============================================================================ +# Frontend +#============================================================================ +export FRONTEND_API_SERVER_HOST=${FRONTEND_API_SERVER_HOST:-"127.0.0.1"} +export FRONTEND_API_SERVER_PORT=${FRONTEND_API_SERVER_PORT:-"12345"} + +echo "generate app config" +ls ${DOLPHINSCHEDULER_HOME}/conf/ | grep ".tpl" | while read line; do +eval "cat << EOF +$(cat ${DOLPHINSCHEDULER_HOME}/conf/${line}) +EOF +" > ${DOLPHINSCHEDULER_HOME}/conf/${line%.*} +done + +echo "generate nginx config" +sed -i "s/FRONTEND_API_SERVER_HOST/${FRONTEND_API_SERVER_HOST}/g" /etc/nginx/conf.d/dolphinscheduler.conf +sed -i "s/FRONTEND_API_SERVER_PORT/${FRONTEND_API_SERVER_PORT}/g" /etc/nginx/conf.d/dolphinscheduler.conf \ No newline at end of file diff --git a/dockerfile/startup.sh b/dockerfile/startup.sh index cc98d07e57..e52384d6b9 100644 --- a/dockerfile/startup.sh +++ b/dockerfile/startup.sh @@ -17,59 +17,174 @@ # set -e - echo "start postgresql service" - /etc/init.d/postgresql restart - echo "create user and init db" - sudo -u postgres psql <<'ENDSSH' -create user root with password 'root@123'; -create database dolphinscheduler owner root; -grant all privileges on database dolphinscheduler to root; -\q -ENDSSH - echo "import sql data" - /opt/dolphinscheduler/script/create-dolphinscheduler.sh - -/opt/zookeeper/bin/zkServer.sh restart - -sleep 90 - -echo "start api-server" -/opt/dolphinscheduler/bin/dolphinscheduler-daemon.sh stop api-server -/opt/dolphinscheduler/bin/dolphinscheduler-daemon.sh start api-server - - - -echo "start master-server" -/opt/dolphinscheduler/bin/dolphinscheduler-daemon.sh stop master-server -python /opt/dolphinscheduler/script/del-zk-node.py 127.0.0.1 /dolphinscheduler/masters -/opt/dolphinscheduler/bin/dolphinscheduler-daemon.sh start master-server - -echo "start worker-server" -/opt/dolphinscheduler/bin/dolphinscheduler-daemon.sh stop worker-server -python /opt/dolphinscheduler/script/del-zk-node.py 127.0.0.1 /dolphinscheduler/workers -/opt/dolphinscheduler/bin/dolphinscheduler-daemon.sh start worker-server - -echo "start logger-server" -/opt/dolphinscheduler/bin/dolphinscheduler-daemon.sh stop logger-server -/opt/dolphinscheduler/bin/dolphinscheduler-daemon.sh start logger-server +DOLPHINSCHEDULER_BIN=${DOLPHINSCHEDULER_HOME}/bin +DOLPHINSCHEDULER_SCRIPT=${DOLPHINSCHEDULER_HOME}/script +DOLPHINSCHEDULER_LOGS=${DOLPHINSCHEDULER_HOME}/logs + +# start postgresql +initPostgreSQL() { + echo "checking postgresql" + if [ -n "$(ifconfig | grep ${POSTGRESQL_HOST})" ]; then + echo "start postgresql service" + rc-service postgresql restart + + # role if not exists, create + flag=$(sudo -u postgres psql -tAc "SELECT 1 FROM pg_roles WHERE rolname='${POSTGRESQL_USERNAME}'") + if [ -z "${flag}" ]; then + echo "create user" + sudo -u postgres psql -tAc "create user ${POSTGRESQL_USERNAME} with password '${POSTGRESQL_PASSWORD}'" + fi + + # database if not exists, create + flag=$(sudo -u postgres psql -tAc "select 1 from pg_database where datname='dolphinscheduler'") + if [ -z "${flag}" ]; then + echo "init db" + sudo -u postgres psql -tAc "create database dolphinscheduler owner ${POSTGRESQL_USERNAME}" + fi + + # grant + sudo -u postgres psql -tAc "grant all privileges on database dolphinscheduler to ${POSTGRESQL_USERNAME}" + fi + + echo "connect postgresql service" + v=$(sudo -u postgres PGPASSWORD=${POSTGRESQL_PASSWORD} psql -h ${POSTGRESQL_HOST} -U ${POSTGRESQL_USERNAME} -d dolphinscheduler -tAc "select 1") + if [ "$(echo '${v}' | grep 'FATAL' | wc -l)" -eq 1 ]; then + echo "Can't connect to database...${v}" + exit 1 + fi - -echo "start alert-server" -/opt/dolphinscheduler/bin/dolphinscheduler-daemon.sh stop alert-server -/opt/dolphinscheduler/bin/dolphinscheduler-daemon.sh start alert-server - - - - - -echo "start nginx" -/etc/init.d/nginx stop -nginx & - - -while true -do - sleep 101 -done -exec "$@" + echo "import sql data" + ${DOLPHINSCHEDULER_SCRIPT}/create-dolphinscheduler.sh +} + +# start zk +initZK() { + echo -e "checking zookeeper" + if [[ "${ZOOKEEPER_QUORUM}" = "127.0.0.1:2181" || "${ZOOKEEPER_QUORUM}" = "localhost:2181" ]]; then + echo "start local zookeeper" + /opt/zookeeper/bin/zkServer.sh restart + else + echo "connect remote zookeeper" + echo "${ZOOKEEPER_QUORUM}" | awk -F ',' 'BEGIN{ i=1 }{ while( i <= NF ){ print $i; i++ } }' | while read line; do + while ! nc -z ${line%:*} ${line#*:}; do + counter=$((counter+1)) + if [ $counter == 30 ]; then + log "Error: Couldn't connect to zookeeper." + exit 1 + fi + log "Trying to connect to zookeeper at ${line}. Attempt $counter." + sleep 5 + done + done + fi +} + +# start nginx +initNginx() { + echo "start nginx" + nginx & +} + +# start master-server +initMasterServer() { + echo "start master-server" + ${DOLPHINSCHEDULER_BIN}/dolphinscheduler-daemon.sh stop master-server + ${DOLPHINSCHEDULER_BIN}/dolphinscheduler-daemon.sh start master-server +} + +# start worker-server +initWorkerServer() { + echo "start worker-server" + ${DOLPHINSCHEDULER_BIN}/dolphinscheduler-daemon.sh stop worker-server + ${DOLPHINSCHEDULER_BIN}/dolphinscheduler-daemon.sh start worker-server +} + +# start api-server +initApiServer() { + echo "start api-server" + ${DOLPHINSCHEDULER_BIN}/dolphinscheduler-daemon.sh stop api-server + ${DOLPHINSCHEDULER_BIN}/dolphinscheduler-daemon.sh start api-server +} + +# start logger-server +initLoggerServer() { + echo "start logger-server" + ${DOLPHINSCHEDULER_BIN}/dolphinscheduler-daemon.sh stop logger-server + ${DOLPHINSCHEDULER_BIN}/dolphinscheduler-daemon.sh start logger-server +} + +# start alert-server +initAlertServer() { + echo "start alert-server" + ${DOLPHINSCHEDULER_BIN}/dolphinscheduler-daemon.sh stop alert-server + ${DOLPHINSCHEDULER_BIN}/dolphinscheduler-daemon.sh start alert-server +} + +# print usage +printUsage() { + echo -e "Dolphin Scheduler is a distributed and easy-to-expand visual DAG workflow scheduling system," + echo -e "dedicated to solving the complex dependencies in data processing, making the scheduling system out of the box for data processing.\n" + echo -e "Usage: [ all | master-server | worker-server | api-server | alert-server | frontend ]\n" + printf "%-13s: %s\n" "all" "Run master-server, worker-server, api-server, alert-server and frontend." + printf "%-13s: %s\n" "master-server" "MasterServer is mainly responsible for DAG task split, task submission monitoring." + printf "%-13s: %s\n" "worker-server" "WorkerServer is mainly responsible for task execution and providing log services.." + printf "%-13s: %s\n" "api-server" "ApiServer is mainly responsible for processing requests from the front-end UI layer." + printf "%-13s: %s\n" "alert-server" "AlertServer mainly include Alarms." + printf "%-13s: %s\n" "frontend" "Frontend mainly provides various visual operation interfaces of the system." +} + +# init config file +source /root/startup-init-conf.sh + +LOGFILE=/var/log/nginx/access.log +case "$1" in + (all) + initZK + initPostgreSQL + initMasterServer + initWorkerServer + initApiServer + initAlertServer + initLoggerServer + initNginx + LOGFILE=/var/log/nginx/access.log + ;; + (master-server) + initZK + initPostgreSQL + initMasterServer + LOGFILE=${DOLPHINSCHEDULER_LOGS}/dolphinscheduler-master.log + ;; + (worker-server) + initZK + initPostgreSQL + initWorkerServer + initLoggerServer + LOGFILE=${DOLPHINSCHEDULER_LOGS}/dolphinscheduler-worker.log + ;; + (api-server) + initPostgreSQL + initApiServer + LOGFILE=${DOLPHINSCHEDULER_LOGS}/dolphinscheduler-api-server.log + ;; + (alert-server) + initPostgreSQL + initAlertServer + LOGFILE=${DOLPHINSCHEDULER_LOGS}/dolphinscheduler-alert.log + ;; + (frontend) + initNginx + LOGFILE=/var/log/nginx/access.log + ;; + (help) + printUsage + exit 1 + ;; + (*) + printUsage + exit 1 + ;; +esac + +exec tee ${LOGFILE} \ No newline at end of file diff --git a/dolphinscheduler-ui/package.json b/dolphinscheduler-ui/package.json index 421fd394d6..da15b722fc 100644 --- a/dolphinscheduler-ui/package.json +++ b/dolphinscheduler-ui/package.json @@ -53,7 +53,7 @@ "html-loader": "^0.5.5", "html-webpack-plugin": "^3.2.0", "mini-css-extract-plugin": "^0.8.2", - "node-sass": "^4.13.0", + "node-sass": "^4.13.1", "postcss-loader": "^3.0.0", "progress-bar-webpack-plugin": "^1.12.1", "rimraf": "^2.6.2", diff --git a/script/dolphinscheduler-daemon.sh b/script/dolphinscheduler-daemon.sh index abc33a4456..3e7f10783a 100644 --- a/script/dolphinscheduler-daemon.sh +++ b/script/dolphinscheduler-daemon.sh @@ -44,7 +44,7 @@ export DOLPHINSCHEDULER_LOG_DIR=$DOLPHINSCHEDULER_HOME/logs export DOLPHINSCHEDULER_CONF_DIR=$DOLPHINSCHEDULER_HOME/conf export DOLPHINSCHEDULER_LIB_JARS=$DOLPHINSCHEDULER_HOME/lib/* -export DOLPHINSCHEDULER_OPTS="-server -Xmx16g -Xms4g -Xss512k -XX:+DisableExplicitGC -XX:+UseConcMarkSweepGC -XX:+CMSParallelRemarkEnabled -XX:LargePageSizeInBytes=128m -XX:+UseFastAccessorMethods -XX:+UseCMSInitiatingOccupancyOnly -XX:CMSInitiatingOccupancyFraction=70" +export DOLPHINSCHEDULER_OPTS="-server -Xmx16g -Xms1g -Xss512k -XX:+DisableExplicitGC -XX:+UseConcMarkSweepGC -XX:+CMSParallelRemarkEnabled -XX:LargePageSizeInBytes=128m -XX:+UseFastAccessorMethods -XX:+UseCMSInitiatingOccupancyOnly -XX:CMSInitiatingOccupancyFraction=70" export STOP_TIMEOUT=5 if [ ! -d "$DOLPHINSCHEDULER_LOG_DIR" ]; then From b4f2e5f4210ce70d6ec50e68fd177ee26841fc03 Mon Sep 17 00:00:00 2001 From: zhangchunyang <18910529250@163.com> Date: Sat, 22 Feb 2020 18:28:02 +0800 Subject: [PATCH 026/221] Add ambari plugin #1469 (#1970) * ambari plugin * add readme.pdf * ambari plugin * add readme.md * add readme.pdf * 1.add license 2. update master cardinality 3. nesessary paramaters default empty 4. update alert message --- .../common-services/DOLPHIN/1.2.1/alerts.json | 158 +++++ .../1.2.1/configuration/dolphin-alert.xml | 144 +++++ .../configuration/dolphin-application-api.xml | 71 ++ .../configuration/dolphin-application.xml | 467 ++++++++++++++ .../1.2.1/configuration/dolphin-common.xml | 232 +++++++ .../1.2.1/configuration/dolphin-env.xml | 123 ++++ .../1.2.1/configuration/dolphin-quartz.xml | 131 ++++ .../DOLPHIN/1.2.1/metainfo.xml | 137 ++++ .../alerts/alert_dolphin_scheduler_status.py | 124 ++++ .../package/scripts/dolphin_alert_service.py | 61 ++ .../package/scripts/dolphin_api_service.py | 70 ++ .../1.2.1/package/scripts/dolphin_env.py | 121 ++++ .../package/scripts/dolphin_logger_service.py | 61 ++ .../package/scripts/dolphin_master_service.py | 61 ++ .../package/scripts/dolphin_worker_service.py | 60 ++ .../DOLPHIN/1.2.1/package/scripts/params.py | 150 +++++ .../1.2.1/package/scripts/service_check.py | 31 + .../1.2.1/package/scripts/status_params.py | 23 + .../package/templates/alert.properties.j2 | 20 + .../templates/application-api.properties.j2 | 20 + .../templates/application.properties.j2 | 20 + .../package/templates/common.properties.j2 | 20 + .../1.2.1/package/templates/dolphin-daemon.j2 | 119 ++++ .../package/templates/quartz.properties.j2 | 20 + .../DOLPHIN/1.2.1/quicklinks/quicklinks.json | 26 + .../DOLPHIN/1.2.1/themes/theme.json | 605 ++++++++++++++++++ ambari_plugin/readme.pdf | Bin 0 -> 704750 bytes ambari_plugin/statcks/DOLPHIN/metainfo.xml | 26 + 28 files changed, 3101 insertions(+) create mode 100644 ambari_plugin/common-services/DOLPHIN/1.2.1/alerts.json create mode 100644 ambari_plugin/common-services/DOLPHIN/1.2.1/configuration/dolphin-alert.xml create mode 100644 ambari_plugin/common-services/DOLPHIN/1.2.1/configuration/dolphin-application-api.xml create mode 100644 ambari_plugin/common-services/DOLPHIN/1.2.1/configuration/dolphin-application.xml create mode 100644 ambari_plugin/common-services/DOLPHIN/1.2.1/configuration/dolphin-common.xml create mode 100644 ambari_plugin/common-services/DOLPHIN/1.2.1/configuration/dolphin-env.xml create mode 100644 ambari_plugin/common-services/DOLPHIN/1.2.1/configuration/dolphin-quartz.xml create mode 100644 ambari_plugin/common-services/DOLPHIN/1.2.1/metainfo.xml create mode 100644 ambari_plugin/common-services/DOLPHIN/1.2.1/package/alerts/alert_dolphin_scheduler_status.py create mode 100644 ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/dolphin_alert_service.py create mode 100644 ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/dolphin_api_service.py create mode 100644 ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/dolphin_env.py create mode 100644 ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/dolphin_logger_service.py create mode 100644 ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/dolphin_master_service.py create mode 100644 ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/dolphin_worker_service.py create mode 100644 ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/params.py create mode 100644 ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/service_check.py create mode 100644 ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/status_params.py create mode 100644 ambari_plugin/common-services/DOLPHIN/1.2.1/package/templates/alert.properties.j2 create mode 100644 ambari_plugin/common-services/DOLPHIN/1.2.1/package/templates/application-api.properties.j2 create mode 100644 ambari_plugin/common-services/DOLPHIN/1.2.1/package/templates/application.properties.j2 create mode 100644 ambari_plugin/common-services/DOLPHIN/1.2.1/package/templates/common.properties.j2 create mode 100644 ambari_plugin/common-services/DOLPHIN/1.2.1/package/templates/dolphin-daemon.j2 create mode 100644 ambari_plugin/common-services/DOLPHIN/1.2.1/package/templates/quartz.properties.j2 create mode 100755 ambari_plugin/common-services/DOLPHIN/1.2.1/quicklinks/quicklinks.json create mode 100644 ambari_plugin/common-services/DOLPHIN/1.2.1/themes/theme.json create mode 100644 ambari_plugin/readme.pdf create mode 100755 ambari_plugin/statcks/DOLPHIN/metainfo.xml diff --git a/ambari_plugin/common-services/DOLPHIN/1.2.1/alerts.json b/ambari_plugin/common-services/DOLPHIN/1.2.1/alerts.json new file mode 100644 index 0000000000..769245b366 --- /dev/null +++ b/ambari_plugin/common-services/DOLPHIN/1.2.1/alerts.json @@ -0,0 +1,158 @@ +{ + "DOLPHIN": { + "service": [], + "DOLPHIN_API": [ + { + "name": "dolphin_api_port_check", + "label": "dolphin_api_port_check", + "description": "dolphin_api_port_check.", + "interval": 10, + "scope": "ANY", + "source": { + "type": "PORT", + "uri": "{{dolphin-application-api/server.port}}", + "default_port": 12345, + "reporting": { + "ok": { + "text": "TCP OK - {0:.3f}s response on port {1}" + }, + "warning": { + "text": "TCP OK - {0:.3f}s response on port {1}", + "value": 1.5 + }, + "critical": { + "text": "Connection failed: {0} to {1}:{2}", + "value": 5.0 + } + } + } + } + ], + "DOLPHIN_LOGGER": [ + { + "name": "dolphin_logger_port_check", + "label": "dolphin_logger_port_check", + "description": "dolphin_logger_port_check.", + "interval": 10, + "scope": "ANY", + "source": { + "type": "PORT", + "uri": "{{dolphin-common/loggerserver.rpc.port}}", + "default_port": 50051, + "reporting": { + "ok": { + "text": "TCP OK - {0:.3f}s response on port {1}" + }, + "warning": { + "text": "TCP OK - {0:.3f}s response on port {1}", + "value": 1.5 + }, + "critical": { + "text": "Connection failed: {0} to {1}:{2}", + "value": 5.0 + } + } + } + } + ], + "DOLPHIN_MASTER": [ + { + "name": "DOLPHIN_MASTER_CHECK", + "label": "check dolphin scheduler master status", + "description": "", + "interval":10, + "scope": "HOST", + "enabled": true, + "source": { + "type": "SCRIPT", + "path": "DOLPHIN/1.2.1/package/alerts/alert_dolphin_scheduler_status.py", + "parameters": [ + + { + "name": "connection.timeout", + "display_name": "Connection Timeout", + "value": 5.0, + "type": "NUMERIC", + "description": "The maximum time before this alert is considered to be CRITICAL", + "units": "seconds", + "threshold": "CRITICAL" + }, + { + "name": "alertName", + "display_name": "alertName", + "value": "DOLPHIN_MASTER", + "type": "STRING", + "description": "alert name" + } + ] + } + } + ], + "DOLPHIN_WORKER": [ + { + "name": "DOLPHIN_WORKER_CHECK", + "label": "check dolphin scheduler worker status", + "description": "", + "interval":10, + "scope": "HOST", + "enabled": true, + "source": { + "type": "SCRIPT", + "path": "DOLPHIN/1.2.1/package/alerts/alert_dolphin_scheduler_status.py", + "parameters": [ + + { + "name": "connection.timeout", + "display_name": "Connection Timeout", + "value": 5.0, + "type": "NUMERIC", + "description": "The maximum time before this alert is considered to be CRITICAL", + "units": "seconds", + "threshold": "CRITICAL" + }, + { + "name": "alertName", + "display_name": "alertName", + "value": "DOLPHIN_WORKER", + "type": "STRING", + "description": "alert name" + } + ] + } + } + ], + "DOLPHIN_ALERT": [ + { + "name": "DOLPHIN_DOLPHIN_ALERT_CHECK", + "label": "check dolphin scheduler alert status", + "description": "", + "interval":10, + "scope": "HOST", + "enabled": true, + "source": { + "type": "SCRIPT", + "path": "DOLPHIN/1.2.1/package/alerts/alert_dolphin_scheduler_status.py", + "parameters": [ + + { + "name": "connection.timeout", + "display_name": "Connection Timeout", + "value": 5.0, + "type": "NUMERIC", + "description": "The maximum time before this alert is considered to be CRITICAL", + "units": "seconds", + "threshold": "CRITICAL" + }, + { + "name": "alertName", + "display_name": "alertName", + "value": "DOLPHIN_ALERT", + "type": "STRING", + "description": "alert name" + } + ] + } + } + ] + } +} \ No newline at end of file diff --git a/ambari_plugin/common-services/DOLPHIN/1.2.1/configuration/dolphin-alert.xml b/ambari_plugin/common-services/DOLPHIN/1.2.1/configuration/dolphin-alert.xml new file mode 100644 index 0000000000..5b82230148 --- /dev/null +++ b/ambari_plugin/common-services/DOLPHIN/1.2.1/configuration/dolphin-alert.xml @@ -0,0 +1,144 @@ + + + + alert.type + EMAIL + alert type is EMAIL/SMS + + + + mail.protocol + SMTP + + + + + mail.server.host + xxx.xxx.com + + + + + mail.server.port + 25 + + int + + + + + + mail.sender + admin + + + + + mail.user + admin + + + + + mail.passwd + 000000 + + PASSWORD + + password + + + + + + mail.smtp.starttls.enable + true + + boolean + + + + + + mail.smtp.ssl.enable + true + + boolean + + + + + + mail.smtp.ssl.trust + xxx.xxx.com + + + + + + xls.file.path + /tmp/xls + + + + + + enterprise.wechat.enable + false + + + value-list + + + true + + + + false + + + + 1 + + + + + enterprise.wechat.corp.id + wechatId + + + + + enterprise.wechat.secret + secret + + + + + enterprise.wechat.agent.id + agentId + + + + + enterprise.wechat.users + wechatUsers + + + + \ No newline at end of file diff --git a/ambari_plugin/common-services/DOLPHIN/1.2.1/configuration/dolphin-application-api.xml b/ambari_plugin/common-services/DOLPHIN/1.2.1/configuration/dolphin-application-api.xml new file mode 100644 index 0000000000..ea4cb82afd --- /dev/null +++ b/ambari_plugin/common-services/DOLPHIN/1.2.1/configuration/dolphin-application-api.xml @@ -0,0 +1,71 @@ + + + + server.port + 12345 + + server port + + + int + + + + server.servlet.session.timeout + 7200 + + int + + + + + + spring.servlet.multipart.max-file-size + 1024 + + MB + int + + + + + + spring.servlet.multipart.max-request-size + 1024 + + MB + int + + + + + + server.jetty.max-http-post-size + 5000000 + + int + + + + + + spring.messages.encoding + UTF-8 + + + \ No newline at end of file diff --git a/ambari_plugin/common-services/DOLPHIN/1.2.1/configuration/dolphin-application.xml b/ambari_plugin/common-services/DOLPHIN/1.2.1/configuration/dolphin-application.xml new file mode 100644 index 0000000000..6e50a1b649 --- /dev/null +++ b/ambari_plugin/common-services/DOLPHIN/1.2.1/configuration/dolphin-application.xml @@ -0,0 +1,467 @@ + + + + spring.datasource.initialSize + 5 + + Init connection number + + + int + + + + + spring.datasource.minIdle + 5 + + Min connection number + + + int + + + + + spring.datasource.maxActive + 50 + + Max connection number + + + int + + + + + spring.datasource.maxWait + 60000 + + Max wait time for get a connection in milliseconds. + If configuring maxWait, fair locks are enabled by default and concurrency efficiency decreases. + If necessary, unfair locks can be used by configuring the useUnfairLock attribute to true. + + + int + + + + + spring.datasource.timeBetweenEvictionRunsMillis + 60000 + + Milliseconds for check to close free connections + + + int + + + + + spring.datasource.timeBetweenConnectErrorMillis + 60000 + + The Destroy thread detects the connection interval and closes the physical connection in milliseconds + if the connection idle time is greater than or equal to minEvictableIdleTimeMillis. + + + int + + + + + spring.datasource.minEvictableIdleTimeMillis + 300000 + + The longest time a connection remains idle without being evicted, in milliseconds + + + int + + + + + spring.datasource.validationQuery + SELECT 1 + + The SQL used to check whether the connection is valid requires a query statement. + If validation Query is null, testOnBorrow, testOnReturn, and testWhileIdle will not work. + + + + + spring.datasource.validationQueryTimeout + 3 + + int + + + Check whether the connection is valid for timeout, in seconds + + + + + spring.datasource.testWhileIdle + true + + boolean + + + When applying for a connection, + if it is detected that the connection is idle longer than time Between Eviction Runs Millis, + validation Query is performed to check whether the connection is valid + + + + + spring.datasource.testOnBorrow + true + + boolean + + + Execute validation to check if the connection is valid when applying for a connection + + + + + spring.datasource.testOnReturn + false + + boolean + + + Execute validation to check if the connection is valid when the connection is returned + + + + + spring.datasource.defaultAutoCommit + true + + boolean + + + + + + + spring.datasource.keepAlive + false + + boolean + + + + + + + + spring.datasource.poolPreparedStatements + true + + boolean + + + Open PSCache, specify count PSCache for every connection + + + + + spring.datasource.maxPoolPreparedStatementPerConnectionSize + 20 + + int + + + + + + spring.datasource.spring.datasource.filters + stat,wall,log4j + + + + + spring.datasource.connectionProperties + druid.stat.mergeSql=true;druid.stat.slowSqlMillis=5000 + + + + + + mybatis-plus.mapper-locations + classpath*:/org.apache.dolphinscheduler.dao.mapper/*.xml + + + + + mybatis-plus.typeEnumsPackage + org.apache.dolphinscheduler.*.enums + + + + + mybatis-plus.typeAliasesPackage + org.apache.dolphinscheduler.dao.entity + + Entity scan, where multiple packages are separated by a comma or semicolon + + + + + mybatis-plus.global-config.db-config.id-type + AUTO + + value-list + + + AUTO + + + + INPUT + + + + ID_WORKER + + + + UUID + + + + 1 + + + Primary key type AUTO:" database ID AUTO ", + INPUT:" user INPUT ID", + ID_WORKER:" global unique ID (numeric type unique ID)", + UUID:" global unique ID UUID"; + + + + + mybatis-plus.global-config.db-config.field-strategy + NOT_NULL + + value-list + + + IGNORED + + + + NOT_NULL + + + + NOT_EMPTY + + + + 1 + + + Field policy IGNORED:" ignore judgment ", + NOT_NULL:" not NULL judgment "), + NOT_EMPTY:" not NULL judgment" + + + + + mybatis-plus.global-config.db-config.column-underline + true + + boolean + + + + + + mybatis-plus.global-config.db-config.logic-delete-value + 1 + + int + + + + + + mybatis-plus.global-config.db-config.logic-not-delete-value + 0 + + int + + + + + + mybatis-plus.global-config.db-config.banner + true + + boolean + + + + + + + mybatis-plus.configuration.map-underscore-to-camel-case + true + + boolean + + + + + + mybatis-plus.configuration.cache-enabled + false + + boolean + + + + + + mybatis-plus.configuration.call-setters-on-nulls + true + + boolean + + + + + + mybatis-plus.configuration.jdbc-type-for-null + null + + + + + master.exec.threads + 100 + + int + + + + + + master.exec.task.num + 20 + + int + + + + + + master.heartbeat.interval + 10 + + int + + + + + + master.task.commit.retryTimes + 5 + + int + + + + + + master.task.commit.interval + 1000 + + int + + + + + + master.max.cpuload.avg + 100 + + int + + + + + + master.reserved.memory + 0.1 + + float + + + + + + worker.exec.threads + 100 + + int + + + + + + worker.heartbeat.interval + 10 + + int + + + + + + worker.fetch.task.num + 3 + + int + + + + + + worker.max.cpuload.avg + 100 + + int + + + + + + worker.reserved.memory + 0.1 + + float + + + + + + \ No newline at end of file diff --git a/ambari_plugin/common-services/DOLPHIN/1.2.1/configuration/dolphin-common.xml b/ambari_plugin/common-services/DOLPHIN/1.2.1/configuration/dolphin-common.xml new file mode 100644 index 0000000000..41e2836e37 --- /dev/null +++ b/ambari_plugin/common-services/DOLPHIN/1.2.1/configuration/dolphin-common.xml @@ -0,0 +1,232 @@ + + + + dolphinscheduler.queue.impl + zookeeper + + Task queue implementation, default "zookeeper" + + + + + zookeeper.dolphinscheduler.root + /dolphinscheduler + + dolphinscheduler root directory + + + + + zookeeper.session.timeout + 300 + + int + + + + + + + zookeeper.connection.timeout + 300 + + int + + + + + + + zookeeper.retry.base.sleep + 100 + + int + + + + + + + zookeeper.retry.max.sleep + 30000 + + int + + + + + + + zookeeper.retry.maxtime + 5 + + int + + + + + + + + res.upload.startup.type + Choose Resource Upload Startup Type + + Resource upload startup type : HDFS,S3,NONE + + NONE + + value-list + + + HDFS + + + + S3 + + + + NONE + + + + 1 + + + + + hdfs.root.user + hdfs + + Users who have permission to create directories under the HDFS root path + + + + + data.store2hdfs.basepath + /dolphinscheduler + + Data base dir, resource file will store to this hadoop hdfs path, self configuration, + please make sure the directory exists on hdfs and have read write permissions。 + "/dolphinscheduler" is recommended + + + + + data.basedir.path + /tmp/dolphinscheduler + + User data directory path, self configuration, + please make sure the directory exists and have read write permissions + + + + + hadoop.security.authentication.startup.state + false + + value-list + + + true + + + + false + + + + 1 + + + + + java.security.krb5.conf.path + /opt/krb5.conf + + java.security.krb5.conf path + + + + + login.user.keytab.username + hdfs-mycluster@ESZ.COM + + LoginUserFromKeytab user + + + + + login.user.keytab.path + /opt/hdfs.headless.keytab + + LoginUserFromKeytab path + + + + + resource.view.suffixs + txt,log,sh,conf,cfg,py,java,sql,hql,xml,properties + + + + + fs.defaultFS + hdfs://mycluster:8020 + + HA or single namenode, + If namenode ha needs to copy core-site.xml and hdfs-site.xml to the conf directory, + support s3,for example : s3a://dolphinscheduler + + + + + fs.s3a.endpoint + http://host:9010 + + s3 need,s3 endpoint + + + + + fs.s3a.access.key + A3DXS30FO22544RE + + s3 need,s3 access key + + + + + fs.s3a.secret.key + OloCLq3n+8+sdPHUhJ21XrSxTC+JK + + s3 need,s3 secret key + + + + + loggerserver.rpc.port + 50051 + + intF + + + + + + \ No newline at end of file diff --git a/ambari_plugin/common-services/DOLPHIN/1.2.1/configuration/dolphin-env.xml b/ambari_plugin/common-services/DOLPHIN/1.2.1/configuration/dolphin-env.xml new file mode 100644 index 0000000000..8e14716d05 --- /dev/null +++ b/ambari_plugin/common-services/DOLPHIN/1.2.1/configuration/dolphin-env.xml @@ -0,0 +1,123 @@ + + + + dolphin.database.type + mysql + Dolphin Scheduler DataBase Type Which Is Select + Dolphin Database Type + + value-list + + + mysql + + + + postgresql + + + + 1 + + + + + + dolphin.database.host + + Dolphin Database Host + + + + + dolphin.database.port + + Dolphin Database Port + + + + + dolphin.database.username + + Dolphin Database Username + + + + + dolphin.database.password + + Dolphin Database Password + PASSWORD + + password + + + + + + dolphin.user + + Which user to install and admin dolphin scheduler + Deploy User + + + + dolphin.group + + Which user to install and admin dolphin scheduler + Deploy Group + + + + + dolphinscheduler-env-content + Dolphinscheduler Env template + This is the jinja template for dolphinscheduler.env.sh 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. +# + +export HADOOP_HOME=/opt/soft/hadoop +export HADOOP_CONF_DIR=/opt/soft/hadoop/etc/hadoop +export SPARK_HOME1=/opt/soft/spark1 +export SPARK_HOME2=/opt/soft/spark2 +export PYTHON_HOME=/opt/soft/python +export JAVA_HOME=/opt/soft/java +export HIVE_HOME=/opt/soft/hive +export FLINK_HOME=/opt/soft/flink + + content + false + false + + + + \ No newline at end of file diff --git a/ambari_plugin/common-services/DOLPHIN/1.2.1/configuration/dolphin-quartz.xml b/ambari_plugin/common-services/DOLPHIN/1.2.1/configuration/dolphin-quartz.xml new file mode 100644 index 0000000000..82b59d8827 --- /dev/null +++ b/ambari_plugin/common-services/DOLPHIN/1.2.1/configuration/dolphin-quartz.xml @@ -0,0 +1,131 @@ + + + + org.quartz.scheduler.instanceName + DolphinScheduler + + + + + org.quartz.scheduler.instanceId + AUTO + + + + org.quartz.scheduler.makeSchedulerThreadDaemon + true + + boolean + + + + + org.quartz.jobStore.useProperties + false + + boolean + + + + + org.quartz.threadPool.class + org.quartz.simpl.SimpleThreadPool + + + + org.quartz.threadPool.makeThreadsDaemons + true + + boolean + + + + + org.quartz.threadPool.threadCount + 25 + + int + + + + + org.quartz.threadPool.threadPriority + 5 + + int + + + + + org.quartz.jobStore.class + org.quartz.impl.jdbcjobstore.JobStoreTX + + + + org.quartz.jobStore.tablePrefix + QRTZ_ + + + + org.quartz.jobStore.isClustered + true + + boolean + + + + + org.quartz.jobStore.misfireThreshold + 60000 + + int + + + + + org.quartz.jobStore.clusterCheckinInterval + 5000 + + int + + + + + org.quartz.jobStore.dataSource + myDs + + + + org.quartz.dataSource.myDs.connectionProvider.class + org.apache.dolphinscheduler.server.quartz.DruidConnectionProvider + + + + org.quartz.dataSource.myDs.maxConnections + 10 + + int + + + + + org.quartz.dataSource.myDs.validationQuery + select 1 + + + \ No newline at end of file diff --git a/ambari_plugin/common-services/DOLPHIN/1.2.1/metainfo.xml b/ambari_plugin/common-services/DOLPHIN/1.2.1/metainfo.xml new file mode 100644 index 0000000000..0d2bbe3163 --- /dev/null +++ b/ambari_plugin/common-services/DOLPHIN/1.2.1/metainfo.xml @@ -0,0 +1,137 @@ + + + + 2.0 + + + DOLPHIN + Dolphin Scheduler + 分布式易扩展的可视化DAG工作流任务调度系统 + 1.2.1 + + + DOLPHIN_MASTER + DS Master + MASTER + 1+ + + + PYTHON + 600 + + + + + DOLPHIN_LOGGER + DS Logger + SLAVE + 1+ + + + PYTHON + 600 + + + + + DOLPHIN_WORKER + DS Worker + SLAVE + 1+ + + + DOLPHIN/DOLPHIN_LOGGER + host + + true + + + + + + PYTHON + 600 + + + + + DOLPHIN_ALERT + DS Alert + SLAVE + 1 + + + PYTHON + 600 + + + + + DOLPHIN_API + DS_Api + SLAVE + 1 + + + PYTHON + 600 + + + + + + ZOOKEEPER + + + + + any + + + apache-dolphinscheduler-incubating-1.2.1* + + + + + + + dolphin-alert + dolphin-app-api + dolphin-app-dao + dolphin-common + dolphin-env + dolphin-quartz + + + + + theme.json + true + + + + quicklinks + + + quicklinks.json + true + + + + + diff --git a/ambari_plugin/common-services/DOLPHIN/1.2.1/package/alerts/alert_dolphin_scheduler_status.py b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/alerts/alert_dolphin_scheduler_status.py new file mode 100644 index 0000000000..87cc7b453b --- /dev/null +++ b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/alerts/alert_dolphin_scheduler_status.py @@ -0,0 +1,124 @@ +""" +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +""" + +import socket +import urllib2 +import os +import logging +import ambari_simplejson as json +from resource_management.libraries.script.script import Script +import sys +reload(sys) +sys.setdefaultencoding('utf-8') + +logger = logging.getLogger('ambari_alerts') + +config = Script.get_config() + + +def get_tokens(): + """ + Returns a tuple of tokens in the format {{site/property}} that will be used + to build the dictionary passed into execute + + :rtype tuple + """ + +def get_info(url, connection_timeout): + response = None + + try: + response = urllib2.urlopen(url, timeout=connection_timeout) + json_data = response.read() + return json_data + finally: + if response is not None: + try: + response.close() + except: + pass + + +def execute(configurations={}, parameters={}, host_name=None): + """ + Returns a tuple containing the result code and a pre-formatted result label + + Keyword arguments: + configurations : a mapping of configuration key to value + parameters : a mapping of script parameter key to value + host_name : the name of this host where the alert is running + + :type configurations dict + :type parameters dict + :type host_name str + """ + + alert_name = parameters['alertName'] + + dolphin_pidfile_dir = "/opt/soft/run/dolphinscheduler" + + pid = "0" + + + from resource_management.core import sudo + + is_running = True + pid_file_path = "" + if alert_name == 'DOLPHIN_MASTER': + pid_file_path = dolphin_pidfile_dir + "/master-server.pid" + elif alert_name == 'DOLPHIN_WORKER': + pid_file_path = dolphin_pidfile_dir + "/worker-server.pid" + elif alert_name == 'DOLPHIN_ALERT': + pid_file_path = dolphin_pidfile_dir + "/alert-server.pid" + elif alert_name == 'DOLPHIN_LOGGER': + pid_file_path = dolphin_pidfile_dir + "/logger-server.pid" + elif alert_name == 'DOLPHIN_API': + pid_file_path = dolphin_pidfile_dir + "/api-server.pid" + + if not pid_file_path or not os.path.isfile(pid_file_path): + is_running = False + + try: + pid = int(sudo.read_file(pid_file_path)) + except: + is_running = False + + try: + # Kill will not actually kill the process + # From the doc: + # If sig is 0, then no signal is sent, but error checking is still + # performed; this can be used to check for the existence of a + # process ID or process group ID. + sudo.kill(pid, 0) + except OSError: + is_running = False + + if host_name is None: + host_name = socket.getfqdn() + + if not is_running: + result_code = "CRITICAL" + else: + result_code = "OK" + + label = "The comment {0} of DOLPHIN_SCHEDULER on {1} is {2}".format(alert_name, host_name, result_code) + + return ((result_code, [label])) + +if __name__ == "__main__": + pass diff --git a/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/dolphin_alert_service.py b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/dolphin_alert_service.py new file mode 100644 index 0000000000..62255a3432 --- /dev/null +++ b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/dolphin_alert_service.py @@ -0,0 +1,61 @@ +""" +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +""" +import time +from resource_management import * + +from dolphin_env import dolphin_env + + +class DolphinAlertService(Script): + def install(self, env): + import params + env.set_params(params) + self.install_packages(env) + Execute(('chmod', '-R', '777', params.dolphin_home), user=params.dolphin_user, sudo=True) + + def configure(self, env): + import params + params.pika_slave = True + env.set_params(params) + + dolphin_env() + + def start(self, env): + import params + env.set_params(params) + self.configure(env) + no_op_test = format("ls {dolphin_pidfile_dir}/alert-server.pid >/dev/null 2>&1 && ps `cat {dolphin_pidfile_dir}/alert-server.pid` | grep `cat {dolphin_pidfile_dir}/alert-server.pid` >/dev/null 2>&1") + + start_cmd = format("sh " + params.dolphin_bin_dir + "/dolphinscheduler-daemon.sh start alert-server") + Execute(start_cmd, user=params.dolphin_user, not_if=no_op_test) + + def stop(self, env): + import params + env.set_params(params) + stop_cmd = format("sh " + params.dolphin_bin_dir + "/dolphinscheduler-daemon.sh stop alert-server") + Execute(stop_cmd, user=params.dolphin_user) + time.sleep(5) + + def status(self, env): + import status_params + env.set_params(status_params) + check_process_status(status_params.dolphin_run_dir + "alert-server.pid") + + +if __name__ == "__main__": + DolphinAlertService().execute() diff --git a/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/dolphin_api_service.py b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/dolphin_api_service.py new file mode 100644 index 0000000000..bdc18fb602 --- /dev/null +++ b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/dolphin_api_service.py @@ -0,0 +1,70 @@ +""" +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +""" +import time +from resource_management import * + +from dolphin_env import dolphin_env + + +class DolphinApiService(Script): + def install(self, env): + import params + env.set_params(params) + self.install_packages(env) + Execute(('chmod', '-R', '777', params.dolphin_home), user=params.dolphin_user, sudo=True) + + def configure(self, env): + import params + params.pika_slave = True + env.set_params(params) + + dolphin_env() + + def start(self, env): + import params + env.set_params(params) + self.configure(env) + + #init + init_cmd=format("sh " + params.dolphin_home + "/script/create-dolphinscheduler.sh") + Execute(init_cmd, user=params.dolphin_user) + + #upgrade + upgrade_cmd=format("sh " + params.dolphin_home + "/script/upgrade-dolphinscheduler.sh") + Execute(upgrade_cmd, user=params.dolphin_user) + + no_op_test = format("ls {dolphin_pidfile_dir}/api-server.pid >/dev/null 2>&1 && ps `cat {dolphin_pidfile_dir}/api-server.pid` | grep `cat {dolphin_pidfile_dir}/api-server.pid` >/dev/null 2>&1") + + start_cmd = format("sh " + params.dolphin_bin_dir + "/dolphinscheduler-daemon.sh start api-server") + Execute(start_cmd, user=params.dolphin_user, not_if=no_op_test) + + def stop(self, env): + import params + env.set_params(params) + stop_cmd = format("sh " + params.dolphin_bin_dir + "/dolphinscheduler-daemon.sh stop api-server") + Execute(stop_cmd, user=params.dolphin_user) + time.sleep(5) + + def status(self, env): + import status_params + env.set_params(status_params) + check_process_status(status_params.dolphin_run_dir + "api-server.pid") + + +if __name__ == "__main__": + DolphinApiService().execute() diff --git a/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/dolphin_env.py b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/dolphin_env.py new file mode 100644 index 0000000000..235605894f --- /dev/null +++ b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/dolphin_env.py @@ -0,0 +1,121 @@ +""" +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. + +""" + +from resource_management import * + + +def dolphin_env(): + import params + + Directory(params.dolphin_pidfile_dir, + mode=0777, + owner=params.dolphin_user, + group=params.dolphin_group, + create_parents=True + ) + Directory(params.dolphin_log_dir, + mode=0777, + owner=params.dolphin_user, + group=params.dolphin_group, + create_parents=True + ) + Directory(params.dolphin_conf_dir, + mode=0777, + owner=params.dolphin_user, + group=params.dolphin_group, + create_parents=True + ) + + + Directory(params.dolphin_alert_map['xls.file.path'], + mode=0777, + owner=params.dolphin_user, + group=params.dolphin_group, + create_parents=True + ) + Directory(params.dolphin_common_map['data.basedir.path'], + mode=0777, + owner=params.dolphin_user, + group=params.dolphin_group, + create_parents=True + ) + Directory(params.dolphin_common_map['data.download.basedir.path'], + mode=0777, + owner=params.dolphin_user, + group=params.dolphin_group, + create_parents=True + ) + Directory(params.dolphin_common_map['process.exec.basepath'], + mode=0777, + owner=params.dolphin_user, + group=params.dolphin_group, + create_parents=True + ) + + + File(format(params.dolphin_env_path), + mode=0777, + content=InlineTemplate(params.dolphin_env_content), + owner=params.dolphin_user, + group=params.dolphin_group + ) + + + File(format(params.dolphin_bin_dir + "/dolphinscheduler-daemon.sh"), + mode=0755, + content=Template("dolphin-daemon.j2"), + owner=params.dolphin_user, + group=params.dolphin_group + ) + + + File(format(params.dolphin_conf_dir + "/alert.properties"), + mode=0755, + content=Template("alert.properties.j2"), + owner=params.dolphin_user, + group=params.dolphin_group + ) + + File(format(params.dolphin_conf_dir + "/application.properties"), + mode=0755, + content=Template("application.properties.j2"), + owner=params.dolphin_user, + group=params.dolphin_group + ) + + File(format(params.dolphin_conf_dir + "/application-api.properties"), + mode=0755, + content=Template("application-api.properties.j2"), + owner=params.dolphin_user, + group=params.dolphin_group + ) + + File(format(params.dolphin_conf_dir + "/common.properties"), + mode=0755, + content=Template("common.properties.j2"), + owner=params.dolphin_user, + group=params.dolphin_group + ) + + File(format(params.dolphin_conf_dir + "/quartz.properties"), + mode=0755, + content=Template("quartz.properties.j2"), + owner=params.dolphin_user, + group=params.dolphin_group + ) diff --git a/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/dolphin_logger_service.py b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/dolphin_logger_service.py new file mode 100644 index 0000000000..f1c19bd66f --- /dev/null +++ b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/dolphin_logger_service.py @@ -0,0 +1,61 @@ +""" +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +""" +import time +from resource_management import * + +from dolphin_env import dolphin_env + + +class DolphinLoggerService(Script): + def install(self, env): + import params + env.set_params(params) + self.install_packages(env) + Execute(('chmod', '-R', '777', params.dolphin_home), user=params.dolphin_user, sudo=True) + + def configure(self, env): + import params + params.pika_slave = True + env.set_params(params) + + dolphin_env() + + def start(self, env): + import params + env.set_params(params) + self.configure(env) + no_op_test = format("ls {dolphin_pidfile_dir}/logger-server.pid >/dev/null 2>&1 && ps `cat {dolphin_pidfile_dir}/logger-server.pid` | grep `cat {dolphin_pidfile_dir}/logger-server.pid` >/dev/null 2>&1") + + start_cmd = format("sh " + params.dolphin_bin_dir + "/dolphinscheduler-daemon.sh start logger-server") + Execute(start_cmd, user=params.dolphin_user, not_if=no_op_test) + + def stop(self, env): + import params + env.set_params(params) + stop_cmd = format("sh " + params.dolphin_bin_dir + "/dolphinscheduler-daemon.sh stop logger-server") + Execute(stop_cmd, user=params.dolphin_user) + time.sleep(5) + + def status(self, env): + import status_params + env.set_params(status_params) + check_process_status(status_params.dolphin_run_dir + "logger-server.pid") + + +if __name__ == "__main__": + DolphinLoggerService().execute() diff --git a/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/dolphin_master_service.py b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/dolphin_master_service.py new file mode 100644 index 0000000000..6ee7ecfcf3 --- /dev/null +++ b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/dolphin_master_service.py @@ -0,0 +1,61 @@ +# -*- coding: utf-8 -*- +""" +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +""" +import time +from resource_management import * + +from dolphin_env import dolphin_env + + +class DolphinMasterService(Script): + def install(self, env): + import params + env.set_params(params) + self.install_packages(env) + Execute(('chmod', '-R', '777', params.dolphin_home), user=params.dolphin_user, sudo=True) + + def configure(self, env): + import params + params.pika_slave = True + env.set_params(params) + + dolphin_env() + + def start(self, env): + import params + env.set_params(params) + self.configure(env) + no_op_test = format("ls {dolphin_pidfile_dir}/master-server.pid >/dev/null 2>&1 && ps `cat {dolphin_pidfile_dir}/master-server.pid` | grep `cat {dolphin_pidfile_dir}/master-server.pid` >/dev/null 2>&1") + start_cmd = format("sh " + params.dolphin_bin_dir + "/dolphinscheduler-daemon.sh start master-server") + Execute(start_cmd, user=params.dolphin_user, not_if=no_op_test) + + def stop(self, env): + import params + env.set_params(params) + stop_cmd = format("sh " + params.dolphin_bin_dir + "/dolphinscheduler-daemon.sh stop master-server") + Execute(stop_cmd, user=params.dolphin_user) + time.sleep(5) + + def status(self, env): + import status_params + env.set_params(status_params) + check_process_status(status_params.dolphin_run_dir + "master-server.pid") + + +if __name__ == "__main__": + DolphinMasterService().execute() diff --git a/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/dolphin_worker_service.py b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/dolphin_worker_service.py new file mode 100644 index 0000000000..2d145ee730 --- /dev/null +++ b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/dolphin_worker_service.py @@ -0,0 +1,60 @@ +""" +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +""" +import time +from resource_management import * + +from dolphin_env import dolphin_env + + +class DolphinWorkerService(Script): + def install(self, env): + import params + env.set_params(params) + self.install_packages(env) + Execute(('chmod', '-R', '777', params.dolphin_home), user=params.dolphin_user, sudo=True) + + def configure(self, env): + import params + params.pika_slave = True + env.set_params(params) + + dolphin_env() + + def start(self, env): + import params + env.set_params(params) + self.configure(env) + no_op_test = format("ls {dolphin_pidfile_dir}/worker-server.pid >/dev/null 2>&1 && ps `cat {dolphin_pidfile_dir}/worker-server.pid` | grep `cat {dolphin_pidfile_dir}/worker-server.pid` >/dev/null 2>&1") + start_cmd = format("sh " + params.dolphin_bin_dir + "/dolphinscheduler-daemon.sh start worker-server") + Execute(start_cmd, user=params.dolphin_user, not_if=no_op_test) + + def stop(self, env): + import params + env.set_params(params) + stop_cmd = format("sh " + params.dolphin_bin_dir + "/dolphinscheduler-daemon.sh stop worker-server") + Execute(stop_cmd, user=params.dolphin_user) + time.sleep(5) + + def status(self, env): + import status_params + env.set_params(status_params) + check_process_status(status_params.dolphin_run_dir + "worker-server.pid") + + +if __name__ == "__main__": + DolphinWorkerService().execute() diff --git a/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/params.py b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/params.py new file mode 100644 index 0000000000..230ad14565 --- /dev/null +++ b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/params.py @@ -0,0 +1,150 @@ +""" +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE_2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + +""" + +import sys +from resource_management import * +from resource_management.core.logger import Logger +from resource_management.libraries.functions import default + +Logger.initialize_logger() +reload(sys) +sys.setdefaultencoding('utf-8') + +# server configurations +config = Script.get_config() + +# conf_dir = "/etc/" +dolphin_home = "/opt/soft/apache-dolphinscheduler-incubating-1.2.1" +dolphin_conf_dir = dolphin_home + "/conf" +dolphin_log_dir = dolphin_home + "/logs" +dolphin_bin_dir = dolphin_home + "/bin" +dolphin_lib_jars = dolphin_home + "/lib/*" +dolphin_pidfile_dir = "/opt/soft/run/dolphinscheduler" + +rmHosts = default("/clusterHostInfo/rm_host", []) + +# dolphin-env +dolphin_env_map = {} +dolphin_env_map.update(config['configurations']['dolphin-env']) + +# which user to install and admin dolphin scheduler +dolphin_user = dolphin_env_map['dolphin.user'] +dolphin_group = dolphin_env_map['dolphin.group'] + +# .dolphinscheduler_env.sh +dolphin_env_path = dolphin_conf_dir + '/env/dolphinscheduler_env.sh' +dolphin_env_content = dolphin_env_map['dolphinscheduler-env-content'] + +# database config +dolphin_database_config = {} +dolphin_database_config['dolphin_database_type'] = dolphin_env_map['dolphin.database.type'] +dolphin_database_config['dolphin_database_host'] = dolphin_env_map['dolphin.database.host'] +dolphin_database_config['dolphin_database_port'] = dolphin_env_map['dolphin.database.port'] +dolphin_database_config['dolphin_database_username'] = dolphin_env_map['dolphin.database.username'] +dolphin_database_config['dolphin_database_password'] = dolphin_env_map['dolphin.database.password'] + +if 'mysql' == dolphin_database_config['dolphin_database_type']: + dolphin_database_config['dolphin_database_driver'] = 'com.mysql.jdbc.Driver' + dolphin_database_config['driverDelegateClass'] = 'org.quartz.impl.jdbcjobstore.StdJDBCDelegate' + dolphin_database_config['dolphin_database_url'] = 'jdbc:mysql://' + dolphin_env_map['dolphin.database.host'] \ + + ':' + dolphin_env_map['dolphin.database.port'] \ + + '/dolphinscheduler?useUnicode=true&characterEncoding=UTF-8' +else: + dolphin_database_config['dolphin_database_driver'] = 'org.postgresql.Driver' + dolphin_database_config['driverDelegateClass'] = 'org.quartz.impl.jdbcjobstore.PostgreSQLDelegate' + dolphin_database_config['dolphin_database_url'] = 'jdbc:postgresql://' + dolphin_env_map['dolphin.database.host'] \ + + ':' + dolphin_env_map['dolphin.database.port'] \ + + '/dolphinscheduler' + +# application-alert.properties +dolphin_alert_map = {} +wechat_push_url = 'https://qyapi.weixin.qq.com/cgi-bin/message/send?access_token=$token' +wechat_token_url = 'https://qyapi.weixin.qq.com/cgi-bin/gettoken?corpid=$corpId&corpsecret=$secret' +wechat_team_send_msg = '{\"toparty\":\"$toParty\",\"agentid\":\"$agentId\",\"msgtype\":\"text\",\"text\":{\"content\":\"$msg\"},\"safe\":\"0\"}' +wechat_user_send_msg = '{\"touser\":\"$toUser\",\"agentid\":\"$agentId\",\"msgtype\":\"markdown\",\"markdown\":{\"content\":\"$msg\"}}' + +dolphin_alert_map['enterprise.wechat.push.ur'] = wechat_push_url +dolphin_alert_map['enterprise.wechat.token.url'] = wechat_token_url +dolphin_alert_map['enterprise.wechat.team.send.msg'] = wechat_team_send_msg +dolphin_alert_map['enterprise.wechat.user.send.msg'] = wechat_user_send_msg +dolphin_alert_map.update(config['configurations']['dolphin-alert']) + +# application-api.properties +dolphin_app_api_map = {} +dolphin_app_api_map['logging.config'] = 'classpath:apiserver_logback.xml' +dolphin_app_api_map['spring.messages.basename'] = 'i18n/messages' +dolphin_app_api_map['server.servlet.context-path'] = '/dolphinscheduler/' +dolphin_app_api_map.update(config['configurations']['dolphin-application-api']) + +# application-dao.properties +dolphin_application_map = {} +dolphin_application_map['spring.datasource.type'] = 'com.alibaba.druid.pool.DruidDataSource' +dolphin_application_map['spring.datasource.driver-class-name'] = dolphin_database_config['dolphin_database_driver'] +dolphin_application_map['spring.datasource.url'] = dolphin_database_config['dolphin_database_url'] +dolphin_application_map['spring.datasource.username'] = dolphin_database_config['dolphin_database_username'] +dolphin_application_map['spring.datasource.password'] = dolphin_database_config['dolphin_database_password'] +dolphin_application_map.update(config['configurations']['dolphin-application']) + +# common.properties +dolphin_common_map = {} + +if 'yarn-site' in config['configurations'] and \ + 'yarn.resourcemanager.webapp.address' in config['configurations']['yarn-site']: + yarn_resourcemanager_webapp_address = config['configurations']['yarn-site']['yarn.resourcemanager.webapp.address'] + yarn_application_status_address = 'http://' + yarn_resourcemanager_webapp_address + '/ws/v1/cluster/apps/%s' + dolphin_common_map['yarn.application.status.address'] = yarn_application_status_address + +rmHosts = default("/clusterHostInfo/rm_host", []) +if len(rmHosts) > 1: + dolphin_common_map['yarn.resourcemanager.ha.rm.ids'] = ','.join(rmHosts) +else: + dolphin_common_map['yarn.resourcemanager.ha.rm.ids'] = '' + +dolphin_common_map_tmp = config['configurations']['dolphin-common'] +data_basedir_path = dolphin_common_map_tmp['data.basedir.path'] +process_exec_basepath = data_basedir_path + '/exec' +data_download_basedir_path = data_basedir_path + '/download' +dolphin_common_map['process.exec.basepath'] = process_exec_basepath +dolphin_common_map['data.download.basedir.path'] = data_download_basedir_path +dolphin_common_map['dolphinscheduler.env.path'] = dolphin_env_path + +zookeeperHosts = default("/clusterHostInfo/zookeeper_hosts", []) +if len(zookeeperHosts) > 0 and "clientPort" in config['configurations']['zoo.cfg']: + clientPort = config['configurations']['zoo.cfg']['clientPort'] + zookeeperPort = ":" + clientPort + "," + dolphin_common_map['zookeeper.quorum'] = zookeeperPort.join(zookeeperHosts) + ":" + clientPort + +dolphin_common_map.update(config['configurations']['dolphin-common']) + +# quartz.properties +dolphin_quartz_map = {} +dolphin_quartz_map['org.quartz.jobStore.driverDelegateClass'] = dolphin_database_config['driverDelegateClass'] +dolphin_quartz_map['org.quartz.dataSource.myDs.driver'] = dolphin_database_config['dolphin_database_driver'] +dolphin_quartz_map['org.quartz.dataSource.myDs.URL'] = dolphin_database_config['dolphin_database_url'] +dolphin_quartz_map['org.quartz.dataSource.myDs.user'] = dolphin_database_config['dolphin_database_username'] +dolphin_quartz_map['org.quartz.dataSource.myDs.password'] = dolphin_database_config['dolphin_database_password'] +dolphin_quartz_map.update(config['configurations']['dolphin-quartz']) + +# if 'ganglia_server_host' in config['clusterHostInfo'] and \ +# len(config['clusterHostInfo']['ganglia_server_host'])>0: +# ganglia_installed = True +# ganglia_server = config['clusterHostInfo']['ganglia_server_host'][0] +# ganglia_report_interval = 60 +# else: +# ganglia_installed = False diff --git a/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/service_check.py b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/service_check.py new file mode 100644 index 0000000000..0e12f69932 --- /dev/null +++ b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/service_check.py @@ -0,0 +1,31 @@ +""" +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. + +""" + +from resource_management import * +from resource_management.libraries.functions import get_unique_id_and_date + +class ServiceCheck(Script): + def service_check(self, env): + import params + #env.set_params(params) + + # Execute(format("which pika_server")) + +if __name__ == "__main__": + ServiceCheck().execute() diff --git a/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/status_params.py b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/status_params.py new file mode 100644 index 0000000000..24b2c8b1bc --- /dev/null +++ b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/status_params.py @@ -0,0 +1,23 @@ +""" +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. +""" + +from resource_management import * + +config = Script.get_config() + +dolphin_run_dir = "/opt/soft/run/dolphinscheduler/" diff --git a/ambari_plugin/common-services/DOLPHIN/1.2.1/package/templates/alert.properties.j2 b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/templates/alert.properties.j2 new file mode 100644 index 0000000000..73840b8c18 --- /dev/null +++ b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/templates/alert.properties.j2 @@ -0,0 +1,20 @@ +# +# 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. +# + +{% for key, value in dolphin_alert_map.iteritems() -%} + {{key}}={{value}} +{% endfor %} \ No newline at end of file diff --git a/ambari_plugin/common-services/DOLPHIN/1.2.1/package/templates/application-api.properties.j2 b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/templates/application-api.properties.j2 new file mode 100644 index 0000000000..70118003b9 --- /dev/null +++ b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/templates/application-api.properties.j2 @@ -0,0 +1,20 @@ +# +# 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. +# + +{% for key, value in dolphin_app_api_map.iteritems() -%} + {{key}}={{value}} +{% endfor %} \ No newline at end of file diff --git a/ambari_plugin/common-services/DOLPHIN/1.2.1/package/templates/application.properties.j2 b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/templates/application.properties.j2 new file mode 100644 index 0000000000..7bb9f8aff3 --- /dev/null +++ b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/templates/application.properties.j2 @@ -0,0 +1,20 @@ +# +# 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. +# + +{% for key, value in dolphin_application_map.iteritems() -%} + {{key}}={{value}} +{% endfor %} \ No newline at end of file diff --git a/ambari_plugin/common-services/DOLPHIN/1.2.1/package/templates/common.properties.j2 b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/templates/common.properties.j2 new file mode 100644 index 0000000000..2220c4effa --- /dev/null +++ b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/templates/common.properties.j2 @@ -0,0 +1,20 @@ +# +# 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. +# + +{% for key, value in dolphin_common_map.iteritems() -%} + {{key}}={{value}} +{% endfor %} \ No newline at end of file diff --git a/ambari_plugin/common-services/DOLPHIN/1.2.1/package/templates/dolphin-daemon.j2 b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/templates/dolphin-daemon.j2 new file mode 100644 index 0000000000..ab99ffda47 --- /dev/null +++ b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/templates/dolphin-daemon.j2 @@ -0,0 +1,119 @@ +#!/bin/sh +# +# 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. +# + +usage="Usage: dolphinscheduler-daemon.sh (start|stop) " + +# if no args specified, show usage +if [ $# -le 1 ]; then + echo $usage + exit 1 +fi + +startStop=$1 +shift +command=$1 +shift + +echo "Begin $startStop $command......" + +BIN_DIR=`dirname $0` +BIN_DIR=`cd "$BIN_DIR"; pwd` +DOLPHINSCHEDULER_HOME=$BIN_DIR/.. + +export HOSTNAME=`hostname` + +DOLPHINSCHEDULER_LIB_JARS={{dolphin_lib_jars}} + +DOLPHINSCHEDULER_OPTS="-server -Xmx16g -Xms4g -Xss512k -XX:+DisableExplicitGC -XX:+UseConcMarkSweepGC -XX:+CMSParallelRemarkEnabled -XX:LargePageSizeInBytes=128m -XX:+UseFastAccessorMethods -XX:+UseCMSInitiatingOccupancyOnly -XX:CMSInitiatingOccupancyFraction=70" +STOP_TIMEOUT=5 + +log={{dolphin_log_dir}}/dolphinscheduler-$command-$HOSTNAME.out +pid={{dolphin_pidfile_dir}}/$command.pid + +cd $DOLPHINSCHEDULER_HOME + +if [ "$command" = "api-server" ]; then + LOG_FILE="-Dlogging.config={{dolphin_conf_dir}}/apiserver_logback.xml -Dspring.profiles.active=api" + CLASS=org.apache.dolphinscheduler.api.ApiApplicationServer +elif [ "$command" = "master-server" ]; then + LOG_FILE="-Dlogging.config={{dolphin_conf_dir}}/master_logback.xml -Ddruid.mysql.usePingMethod=false" + CLASS=org.apache.dolphinscheduler.server.master.MasterServer +elif [ "$command" = "worker-server" ]; then + LOG_FILE="-Dlogging.config={{dolphin_conf_dir}}/worker_logback.xml -Ddruid.mysql.usePingMethod=false" + CLASS=org.apache.dolphinscheduler.server.worker.WorkerServer +elif [ "$command" = "alert-server" ]; then + LOG_FILE="-Dlogback.configurationFile={{dolphin_conf_dir}}/alert_logback.xml" + CLASS=org.apache.dolphinscheduler.alert.AlertServer +elif [ "$command" = "logger-server" ]; then + CLASS=org.apache.dolphinscheduler.server.rpc.LoggerServer +elif [ "$command" = "combined-server" ]; then + LOG_FILE="-Dlogging.config={{dolphin_conf_dir}}/combined_logback.xml -Dspring.profiles.active=api -Dserver.is-combined-server=true" + CLASS=org.apache.dolphinscheduler.api.CombinedApplicationServer +else + echo "Error: No command named \`$command' was found." + exit 1 +fi + +case $startStop in + (start) + + if [ -f $pid ]; then + if kill -0 `cat $pid` > /dev/null 2>&1; then + echo $command running as process `cat $pid`. Stop it first. + exit 1 + fi + fi + + echo starting $command, logging to $log + + exec_command="$LOG_FILE $DOLPHINSCHEDULER_OPTS -classpath {{dolphin_conf_dir}}:{{dolphin_lib_jars}} $CLASS" + + echo "nohup java $exec_command > $log 2>&1 < /dev/null &" + nohup java $exec_command > $log 2>&1 < /dev/null & + echo $! > $pid + ;; + + (stop) + + if [ -f $pid ]; then + TARGET_PID=`cat $pid` + if kill -0 $TARGET_PID > /dev/null 2>&1; then + echo stopping $command + kill $TARGET_PID + sleep $STOP_TIMEOUT + if kill -0 $TARGET_PID > /dev/null 2>&1; then + echo "$command did not stop gracefully after $STOP_TIMEOUT seconds: killing with kill -9" + kill -9 $TARGET_PID + fi + else + echo no $command to stop + fi + rm -f $pid + else + echo no $command to stop + fi + ;; + + (*) + echo $usage + exit 1 + ;; + +esac + +echo "End $startStop $command." \ No newline at end of file diff --git a/ambari_plugin/common-services/DOLPHIN/1.2.1/package/templates/quartz.properties.j2 b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/templates/quartz.properties.j2 new file mode 100644 index 0000000000..e027a263b5 --- /dev/null +++ b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/templates/quartz.properties.j2 @@ -0,0 +1,20 @@ +# +# 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. +# + +{% for key, value in dolphin_quartz_map.iteritems() -%} + {{key}}={{value}} +{% endfor %} \ No newline at end of file diff --git a/ambari_plugin/common-services/DOLPHIN/1.2.1/quicklinks/quicklinks.json b/ambari_plugin/common-services/DOLPHIN/1.2.1/quicklinks/quicklinks.json new file mode 100755 index 0000000000..8753004fef --- /dev/null +++ b/ambari_plugin/common-services/DOLPHIN/1.2.1/quicklinks/quicklinks.json @@ -0,0 +1,26 @@ +{ + "name": "default", + "description": "default quick links configuration", + "configuration": { + "protocol": + { + "type":"http" + }, + + "links": [ + { + "name": "dolphin-application-ui", + "label": "DolphinApplication UI", + "requires_user_name": "false", + "component_name": "DOLPHIN_API", + "url": "%@://%@:%@/dolphinscheduler/ui/view/login/index.html", + "port":{ + "http_property": "server.port", + "http_default_port": "12345", + "regex": "^(\\d+)$", + "site": "dolphin-application-api" + } + } + ] + } +} \ No newline at end of file diff --git a/ambari_plugin/common-services/DOLPHIN/1.2.1/themes/theme.json b/ambari_plugin/common-services/DOLPHIN/1.2.1/themes/theme.json new file mode 100644 index 0000000000..23e46076aa --- /dev/null +++ b/ambari_plugin/common-services/DOLPHIN/1.2.1/themes/theme.json @@ -0,0 +1,605 @@ +{ + "name": "default", + "description": "Default theme for Dolphin Scheduler service", + "configuration": { + "layouts": [ + { + "name": "default", + "tabs": [ + { + "name": "settings", + "display-name": "Settings", + "layout": { + "tab-rows": "3", + "tab-columns": "3", + "sections": [ + { + "name": "dolphin-env-config", + "display-name": "Dolphin Env Config", + "row-index": "0", + "column-index": "0", + "row-span": "1", + "column-span": "2", + "section-rows": "1", + "section-columns": "2", + "subsections": [ + { + "name": "env-row1-col1", + "display-name": "Deploy User Info", + "row-index": "0", + "column-index": "0", + "row-span": "1", + "column-span": "1" + }, + { + "name": "env-row1-col2", + "display-name": "System Env Optimization", + "row-index": "0", + "column-index": "1", + "row-span": "1", + "column-span": "1" + } + ] + }, + { + "name": "dolphin-database-config", + "display-name": "Database Config", + "row-index": "1", + "column-index": "0", + "row-span": "1", + "column-span": "2", + "section-rows": "1", + "section-columns": "3", + "subsections": [ + { + "name": "database-row1-col1", + "row-index": "0", + "column-index": "0", + "row-span": "1", + "column-span": "1" + }, + { + "name": "database-row1-col2", + "row-index": "0", + "column-index": "1", + "row-span": "1", + "column-span": "1" + }, + { + "name": "database-row1-col3", + "row-index": "0", + "column-index": "2", + "row-span": "1", + "column-span": "1" + } + ] + }, + { + "name": "dynamic-config", + "row-index": "2", + "column-index": "0", + "row-span": "1", + "column-span": "2", + "section-rows": "1", + "section-columns": "3", + "subsections": [ + { + "name": "dynamic-row1-col1", + "display-name": "Resource FS Config", + "row-index": "0", + "column-index": "0", + "row-span": "1", + "column-span": "1" + }, + { + "name": "dynamic-row1-col2", + "display-name": "Kerberos Info", + "row-index": "0", + "column-index": "1", + "row-span": "1", + "column-span": "1" + }, + { + "name": "dynamic-row1-col3", + "display-name": "Wechat Info", + "row-index": "0", + "column-index": "1", + "row-span": "1", + "column-span": "1" + } + ] + } + ] + } + } + ] + } + ], + "placement": { + "configuration-layout": "default", + "configs": [ + { + "config": "dolphin-env/dolphin.database.type", + "subsection-name": "database-row1-col1" + }, + { + "config": "dolphin-env/dolphin.database.host", + "subsection-name": "database-row1-col2" + }, + { + "config": "dolphin-env/dolphin.database.port", + "subsection-name": "database-row1-col2" + }, + { + "config": "dolphin-env/dolphin.database.username", + "subsection-name": "database-row1-col3" + }, + { + "config": "dolphin-env/dolphin.database.password", + "subsection-name": "database-row1-col3" + }, + { + "config": "dolphin-env/dolphin.user", + "subsection-name": "env-row1-col1" + }, + { + "config": "dolphin-env/dolphin.group", + "subsection-name": "env-row1-col1" + }, + { + "config": "dolphin-env/dolphinscheduler-env-content", + "subsection-name": "env-row1-col2" + }, + { + "config": "dolphin-common/res.upload.startup.type", + "subsection-name": "dynamic-row1-col1" + }, + { + "config": "dolphin-common/hdfs.root.user", + "subsection-name": "dynamic-row1-col1", + "depends-on": [ + { + "configs":[ + "dolphin-common/res.upload.startup.type" + ], + "if": "${dolphin-common/res.upload.startup.type} === HDFS", + "then": { + "property_value_attributes": { + "visible": true + } + }, + "else": { + "property_value_attributes": { + "visible": false + } + } + } + ] + }, + { + "config": "dolphin-common/data.store2hdfs.basepath", + "subsection-name": "dynamic-row1-col1", + "depends-on": [ + { + "configs":[ + "dolphin-common/res.upload.startup.type" + ], + "if": "${dolphin-common/res.upload.startup.type} === HDFS", + "then": { + "property_value_attributes": { + "visible": true + } + }, + "else": { + "property_value_attributes": { + "visible": false + } + } + } + ] + }, + { + "config": "dolphin-common/fs.defaultFS", + "subsection-name": "dynamic-row1-col1", + "depends-on": [ + { + "configs":[ + "dolphin-common/res.upload.startup.type" + ], + "if": "${dolphin-common/res.upload.startup.type} === HDFS", + "then": { + "property_value_attributes": { + "visible": true + } + }, + "else": { + "property_value_attributes": { + "visible": false + } + } + } + ] + }, + { + "config": "dolphin-common/fs.s3a.endpoint", + "subsection-name": "dynamic-row1-col1", + "depends-on": [ + { + "configs":[ + "dolphin-common/res.upload.startup.type" + ], + "if": "${dolphin-common/res.upload.startup.type} === S3", + "then": { + "property_value_attributes": { + "visible": true + } + }, + "else": { + "property_value_attributes": { + "visible": false + } + } + } + ] + }, + { + "config": "dolphin-common/fs.s3a.access.key", + "subsection-name": "dynamic-row1-col1", + "depends-on": [ + { + "configs":[ + "dolphin-common/res.upload.startup.type" + ], + "if": "${dolphin-common/res.upload.startup.type} === S3", + "then": { + "property_value_attributes": { + "visible": true + } + }, + "else": { + "property_value_attributes": { + "visible": false + } + } + } + ] + }, + { + "config": "dolphin-common/fs.s3a.secret.key", + "subsection-name": "dynamic-row1-col1", + "depends-on": [ + { + "configs":[ + "dolphin-common/res.upload.startup.type" + ], + "if": "${dolphin-common/res.upload.startup.type} === S3", + "then": { + "property_value_attributes": { + "visible": true + } + }, + "else": { + "property_value_attributes": { + "visible": false + } + } + } + ] + }, + { + "config": "dolphin-common/hadoop.security.authentication.startup.state", + "subsection-name": "dynamic-row1-col2" + }, + { + "config": "dolphin-common/java.security.krb5.conf.path", + "subsection-name": "dynamic-row1-col2", + "depends-on": [ + { + "configs":[ + "dolphin-common/hadoop.security.authentication.startup.state" + ], + "if": "${dolphin-common/hadoop.security.authentication.startup.state}", + "then": { + "property_value_attributes": { + "visible": true + } + }, + "else": { + "property_value_attributes": { + "visible": false + } + } + } + ] + }, + { + "config": "dolphin-common/login.user.keytab.username", + "subsection-name": "dynamic-row1-col2", + "depends-on": [ + { + "configs":[ + "dolphin-common/hadoop.security.authentication.startup.state" + ], + "if": "${dolphin-common/hadoop.security.authentication.startup.state}", + "then": { + "property_value_attributes": { + "visible": true + } + }, + "else": { + "property_value_attributes": { + "visible": false + } + } + } + ] + }, + { + "config": "dolphin-common/login.user.keytab.path", + "subsection-name": "dynamic-row1-col2", + "depends-on": [ + { + "configs":[ + "dolphin-common/hadoop.security.authentication.startup.state" + ], + "if": "${dolphin-common/hadoop.security.authentication.startup.state}", + "then": { + "property_value_attributes": { + "visible": true + } + }, + "else": { + "property_value_attributes": { + "visible": false + } + } + } + ] + }, + { + "config": "dolphin-alert/enterprise.wechat.enable", + "subsection-name": "dynamic-row1-col3" + }, + { + "config": "dolphin-alert/enterprise.wechat.corp.id", + "subsection-name": "dynamic-row1-col3", + "depends-on": [ + { + "configs":[ + "dolphin-alert/enterprise.wechat.enable" + ], + "if": "${dolphin-alert/enterprise.wechat.enable}", + "then": { + "property_value_attributes": { + "visible": true + } + }, + "else": { + "property_value_attributes": { + "visible": false + } + } + } + ] + }, + { + "config": "dolphin-alert/enterprise.wechat.secret", + "subsection-name": "dynamic-row1-col3", + "depends-on": [ + { + "configs":[ + "dolphin-alert/enterprise.wechat.enable" + ], + "if": "${dolphin-alert/enterprise.wechat.enable}", + "then": { + "property_value_attributes": { + "visible": true + } + }, + "else": { + "property_value_attributes": { + "visible": false + } + } + } + ] + }, + { + "config": "dolphin-alert/enterprise.wechat.agent.id", + "subsection-name": "dynamic-row1-col3", + "depends-on": [ + { + "configs":[ + "dolphin-alert/enterprise.wechat.enable" + ], + "if": "${dolphin-alert/enterprise.wechat.enable}", + "then": { + "property_value_attributes": { + "visible": true + } + }, + "else": { + "property_value_attributes": { + "visible": false + } + } + } + ] + }, + { + "config": "dolphin-alert/enterprise.wechat.users", + "subsection-name": "dynamic-row1-col3", + "depends-on": [ + { + "configs":[ + "dolphin-alert/enterprise.wechat.enable" + ], + "if": "${dolphin-alert/enterprise.wechat.enable}", + "then": { + "property_value_attributes": { + "visible": true + } + }, + "else": { + "property_value_attributes": { + "visible": false + } + } + } + ] + } + ] + }, + "widgets": [ + { + "config": "dolphin-env/dolphin.database.type", + "widget": { + "type": "combo" + } + }, + { + "config": "dolphin-env/dolphin.database.host", + "widget": { + "type": "text-field" + } + }, + { + "config": "dolphin-env/dolphin.database.port", + "widget": { + "type": "text-field", + "units": [ + { + "unit-name": "int" + } + ] + } + }, + { + "config": "dolphin-env/dolphin.database.username", + "widget": { + "type": "text-field" + } + }, + { + "config": "dolphin-env/dolphin.database.password", + "widget": { + "type": "password" + } + }, + { + "config": "dolphin-env/dolphin.user", + "widget": { + "type": "text-field" + } + }, + { + "config": "dolphin-env/dolphin.group", + "widget": { + "type": "text-field" + } + }, + { + "config": "dolphin-env/dolphinscheduler-env-content", + "widget": { + "type": "text-area" + } + }, + { + "config": "dolphin-common/res.upload.startup.type", + "widget": { + "type": "combo" + } + }, + { + "config": "dolphin-common/hdfs.root.user", + "widget": { + "type": "text-field" + } + }, + { + "config": "dolphin-common/data.store2hdfs.basepath", + "widget": { + "type": "text-field" + } + }, + { + "config": "dolphin-common/fs.defaultFS", + "widget": { + "type": "text-field" + } + }, + { + "config": "dolphin-common/fs.s3a.endpoint", + "widget": { + "type": "text-field" + } + }, + { + "config": "dolphin-common/fs.s3a.access.key", + "widget": { + "type": "text-field" + } + }, + { + "config": "dolphin-common/fs.s3a.secret.key", + "widget": { + "type": "text-field" + } + }, + { + "config": "dolphin-common/hadoop.security.authentication.startup.state", + "widget": { + "type": "toggle" + } + }, + { + "config": "dolphin-common/java.security.krb5.conf.path", + "widget": { + "type": "text-field" + } + }, + { + "config": "dolphin-common/login.user.keytab.username", + "widget": { + "type": "text-field" + } + }, + { + "config": "dolphin-common/login.user.keytab.path", + "widget": { + "type": "text-field" + } + }, + { + "config": "dolphin-alert/enterprise.wechat.enable", + "widget": { + "type": "toggle" + } + }, + { + "config": "dolphin-alert/enterprise.wechat.corp.id", + "widget": { + "type": "text-field" + } + }, + { + "config": "dolphin-alert/enterprise.wechat.secret", + "widget": { + "type": "text-field" + } + }, + { + "config": "dolphin-alert/enterprise.wechat.agent.id", + "widget": { + "type": "text-field" + } + }, + { + "config": "dolphin-alert/enterprise.wechat.users", + "widget": { + "type": "text-field" + } + } + ] + } +} diff --git a/ambari_plugin/readme.pdf b/ambari_plugin/readme.pdf new file mode 100644 index 0000000000000000000000000000000000000000..12093757011fb20fc56a6143d4be1c6bdd0f6fb4 GIT binary patch literal 704750 zcmcG#bzB_XvNj5Y;I1JMT!Om>3lba>Tm}dPpWyBgB*B7fkU)?iGw9$tNN|V2-Q8X8 zB>R2$*=OH#Wq+ z)b^$}7^0%ooXX~QmM&J*0`N=f)SPduKf0JZ!H*wJT+C(6KiHd@Q;UmZIJ-ERo7iG_ zAVT!??8)T_{IZNkI0Ma|*q!@+{Pf;B%#p~-RxWe?eYq|&5pD#cxk7G&DBH>6v2is$4dLcy0rVeN>C#;l`aR+?CK+1?PY}sxfv8&g*0-~YClxa7^WQOViy_* zqiN@k$Bbmsyu#S76d13AHciFE<8*GD;FMm7dUkvDdL*c7ZsP zZTl+4v^u}@fdCKZgSrN28E=UAEB$V2d%5+qw>y_*LvZHUIp;<<7(b`b7Z36Sntc!-3r8dqA_PkOi?e5pyB0u}uD*|2=N0*@9*Z4Qcj@Z9@9LrkH3(5k;yQWsdJL4x zr}*EjdiI{_)amqBa-Ujs32lC)J#uFAP!tnLTT$a|ws-54oh75q4L9V_&Eg1Mn{RJ9 zKae!5MK>1s-T-PrV5lNZ@ILdf6@`2{{=&qoLOleE$5$gs%TM1-cm{hb&Fl;Dm!x$U(ejchUw1e99Lk`smYWvn(I zbGrcNT9O=_80IA{qJb(a{0Keg0<{$7H#;4nE6)cR{&#yg1c$!Wfv!s}qdQw?7x?KBxV zhiWHOITvlaG>*t&8jbHd+K?D7XESdM&{f;fSI@B@xk4yZgGRIx=9lM7#;*bv#%Y&w zxCoy1r=X;9y>Vcvt$eQI?w7_P z7h(MC)c*2d#u%?8pb3_KzTtCJ%@9<{d-8C1W?W#|F4&H{Hb2W=Fp`|>D~I#o-jc@g zD0+3}!A15NvPsmSMR$k*}y7eB?l-ik<*nv(R zsh)QzcwKhQ#1kH11as6?4Kfe$3us#t`{&8xkw++W8IYY+V$w`?cNer8%&mNa?#<{m z2G(YxoxS?b5Sb{3KcT#n3!rNoYt+i|i6LA#! z%+8)Y)7&SbUHzgKHg5RZGFMqoM22j-p*fnS6FtK3bJa63(EFziyh!xzx6vsuW4=Jh zBuzzT&#Vxu7=%~@lSR2y*HWwy_B44n>Dc=D*wD)(cHfs)3@tJi{PB_ykjjZFD7GBD%g+WEbN@65v9rl7FF z?%Caf*xl;5zb?E7-8(*QYc-=>GDOVeS(c`!2q;L_WoJ@vO$rO{6e89ZkZiktPfG(Z z`Qdx!S5Kj4e{dj&S&>fo1^;zF`$)Ll_A|*8Zkp6Gzi0yJ^B1eTBiqDGOfdS~xgcCj zmsfdOFIh!dPbyPQxKz-jxzaqN<>|g4u99q=l;R+-?u>;sd2Ma86@8f+HugVw%fy%^ zZ`z-%Mqe2pvA2ms)HG@M^b5tDmy5Vzstoday?GtTMF9F7i zEt@xreJaIgF?-`ezx)i9WCH4wrhI&r`ZKYd_u{KMoP0}Azw_n3HC4UxDEwXp!rd1d zt2ZLCM$+J}v?79}@{8%GM@Y-Jr6_&fW#fX!GkaHTb(ikj5_%UHLqlaY*;DFV!%BTbBM~eV>jHaf2^~J)d5m zJ|eo2+qq(x+nN2fw{xN9 z`v;>?H+Qyob^2iLOfC423)1#>F7W%#)W2B>oN}m`n^~Jk*?UkMaKSGKaPv?L3iB9Z zaH`w8z&Qx~5}beh^~T>L{LNe5I@y2FFn6IgfWKDe4K=5xxrYlir-Ch91gXD{(tjNl zs12z(rR_i3J83wWd@!f}O?#xBxv7PI3xFXmPR%Ln;UcH;=XJmD$-(dO^8CK}`<#cG z_qV3%a5><-2rm3Tp690K{w**3FPfS>)ZE;E#)6;oQgi>~@0`-`%YTicsmV<({Levj z)l6;7Km21H1zY$7e+2n0tN=C7U(=>w%a6hH*V78N@R9x^XbQG`)I5K^Six41n&+R_ zg{gV}c^xj?KgSWG=J}_h|236=-ud@j{>1^g{~;phKaBael$;u_rY?WX`)6{gId!bf z;Diyb8mGLuwWXB{H8)&zxJ{iL?0>UH3>PO?xEK7(eE+4MhKh+Zkec@|!KAERoZp%| zN!!~x*xUW~7oq==eZtN2U+nY4dul8M1bPI-|FeVsk^Q$m+}wYA%b%(IHx+sQa=QOm zMYy*AfsTI}<$q1bXHTV>kr5Ef{wFp6$p1&pe}>^7FZp*h|8l|q&1(K-*#8Y`{*5i% z|5Wo&0Q{Sp+I_En!VjDE#3Hw=dpcLaMgpW$m0ZZ$wH(sI=3zzTHDu}E}N2vR>YPXyFI$U zU9R=SKlXn+T#y(P6RxiEDanh(pDJ@4L#6UL+MBNIR7xVD&dgh7n&ju380YtS{48-( z*>ZKz&Hdp%O4MP;bW-EfB$KA5Z`#r6)$uD-p8WtIYZ zsz!lY12F{DF88&5eYsJw$Vo70W6W}}7NRL0xY6K#m>p~ndIrv8kf)RwDUP$-xX5^g zD0ZMXKsrLvY9WG?E8^xd{Mb8F;rdO}=i%=9$FFEAQIDe$IjSFlLl7X0C66>b!^4G2Sfa4k-qfqZ#?>VH z0T@5uU@C7xc6pdLp;AaR0v*F@qKg^b(|qMr5#O7$>SU@D5C}9dFrZPOYHMZn69oD| z7FLqd06jsEfBu}D`~6InweINX=qH6h+>mrBg!1|u6mAQ^w8x43D5iL~ncZ3fvqoB4 zT5EQ$57JB-sL0Dn#dbfpmqkZdRu@`OpFhbiSKZ>{6lIW^=woYm-=E4mJ3Ffb2Glt$ zL&0G1#GvIwkygU)FX6E)!*bKk61~ddfMdP#f|$hq6z(VAdG5Z`?~C_J<{F9t%J+V> zjA>R__-~&ZZ)ID&#MS99Guw|47$UN>IxadduoDeePRHo;PNHxUKQ7YTYglyK?)&_B zOogPN-7HD-c^~F)SodfmYRjAm)2XtxZ*cHf&ZsQT1k9hcMS-Xm=Pr&zAZob%4RuZD z>eRN@!Zj8<(--<5tZ7k$fi63K?ot)tXWGR%I`q^k{!}$1A*x9Kb z_a$zfpXoflsZ_1X;d;H>%W;t@%J2SwH2UeQY?-BKoU=C@(EwAHPleg^NkE)6`&f2^ z>{3_Mw-*@^vH>U3pK�RrlvFZBi$y3Al1{WEj-aw=%ea0@fC{=~~4whcS*Qse?A* zp9(BllFmhdU8bQdxw!sJ0BbdIABFDt^!d>;9U12f4K*n>>H=Ofgj;@@G|KCrb!R_>e>>e(ZtqU!DigK7w_0Y&&&<8v+yTg>pulSm zJr*D2XUw(Q&GU|zLzpm@wOl`G9VtyI0s#R5Xsv3=Pm1L7;gflk3q z$lG;;XF7QXdFjxHlaf)83&jp@zRUJ#UQ7WVh@YRI!p+_Nv-N}rgHjS(Hip;9&`+jT zQPKjGbl`wjLBiXpWafR}xbrvfwiK#pN$i?7NB8A(5~Wl_*2R=4iq79QvS|Gh2e@2v z>g)?i)?ZRRydEp&&jZ;G-i7oD*e!aU?DRxGIBoxE0gFC3irPM`YZPznr#oJM!2^*? z)RwQ)DUia%UsZ1%1l9>6SwE(e-&zq(VUcihyyK$7RIcxLe&}Ov?ZmHdcX{Q!QD;Hm*7c7>QbIUCs4}GU_6se zeuO#Bk&ldJb}8A|iw3uzXP!L%v-%T2CC|=p*LOOc0qI5wNCl@2kw! z5a$HSyeN*fj*hiDGSP_DtMN6y{WLg}zY(T&&!?(oNcY+KRnQ}{W_z3k!S}Vv^A?hy ziSRngD4}OflFJ|*dF=!l4#N~`O+Uq9S;`{~xUt}(VfhQW&VAlb6Y7`#u>!LJu)rnWd#O`|;m4hd8-yNHGVkM)auK{g#sRNHq7-HK5 z&?c*dH$4mqm4fY_enyXHk*Ub3z{szuD0(iaQ)^P zQvbdUY5kpWu$}q1!-p|wZmQ?$#=G)n8uZoW4_b=}JW#^2s-=XnW(2;qdW`PR-1AikKSHZ_z=pgbR=%4%p_irh4{ygNq7IF9PtO3C@sc79 zI~ilFl$RX7FO~NW$aJ*T#6%r^n#G2A*Yq9Fyl(c$N-a(kytYrwPhtqv7JM(~t}*Af zkf`kfk8tr66Pz!Gm+25S3x-lWuTO}Jjz&4M+!uX&8|wF+H|k7hFw91-F-ec!$&icm zNEXz_WqN*m=xwa)Q8e+SRRuv-dtq)UwX&#EJLBo`kmF^1nn_Wt3PZOTer8O zFJG@I%=zmd<1}k_%)ZvV=E@O|s~yRi_Qjr*b5;Zt*PCZb_|tH`c>u5yN0{OaPhU!0 zAy`^qEi`{9h)iRG)mt7mM@u}UOdv3_BP7J^Czk@y6Hsw@tzT%#29|+6p7|UNFMD5_ z_Py$(Ye8c2qLb315Ovu)Uj2p8r%TdPWI-tz^d0!J(Qh>rSg@(MtL=Cy%ia8J?^#Nzh;#a0S^_pRtg9}tpuS9pphR*CR7qw9+N8y$ya=-B=p#zQ|D~F)>MZgk z3;|(;<)XI3UP#RWvDUbE8U4DoCn+I%vrKfB1PaL^iBGT#F($^NRt<9<<}(i`0I74h zuzrtqT#j=x2hM90ZfuvhqzfVR=C6!|XAFamCi;>&VNN{03vwFys=!P!%YzD-8oNO? zmX@t8FAq~7p;~V1bXvWa0@R|xQbV>~juM<0fKKN3c4bVv+U&h{`&#ZOD6DSMalpC4 z{Eg($r%DxZ7gp22aEj1|)udqfGSFB%d5>FQd~-7C|A?Duy#X=Sq4e3amJ}`5gD6F_ zHTv4EztWj(@wZ(?72nrGJZOq2&dSt#xYGSK@4F8ZP~FXVn@PAKnz_yad#?7T zmyTh%#AB2ucX_~|rn{Ev+4u*sU-JY#RKi}$ec>%h9evUDOqGSoP#i&>*U8TfEaE^&P$*nB-!^Z6+TTi8`-^oEg;EDTLwD?a-l1v~KZUR>@KgFgt=oJ$i!+ zh%YO0IumLqb5fp^A$q;Y>xJy#A@kHy^%2+GKL)lIwwi>3NS{LMpSPD_upL~2W@}88 zh)8lM8Z7z!w?~UH-v*26G;+Z(Mp%HH5l0*B2D0D$8$Fqz z*W<|66T<(|0*TE|1G72njOgrar06}4L7+TER3Wu3o@#Uu-GKak!M)gDpN&Y2I zFeT1X@P*bXEqJ5>Vo5_1NcIariff~9j5u_*R_P-9y%WU;E4XcgcTA%AvERRN;pwp& zWjlfTDU|A@-;X7Mu=EtE?4Cg{*3y6>$U)d+9!OkKQ+W0#X6oX8EA;y4y!ITJ1=mH* zq!%If35Okh?`J~MVJt4iO9=Z|br@(JCr|IVU@VlNGua??`CTYv1@&7sw|?xw7*NP9 z`7J}V9IFhg3+o%jkBInR96?bkLiZ;C(PH0=QS-jJ@Jh>}H{U{KhPBlPwzjU!S_6f! ze4db+P|dLvR`1e9-L>q%#4%QYwLaek3Wh4f*gg<3TG_NRWPn}zm=&1-(O(vKbzQ7)eM-kS}JV&!Xy zflsyhU-VmY1AP^c#lfPg-g8{vA0~=LVR|3%w?fR5&O(;l)M$sL4{W$~64?I6WE$YEB4G7)=1nHPh1(^eP&v z0XSa@1`|_paj-UyRDlv|wQYzWD2BnYJtQ!%4P~|)WYQlu%Jra*z5ZMn5XdJlecq1r*^x^8}PW>u6`PJ`bVR=zpN4#zC)B8=i zi@EF5NPIf^%nlaRZt)9C-$)q~buh$MwI4c4r>5kcB{v2#7-Xte8T85tE_;M5?L<;A z4*~Vg&kXwt=Ls!%+-5j5XxsV`3F}zL$lnDfFtb0ZVmp`7F8LKW5;sh+OT0yEbS$u& z8Y5Am2xX_~e!RcM{jTMDEEnqE;ry=KPA?Eo+5|h0U{$}}$AUxAp>M5aIWlb%)ubqx zy(b3P$vu`=x3Zp1#-AyvqGEd;y!RO|I9#(}(!;_^ctp{0%%x8Is=ifuu$ShbWacGV zo&jRE%N_DuX7sF;mg53ODDDglLk;~osOmaH zHpJ74ukISd=yJwaR1`q6p_2FN9HsIaEA|CyrSsKL&jW=kQpyjP8wU1|+d3^5t66t^ zY=keFCBa$?)9!UD*~2C+Pr6WXj**ajwtSf{O=RK~PadDjQ;sh*`^tUO7!bWV?thk= z(lq|b5gVp{pmWZZfj0~Bj#j3;q*Y7=vfl}R4~S8S-gn`Tu{4k2{zQzMuIQ6uv>Os# z#$gPQx%;8`I-rK01=AVpKnY-=C+bMNdgIi|_Y$nC$#+%i!2#=)kV6AVMsSRT3p~2< z3!u?Qy9Gx+2po?yu0ODXikZ?m?~c!}^Ny%EGuUJQ1VN&jNJFatk5O^-{j#H@(&#g~ z!psC9!%Nn<+RCI7kzh<3lB`Mm5+)1AxU)X0pPESHTf!|5Ar za$*zqH90-ZRrAqInqwk;Pd~h?k^Quqg^c{Lc87*wf|**+zGXT&mt#4UUy=8yruL8} zWoW!qt@ful9RRENev(s|?%}@W&kUIs^h42Kk`T_ofI;3`;$W%db*-GQ{oE(Hue9EC+YkwysFC;Cv{YrCqb*M|%LRG*zOh1uKt&%E5qyOTOK z7vOqt$9)aKZ9PE^Z6j_F*!LE?3Z~~ajCNg^7r6&`dIr~&8Hn*@bgBKQ8`sSU-1>D|Zp!Lxl5NeLRbs`Ekc@$;Io0$2`j+huULqTgE&`;}tj+nX%E{oNnG5*hI}G!dOZH+%(2gd_oo z;pv}Qq~9PCxdY94412p1VU3Cvl?2J?VGa(W^#r%1RKjVYz3tyP*f@FF+iHxc$0xMM zDcSMC#ANrDsA`0xfFc0USo*0LYOe7~2IcIw4Yggt&69$)@$k4R1J0vjWgXEB>+oko z**u1XsG?@((pdH;pskh_W>~L%;GMDqetB?F5pt?SL!;|ghK>%L@s$~q*GmcFcDJQf zh+50-NL0p#TX7HKH+NFoAH(i^u^ul98;AD7*Yth&aB&sC(@${6?Ht!m4tV2~9}>#| zr-OA}n&rc_FrOsJQ051#=Y}3ZzKhVwv8bbmpr7SBs-^Ty72b>U${q9qNVU z5dzOF)*~%@y5jzfwsP!XihARi*L6slRgWCXDhBI=bI_cT`7R(gD{!(Y|K* zL|E$mU9RHvI340@uXr6gG{!>6=n5i5dE-y-1MH_OZx(MLx8UdF^Kt@xgmpUfryM^w z=`!KyLi6KU%0buWKqK%^9H5M>UTtcx6dENO# z3toE)M0ZLwTAJ+*MGU`VdT&RS;l^TB4Jli5d?WB3|iCUVp8sOi74nfOawN(fttK(i2RC0c(bqFNL; z2-3h!~v==W)mp|iD zoqP``{Rw1m&LBF$hr{p&&PVZ+3Q!7e_Nehp_%2lly|Q&yvzlkL7K;??vO4*L105)7 z$&J-6aGZgIs=352K^C8I)?xS)DK@?Jv94OQe3308a+UBWrjDBvp2z?cyjEr%Gw9Ga zD9w>esp5+H9h?-oHfqnEr0!S3vxRF?sbNkgO4d`37<`jlL@*GGApp>4R&m!YBLOUdT#3TXff%)IyC)5U{gb zD)R})m33uCPO$IRS}aA@X2KO}VWz@vNF+RMUVicGiYi9g-Y_QcQ!>sJR)oI*K4LVp z@8N=&o}VFj!oO1kOnk_1twb3(@0i@E)^K?CwA@jwNh|tJPMNCaUPpf-hFYV0b@x^0A{$)n!Exk@5rk2U;dwhcN9J$-Hvud&W?zDX2_C(g3e>zAA>xzr}-7{%5L+> zzkKsoZYdS{!~M5Jdr`aL4w?`r5^h)k0t}<22*6sL#hS{OMdO~V*@BLYm!iq92i<=& z3QCuve%mS5@6f2euAgWfdF+aftvAQcdQ|NfuoPP7X#Hy4U? z29-LO9iZjMTE;Uz)0~@lugHL99RM`XV#Cs|C$=EYNo3LVk*)b%Y&9$^3o3h(4&{b> z;T#L|Q<;#EE`pbDn`x@dMJC24pLW~m_%mLcuIckmle0>=6i1>7((4>b)Sa%iA7CrY z$>T|4nG~7vqy^T)Ys?S}wng3fQ~U)`-{Sr3g8z9-)~(=U0V9r%=>2RDlLScg;`R2p za3EualaQm-z{B9_WT(P-5aaYLxJ88wSKE31-(73H&Bgj zJ!?!Q7o8X71H@=+bvkMvzX`}u2p6(8M^q3JVtsp~gC=3mDEVeWv1?IvE&f8}yQoih zOC*^WExOM>`Qlx#PK$ueM+Mz3c4o;IZ4-1r62S}r<;$}G@NrzDa z^@n_BK%_$O%=fb~#{1r$J=!k*mu>sf;NDK+2e&%yM`E=)ZLMd1KX@nHj$Lu13x!%) zVQHg^Hbjxu(hg%!zzwZ3S7 zLO~Xq@c3*DxL$p{+xAzX)zA~>7?t-=xGVir(n;Mtu4BrrRM<44rzR?bLpYoOlHj4! zAw+f20F&=9d5j3{JiT~3Cf;A)QFG;&+;L7u-rbDUw7A-wUD|8kU73IwahTVLkiOJi zAlGX9ne&Ar93umIa>B$;9A^-b6U*^Sye)#7H_h>1Kjy-@ zc9lthyt0>kVrW;_=<#&qjH3Lz828#(AKjmNDyrut479iqh<7f5Js1x+>F-> zy3VRf-wh@j&Ue}`OC>z??@x@E!#<#GO#{4Z6fa5L*n?yk_*Ek?@e5>7miInw3_&V> z$$GLK*@f3^S8_)@zsW8$+pqjHI?S8tTltL{M^=8 zGY55IpBW+wN|>MY^RiOXuob%Z+=FZj0D*Js?Eqjm2(oy8GBBWqgkF61AmHGC;A}i_ zNPSte%w#^^>kRgK(E*g5reD|pmM>6h&_a_gLMj*B8nR(3kpSO{Ec(V;&K)N51<`c2 z>9)KHOdD^vM=&Jd-hGuNB$mj9w8F98wTUHRuZSEfZ_lb@tLSFaiQJ@}>XCZW#XJBS z`yBO5p>dMx;VR0*RNu%7z-lK<;5|0~$#eGN#mN-$7xI!O>UKZ%iVU*3*o57=(@p(o z3bCD?$mGkBbOD&+k46cG`w%>(ARzmZ@KR2iz^uZsMV&<|J75W{`uJA}tXR_%b|U9X zFbd%$;?N`T9t%<65GL~))$|u2;lY3>)c9A?BiPPcbTnz`yU#Wi$2hMK>Iai6!0)lPUz=QbMhIth~H73rT`DEihN1GPH9cy^+gwL zxQ?XIakeA8h&VuzVa~L8dr5WF4x-|4ppcCdNTz}He!Gp8FvSKA z9np}~uUl>nEovwBKGZt*QBLzYdUu~RmN&MMXB*_-8CQY6Yq2je87umoqufA^!_&dy znoeQua=`C=u)#mVT0XRH(nImho{#_d0WQDem)wO#l!GDPgJK5Nd=ZTLeJQ6CwElw$ z^=GBT=6MM~*Tk#M+0FiV2OmB~Y894n+cVn2IFIT&;yPPt-a`-&0rAp~{jQNY`g-~0 zr>%~8oX@&Zt2_1u!05I1CY-)n213@NrSCR7J1q|b_SeXG-|gV$csqxMoHbxjZ3Fw# zt^jY6GO?3CgC0062h_z$EyfRv>i57v#bmUWTsO53lYSx>+N z+vOQ-=29!KtLkBe#nVh#e=<|B$MbWYkztLu3`D)ay2gT1q_|kTS`OIGZ(6QMcr_Dt z$W7PDQUtoHVa1m{8pXQvJCNU&LYVRg`xT>F26u&wK;Mw%2=`8b))cu8TlF!85>n0`U9xP)?Zhox!v;Q3Ij_n0R z4~d!7Rz7?vxVY`7c)nsBb(eo?Pwm&6=e|<=teOT8cY%doToDHc^?Ikf=s}&kB|mfO zcin|$*oa_Pb#vFDV?xomve&kxng8di8sUlx)P=86YD7Y08Ba^XB-*3TPp%v*g`Kp(Ac zpf9iuugz;HQ7L``ig5{YBxWFV@ogBT0HZcx`hI(|orT6$D|kwOW3g{ejm{NoSWxc$ z3ImTPKL57&O|N@i2gE7b_hPTEwQI~nG@N@+Fp6RBt>YK6164@EDwX%~3IoC6_pr8O zyy1fwla`N;8DfsKljc8BF8OPYPA!IxSHB#izZIiUth<0Ms%9BA2%w(qk+w!-#_Ic@ zY#10nb_p(eoPq)**G!fjKru{L6&PND?6YM6Q*DPNAp(>7M8r6ik|_(rp`AIU-tRX` z7@^`PaA&T9CaLZ8sgyEgO0J>RhsbiH0)hxDGy0NBT|yXeub^~}`-ijs)ND&hGB$4m zJ3z%L-Y;5`1YPu(3}+4 zT-gBzB>J9(VlY?mO%78(q=Jg1Ajf*5(m1qllX8j!PNrXdq+$5)0KFyQwhYnPzf2W= zk9K_0bGixGfKFA~^t@7u4QL2S?Z?xJUO$a&-KsyKGRnBR_m1atls5qncyvO(vcCAh z+=9lox77g&z$H^Vg70YV?vg_Rri}zsQj_s>1Ck3=hjh&G{%xIgCdgly0!)rv_k>~% zPoV%-YI#DBp9JzW{Sb@-MJi48wQM8igr+JUzKRMj9v|j%I-f7v>-z0xKSn{df=Wx% zSlCL1BFda7wOP-mXGoigr^r^=?3zPo94qE=gqD3Ko%KXUz$hX1=eo>p{o}gYPfzX)vZ}9` z?>1^$PBSDPBc^~tc|C{kP{V;CU2XeORTldqTSV1N@rIqAWk^j&uR^KWH1H7@h(1I( zrw%D&36XgAuMvF!HBHtP*eXQafQ~)1c$isPdwtj7ma~@k0c{ge4Y@PHH(<9AV&Ca= zHX8H>q2g|NOciGC-`!A$$=(fTk4JvoVN&TB`;J0yWoV9+nj@*JBN|{z=@c~_@$^N6 zS=f)Gvu(eHxEMBy3jLKgUs=To$%nYJ-!QxXCgYr~kWoINzQ>Iv-C*`(`{bQ*=ri#t zwS;Iox#*rw1|2a=^T<0L{n+R`4){LQUzNp*9F+Iy@XCrPU7UwocKG(Vpi9=tkVcci zQMttXQ16t5XJMg4wUY~*c@1{R>aYa{p$e8h7BZN?!0ms-@$AhhS#Xn0%TPZ){d_}3 z_j_%O+HXB(p|xIYS^mgxlvy7aqRA~x1bkpW_&ybo#j<3@rcI-tt<~9}wGBThOeC&&o>>QCuNEVj)n}d;v0rG^9S>-Z z4#pYQSb;syCt&~-H8)0kPRnwZ3)E_pH6IBU1Y~$1-EmU0?(wM%Ph&~ERG9lAnocav zi>DCKKF6KP>>4hKD46Brv9a-iMt}XAGzMWXseEBrSy>Yqr0AvPDL#@uOblc7L z6jWtvE+}j;SMyWGa0VUU=~oUyM%&6OZq^A!mQLX0Z4$XwWXV^^ZmxW7-#Y-iHgmu6~%h z%|4who>Y1}Tyu&T@1V+PTzMjwpKKdN8QF^Xvf~i35ucn5%ZWR~vzhmaow`(Di0(!J zRGyJ?R@n03F8Rk419_@M9kR_QP;U2EnTc_Gf~`MS&eF`pJ65=!9Ws@(*YAhGFjQ< zVheSQ1LBob~s3prAsTVJmZ&`4*%G(2_-e2ui94>O)F6>vBV6KQ@ zX==jKMSRn020aO}Z46ax(?wJSYGDLfkq#ipIX1kCs{f}i?2i2Uf^4n~3y{>u@MZ(J zyF+scb#;Cxg{LVhOE=(vNkeq(U6LtU=L40QS45v7V^Tn(?WnHF5^u_3foze*&S%qD zCr9oD%a%OJsaX0{+*o|{Gp1OvDx4+HK#RH}> zaeMMCGFIQDm+v(?1;4T%5+XPH=h>2&JF5to@1s+K*(bH?9VvZyqs!`#O)zGB&xRh1 zvN8D;BQHaxqX{-1NU_F=g+c29`fyjo%6x;F1hh?^ssmlmEnf>pyDub;S8fgV&pg0r z?%Nd9c={n-V}v~rj8|fKK}yjP$}7a^&MkcP4VEP;UyQP>#_PU6QhUh1zT&Z zGB7p_7G9wa=PobTk}?adn+6zzsH{^k+Lt~@Qx+P~na=r!eJf5Q9TFgp!Ll=4DJB03 z4d*uPDVcVj*B4qv{DDVA+Zw@T_x&C5MLD6#kWsPzz!B!`Fwrda$*ljy=HmSWE=1It z@exoQH~Q(>qELmn*Iml5qkB{wMuBGqLV&NauzE8yDd-XM4$mdjwvJe>>Fhv5Avp)v z${E*p?QkyaX=Vu9M7aL$QnoH0R$P+2E>_|=)+@5Gf+}2?KAW?ow_K=_)*E`|vc-DW zZ_9KXRjj3Iy9Yw`o{e)_;_5+}Ds%O$`PZheb>Go?olXE@W*C74kK|B6tVgwdd@@m4 z^Lgn2S<2>+PUA@UX;Dm= z^r*XQ541uU8OgH^vc-?05AA7C19n%4fgB6o zJ0f4>4t47lU%d(-!+wFs6)mm0-p!T#`g|yYJERMel%QI|ktN_hdUJlxE?BzZ?L>r>WZy0$MLZB%l>ZK*vI|qC$|@i zHRBLGH}e*K>cO7&oy0?XHBi|Kb4Wi(_v72az@%+kG90~)=i>piQo6#}=l*nVw8esR z&2GO2-r}_MVr+OaUzd8=u`c@*?W6GFO_1Gu^&dm-6+Q!4&xgzMHQQ8ga>`bBDb1+3 zx+$>xrA^ByFy#_?&6M5`m_^DcqyTM=hkFXup--(`CJhCl!8VO?xT=rN{6hn|;p(BV zPXZb2Vntj=jN4Noumw%-Z0i-(?pbNeq09)&FoVo%EJui5^LzYSzx&XS30)Ff{I$rJ z!Q_JW!thG+5|x93w+R5ad|C_+saw@ISvcZR5ixP136K_V{=t>*P;@N5GV^>%w9*bs zV0>bAlAKOo;p&KfS>Je5v-1ZrI1#N_uO=!0-2t>G4>9e;$kU?Zg~F}=JXM~oOAw8) z2wgE47}6AZIdX)l*A}Hs;J&!YrdVOB9IEJGhRvHwWbsqrbZ0%j2pU?)kSd5@*p_AN z${?#nSvyHgEGIhF8&1c{8VxU~8tA|pn%3{ygkjMfwXPqRZ52*;bY$ElNJE}NAsacs zd|peKsqeQ{b(8NtD6tWX zBMGnQJGWuyd*$I_2N0dof{No?kyKPProLjF#J_xE#60OfXxR-U&0LaWJM%tCPPu^-_=! z`w4$T1Cdg5)q;!)BBZPD!Kfz*%m_FoH8l%Iqlxt#CtQ{7hbNc%kJNkx=H@YAZ(rgg8oFzbMPB@^sbZ=;KG1rw{OE+n{H zjs&bNzboLc8E^5?z&%J#K`!Fr| zfDSygZ<|B##5mO8n2ly{=O8s%Y%DW1`4VC*#Srb z0-mQM1gS-eA=&Iae-r)UvI#!3O!iG;S0Ywois1nRfw_3*FW&;t>=*0elXtI?jiv|j zf&+`1{o1=EzgTR)2=Tt$9}53Hqk7v;Gm0oSZsz8Zv~|CCS+yqk>jUUF^bLDFuKziM zJ$E55oT5%uBp)1vK6M|MnTN{QR70ht18NbUO9yx$fjzq*I9~Na(qIS}oSA%P(zNzO zItA78oqKp!-|7pDh?fshcbir$lwfxDUMRk4z(DrO#DJQotT}NrL}-p>gk>LN9wT#S zDNV!srZvp7=A~VS2oOWyh!t=+pjY33R585 z8MPB@3O8w)Jo&&mg`!0G843yQ8v~Bm5Mv@WE^^mZH|KcqWIs6qnm-Pkme1|=lOPG8 zR)m;|iphTFB4XtRD;#@ix~z{`_M6$4%(@>OIlZK+)`fqf(jTc;SveY;L%vvYCSuF* zgr}i`|4l5Qn8=OISY>ZA%=cCZwN_jBu6^JC!u{}(@I-tV8%6#ILm1Y=C`CMQ71 z?@q=NMxYi{$uBMXjki2dHeiaUah}@yL%CWKPZ0^4Yn*dM%~Lq3=p*5r#laX{i| zZKM|oYW?xvT+6mhut^A{v$}Vs-DhiStF@4L!S(uN-6u#Z1qdnxk?R`Guj`X3`BfvK z&6?07fhtLBDe~8tBZ&t6m*EW-(2aK~YD#z4S~@&(BnN4JvE9YixnldIo0Fx=uUsS< z=h2-(f9LE4huw^^7>;+F}>xXM4lz+Oh6 z4%!>;syRaCGx(4Y08G?SVK3k357EHp0w+iaU1+Hiivy&Gp&tigXwVF~-N-D<;NN(B z;c0tZau9H6h!KZx)d@PC>HhwX=n&8jT2|Wr+Td?gt8G<-cxJB&TvlDyEKvOxS_$rJ zeBA_XLt&%P>1xix?v6hhA{Y}XU5H0xL{R&ACEZG(74Y`SbpiM5lICp3u1LQwEf1Y* z#>v1w2mu2>%sYHLg|WN!=mRkoo)Wc10TivZq{y8$6-uMusR_{KPeDla@u~_({OyXk ztLcex@GjK*%F=@r&-@1Sp^|#=$w@l0r)sIjF?@e)eH&s5!Jr7NK#wnCgsG-zUU#W2 z(nc{evpNI~G?x4Z%-t9iqUTUVNtEgADEnnxtA}4(GtKtMU$k)^FOI2>GpOGHm+6Fc z+4~={$V%jK2K*f9u-$)s8wpY4O$(tO9v$uVAVzXc`my`8KxIm&`bTf_lgq4Q?ZEb8=f|pvu_NyvoKL<#(>?o2L3Z7a! zh^q1%d5YUsR-XhRNN^Cc7GL|4aohBh+c)$3jHldH0YOAK@Yrz@BMJRK7<=ouD7&qH z7!VbZP>}A9p_Pyp6huT2m;r`vVJL%c2?YTW=@cb|Vd$Z|yP2U=TDo)IOYd{e<9TkM z_x}CEd}bKdwfE}1_WCXy^KLQdT!_|Qd0(ftenmtPMp3iVue ziipDQm=X+>g%_Hth3(Q7BM~r1Bjlc&A+LzF_4x3!Rdp-0yJT=T$&j)oPrn>8-$gou zlslC#2=vW==|>fF%zu*t) z&f<;IwI5(nH&!UR8Xrs?6Z2lmASS3zKHrV9uni(y9O#40xLMGih6~1hZHt+$H_jHk zMWv^KlJJsXeb}v0iq(Z$Nn}u{zu7K<)9PYDuc5gT?As%)_W21fHja=|cnHtn_Q$U? zbfTWaI#r8QFEP`|dl9oZ9i7|?;vhkaTvFPZH}o_j-K~$(3ZAgxh$^s{6gFzzGG^&X zx~EdDHKL(?~VMH;WbXIfp1caE&5THCK@r16^FG3WZ=?R;jaAzE8e zah0b}CNLDsgE@@uj{WdxpQrF*ubs?f4*ShzDtnt*04BfCXna3UqWi!X-sYN zi*N5}|8KrMoT*5*ggcq?6tz`m$SW8(riT==7y4G3{m8ajsfhhP?=F`7?;Xfd`3>%n zjXwJ*mhvvExz42t2O_-k(*dkAiw3+3+pb?L+o9Jv_kxxQ?cTTM>KY}a?)>WDCHC7Q zf3CIY9TvvPN;Vgu6+A#NV1xa>=$&2Rw<3VfN8x{_!)1+?mPA-TaWv8Sd%x)vq* zZx;NE6TiI#(E#!$Lf#Tcx*VSbB^3&){13x^``Ukhc#XhMIU_?-WyT$Pcu~i5*6YK2 zhQg>Hssg^L-iPZ9_4Bh&MOzLaFAw?b4GxHmdh7=z>@OkzOV7n#By4;hOiPp>D z|NWp}n)cg=Cl5ZuUw@a4y0G@mEp9SZ!n%&Wn(=k_-p5?N|C?<9Ka9Cp1(C0RU?7J` z(`E9W%SX7Enqz3%$8AK5t9E~r>3=hDH>jSm^O83*v$(OruRhko=VMugIT7_w8~ANy zzrVXJfFq)Cyf;;=7KCM+%@b^X+edQ8<2sL0elEWeRQe|bq@XD0TzzrH=iXe^$FCKc zIlN77rXxiEQU5D}oxJ9P4hM&yVmj+cjrHr2j{O^^ zVC%{Mv##%Ce)9)If?fYjnUqy9Gu^hGV-A`K&CE)MCMIH1cV}aC(6F{lL_ZG&4i1Tp z;?;5#26m3l6fLuC5p@fv)zh%)_+^_9>mkIJ=$nk{&KqJ1BVJ|@Uf=iJWcmT6nk?u0-adH2nDO5NQLjZE|+ulp) z1bp{EeRMg>&x;s|BJxWM0)C-j*NmuBpay8X-%so47X94~{-k`zET=43 zead_SEvu|h_`Fh%3b&Zrovhq2RvWwQ-?FDrxu z|87z0>A)@%@Qo;!pqP=87u)bJcen>~BI8GmonHCLy?BA>=iY|zt|UY~2X`=EeuGVk z;Svz|={q6-(G1^A6bkZXG1iBDl(`!}_Pb%avN#T(5QFhL0i$=zZGPGi6lS!lfOIDKRO^Ag3i)(89+p#~~!sW6XFSPzMV zZYbjR4Rp@y5P)GD&!$c#+i$k>qisMMt9p zgu?g~yEG6et7HKJZ%=7PNGcHd@xBDMNQmq(yWo@fvpQqP&ePR*ThE#ck$Q7Z1G~RH=mOLO3N>MFRp}K z9Jm9~1oqN>-KMzn=Jo?7kUktinDRU^j#;n(knq|`gsJK80=mtOV-hsP=l9w!ep=LB zUzs6tBJcZ@A}1KUFwPItfFP_&}l}C|+x)Oz;zgpl(xc&gi(E4Jr64 zKJDCQwNma%-eLK+#J<~GM;!#5q@+c|4QIe|*??5|**0Vxc=(DrLAQ&$~%A~c@c zN@Ckuj$-k8E?oS(^8+^7?P#Mc6AXE-VW)kkGgX^^0T)F<$!7lS&b`MzqODVJ$Cc~ITOhQo4q@^ZEm$n=uJGNa-RSXlj-l9%Z+$f6g?VGmj@`i#j} z&gEY6syuUVTbl8;I?H zk5@6VsfH>7fS^b-YCP%wDb zAB;;Zvj<=udq0s`DSV@ievNg;*gImjHysi5dLFZP$_!NOEOg3G9)-ecJ~9AQT*mxV zyh1nM??k&9P@i+k&FN8XstAqKZMOJE0m__Cq6&Ea-(qtLUb7i5H>dOZs77gLrxF%6 z6km@;wJ&bOut0zPlZXnAH-c1wX|}g=L#}UdWZg>Ohpk;IYv_Ep+l_nrwf3*>-wL%1 zW!V-ZYP!N&Rc8{J!agMR%6Yj{J_msc08 z*EcuWn?WrDWsr8x{Q&byiA?!9x?p`sO;+`nR?>@lHopm^|HQ-78VXYWnpOfo)>|11 z5PxS_=RKuaE7yFD08{M*+(OOk?n$<)<%FVHvTn-prpQ(A-Y*d5opcANssg+zSQSK; z&OEZR*v{|j&W#@s%9H}swR(;?ma;a`ziG))fHdm+IS{Lls!+m>Yb!20f_shsi~h16QkZ=0{y9M;~l&;IPo zvOgGD>iqup8S~2p_;TA?>x-eKh*pSLEe|iZD))!PZ|)%gLUEmzg=W`;6&dVnH&^*l zRUOv+Z9vv-4~Cd(NzqAut!cCW5I)}dOdWoTVT5BCGcBMI= zRv%4!Xy+N~P1l?*Attu!Ffs$e+!fvnOK*Bx{Gci^`PDBEa@v^=FRwG|S9vGPldsDD+R?6KwInuwVF^!aQPi4b97R$of8%q!7uR3%dC%zJzLmIMGeDXzF-v`{En z86&gM?6I{mS&RlJieqBI4$TWtgD=sZrqXl8ZpK>Mu}iBj^%YhRJdoj6-n zL)W7zy?NP9vA&)s&YUly>n;y~Ni~iEH$=jX)`5X7aZX0wC+uv@Gg~#7%NWFjmPuTg z6fpQDzXP~JgPF8_xlY1(dA=-*ncDJJ>ZtKJfYWqMVo85}MvODdFhvW0u7zA6uU zkzQ!Y=CH!<8N)^RmPpM}PJ*Exb&37ZkCt^cSw+PJomcaQ2T^FqH5T$8)d!{*@!Fs( zXXeVF4S1=81jhEL#^^Dqi^TAdbLZXVxYrL2A_Y7$M|Cu3WRNIA^|6xqO+w z2d88L6jhZU@9RmxG z#cU_PW!+U^&eB)LRPzix`E0Lz5^%6C<|wM3a%jR(4%zWYa-46%&oX*2h`1fJTJG4f zT5L5(F#DD1eS5v@HgnT;=R(*C0pd;HGBCYpF$#3JS*12IwZA|13J;GO6xoP8MSCrG zkmaar_qZOd4!v3ycqe+HX!-C6)Tc)1db*#T6qptWi0~BD`waodZw&CCXwaa@M2Icv%j}2v(MhXppC!4$+#jZ?0M(Z zTy6s4dU@`4YxXv8s+P$Io-lD5OV++**$-4vyirWlIq+*iwuea-IYw)Zj>0n$4-hNM zv>_TT?MYHjA&tO%h6gu##TQmD0-(suP8pRn!9r^bG@JAUt5tY= ze3RTXvj)1H8TSgqytBa(aO1qUT5KdKz_cPV9j+|FYm=wl=hDew_JVx-sr}qWENT>h z%d8xKnk>VYOv=@G%=+ZLO{nF!77kbrPd1n*o4W60NX8>vr}xqV8cX;$`FjkFT@ZZNdtTqTloA zWo4TvBQ%tITv>D{B$RlqX6_b|E|31 z;&|l`*dg3~KO@L@mCn|!Nn{uKM8zx0Kx`q5f_C)xCMV{XvFYn#w5XWtdC{M9`(fq9 z5W9O}{{n5A3K}xVD?C@0BK$xni5e6&JMGr3;Z&2}GSlC283_Sg*#lAr+pWtBO})&% zg6Qj!oo?E2%li;4?!pi?xjjm;Rtnh|)((mlxM>8`e12 zc-s`~rHC^wK}l2*y-CkA2BZXea4i;3h+R;T;Ojs`$uY9ir24vCndd#~aGpWaZq_~xXsXexw2e1zd^j*U zPQU>}<(}kz00&h>Lv1D~jXYnRbVj!r7}(vh+J*89-8}w)fBHh5GgX}J4HV>zvjo6p zpXX(k5+`$++@y5jyH!|%pb@d;jg=Z_kakb!i@e2Vg20Yx61?n+5ul{gv0E`d3BZ?h z-n_k;@X}=1EB7UBboG~5*UN8<0rgJN+diPmS4W!$g_F|JcNvYLIB%8VN(8qDI{U|g z9_1=k%4v8awH|dQ-;W?ua)v$tE-VV)Flql!G^)>0Qj{d?Gl9K#rOd+_S3nXl*@H#+r5&LE^1x#Z40}&DUqDljon~L zu_K|us2ggVX5i1obUDevAGNwnE)Cer`OQYw$gVoqBpShgKRX1~Gd--gV7f5FdxTI1 zk6b6FyJ;9j_%^RpYmdKfZAo)vK_FE!&k6m}aC#x=xt)FD)fx)rLK_p)LABl#sgXXU zd`Y8*%^N%4N8baz!q^I&+Ip|NB}UAYyYvIOJ z-mSFmf`}|!`ctLiTZ|W$tK2ZLCUUHE=wKHtXmRtiCy$xi`@DL0v%%o-lBK7>&#upv2ajEutI~4`3sVpAHB}4` zDeF1~Y26NOWzn5=SF~i~yJF1CyP~P1~6P)9JE-x5x{1CAH*-CBpS}_w{yI?g4lM9&34nJ5T+is`Np( zu(;-?Hf}8#A_#}S@Gs8$#rwhx$~TbFCS!h7v3M3)A915#$%R1XrLLjN`I^x-{SBk# z)xawMI=i?%8w+2{+lsMblSupO?of-wB|E9%qGRB1yP`h`YDk+!Gd~%REePBhkA68P zk52X4a8Iq8n|;IAwq~_&v<28%eVlk+#la;KPfy=j29@Q3|U*_Kp z#lpr7cx7(@HAbH?c@=Td`H&Vj|8{`!^OI+4swo^rt~t)U4VCSn=3d=}+skN(g93p* zk}~?0_8&%rSxGx^|S#iRxs+C&@Ltk2Xog_&vDUYk-c7y~(?mpH`k zHk!+up+rQeGCx^(|lJE z;*D1p(t$kC%xtlR*;YCVEtzw7W=jO(9!2KQ3@AEYo&s6IJprC1-dt zVrdFK2?)NKPNjycwJHL_f4l$u_}xbh(pDpAXwMgui()*g85&V3#ntNX5r5eP2CxZ{ z(0ZVbR*-bBex+POvlBFx@VfXVBN8wQ+jS2UpN{F-&Vt=hzR`QJFmfw&UQI)^Nynwi zBm0d#Vc;aP`{lI#MLvR%Y+WyqEomoG=u>6&eMTK>$FC1DYYX|1AVCnx4 zFgZkUr~WD=Xb0?s99l+f)~GqHJjezij`x9ZFpr|Nzo)2hGj2+{OpoHB(bgMG2x|lQ zk)q>H?PUs-e;eCGgthMI$Jm|BbY5=wJ-C-BceaiOcC3-po-!zt4=7mw z50131;$j4Iu#6!J?(WtVdUH1^z`AQL`F_XsE+{`$5Hh89WQKPOvNK{y#ekyJ-2+L#OLw%KB=eLken?{ zR(e@AE%6~$H77%O1gJ5!3IYrB@4SgveU;eX#w|l>`x)JIUhBheB5;ZDfUY>=y7Kcj z%*hZYpBtaqboHP6o*pE zZ5LO}fT-$mmFGKkX5_Fum?iF=&M_@9h4OtSOTC#J$I#d(H2Uk`$DylHca`*Uq(0%} zVpn$}EltU1Z}jAu;IjE*J+nFx0j4;ezO=7)PfVOPAz`R?LpE3y{A;3 zQ`h3V==-B};fM>6KZ+=qaH?(V%gkFC`*S&Ci<=w;W^i<;pvM9KG`LMwTOnx{YgX3n%gq^V`R{j_ey&!^HTeC~ ziQu~5qZlxOB&AD+s#)fBSQ0zW`(E60TI;5c?e^ULJ_WQtx0EuT0vJYt3dj!3YkSrNnAYRecNk~@4$>Tc zysxeQqa=DfGrPQiLEIHDFR1*wDd*D{_DR^}s8FAo>>}rjYs21?l#0GsI0Zc`M)N=D zGi}JM0FWqi+szdyGMrQMw-o_E(YGO|e+pEl(qh$8@A*^j*e+<%=!2@+d%sj$`Ch56X8I85 zinrjH=hfGrvpE7Ez!!N;g^kbTds(q(>QSgnupq##9(FZ9UpnHFj>Iwo6{Wxh<&h0pYDCPM> zQmy4uSgP?}uU@L=7h0{8pT&{#hh-1)jsU`BL|{{netz$|W;=DE2A{T`K1V$#vKKI5 ztRGUcl%S=_Pjg5D@>R1D?-3UEKDwl~a=J{pgP%Ta{A$lJ>zLI^LfoR439czN zF2nh?u60ZD=>vQHzvZt?*l73f?H`v=t0~nMOQt{b);nlAH3j~1)Qnizo94vV`%{8h zG`hlb`!`}?7wd0*NIS3zX7G$|i>nxGls_1K7{-hh*2J9qBf)9?T_cafN^ui$;QhP< zlL(kI%}hi7lu;v!~GU%}1vUDOr8Et3e}vK6Hd{PBHCnONJ;mnkR1m z@UTZ1NDmcv3kDCo3FiV)9fF~u$&t*FV_vWw?+w5~V~&7258ay|VIxS-tHPdmtXLV!;*fIHMq<2+Z~E}(O%pxfXk`Onyz)yY z^t2bffeMFe-p8=Q9Gf+^YmF!GDGkgftF2}-2nuoB)sk_rlWf_2>py;N!$~96U)i6UeMJ+X{&O)GEJ)Hnnmf-nD^(%^|Ee z7Oda#3Vew+xftWbej3f7nqwXCj4&!+@*<9OPHW20VQZlQb?9asbfRH0@=e@l0Vs$? z)+}=c*dr3g>x6;rf`IFUoxNtN!5>H<=lW=-5;%Xu5h3G+zZb)AI(ME+J~3Jkd$|W+ z_Zt0HNNQ%%l6JZ7H5KpjJmDB?lk`EY-AT-$h{xJ#Y5y;M%nHJ$IyL`FbC$erB*D_=Rm6u4oVo{=@1wYJ5QY+%m;U_pR;sGE&&OlLg?-Y45cpC9m}JAjS=q(x2T-; zB~a}A!Xo77W*F1xXuy3m4?UwQ6-))r0Kf?FD6vU=&jwlk=aeZoX&7&qDYkxKE--1j z*6@8C3FAs*nE_&22E`(+sC_z{pAr1^MoXWMjLRil%sB(SQ)^Uxe6Mro>2*(YCaBBi zIDPT%QJvNOSSU3#F!FG$c+C68lka-t8lQkT?{Z|*fZa*|rB~ zUgLFOZ|yY~bGcrFg!#|b?QV+@Du!;6zs)DOv{NyIs{`}dibr~YjlNImbkPOp@~N66 zX7TCs@sW07ozi!VqwalgDX`u%`x#NYDZ~d2%4CfSDlqOxspO{9-(xC9pU@fL;B8B% zN3+iPNn20=8>I$YAs)rZLjs4mV@N!T#Li6fjFl>eppU3 z$h>)W^C_V{5&A(ePG=YTcBTG$!5phOnyvGp#<4Q-iBf&REmn2fC6z#szqA6CL^vU% zb@4V#>fT?7oOmjHtu;IuEoF{^h0gq~ovzI$H zS2W^<6x6Gv=o1a)gdm0k2XWn;c1FAd@93j70)u4ES6BfQxd}a$P!7i8gb?xN8OybI zMqCLM{pPPgBAH?VvR?)6b4IiY=8~>OH^I?)_isK!xAEvZXE(kjS4-GSm1X&`NTcN! zOkbd16(Rmf{4QLji~~g`qo)t@7tfXd+|YQ7SZh9TH$jvg7>?N9dHI-!`<(49D11Mk z(4Pooc%A&Yb*lHxR9DS0;1_ z*x?@jfaA2xJ0gE;md5s|BoHiL^RoQBAHFV~4KDu?63G{PmhXbmW#%$DY2E94fIozk z%Xe9T{@!CEh~7hp1B?)(@NlQli!h3|TmmILAw3%m?#sSUh;; zS5w$)P%f87o9|>gV*Z9*W_{0|d{5@&`^h3m8z`#xmFk*ax%E6%#v|xrl$ZU|i_-1U z0#Ma_?|p6=7#O2@IpZ@VD20sLW2x2`1YLA)JTQR|00S}+MQROK_)jh zuhwZ@G>EzH`pu`QGJI*zUpSJV6a>D(pIiXy^W5Yom*?*VdXw(KQqK&k5y9L&W)JjkU@G~u8;wc2&+6*`hq(DRb7Z{qcX zL%dHROvTTww%yfi$qU`eW{wGMtDKLFwl0h1`8enOsC?V<$CETTmjBOEW4(pA5&($u z?La=*tq|p)^hWsHv$z{YQ;2g{0(2snMw9=&7&H^|Q^P&CXZ>tlL}0y>PPAfA*#&Kl z)FEOF)?fqaK-C^TI3!=U$g>3qNQXKRNuD5fy`lbQla}w}a(dS277%%0w1`t)6J7Fyr!L?1y$nU+g1v$TYK5^@^SO@1n$`o`rCi;0v_qV<6|q|+4PBooO>}rvR5Y^mse`@s;>MY)q6WWnI34{u zw>8A$47iP=aN$dnczeuQ<<-~BGvF_8{irJcwCVU&$UUarZX!n}0zbO7NjZ*e@YlA` z42YN6SKo?$X}E! z^&%HI7M(1(@^cmfk6f+Ss&^_PnrV0@au4F& z^n;zXmAw*p*DIe}$sP!P#2;3Y@1j_p0aF0K`m!#a0)AN&3$Cl2zW6u{qIpc6N1o|2 zTrOWT9R?Hmy;t=nRXScdp;cZ*kNi;Tjl6^Wtk|aFk-W)x26Ux|f;F2ex=)OB+i|TV zctmLaZVZp9r}aWfj9he1<+U+40_xJX9BbQor2B4uIjZ$}9=Y`DNb;mo{IznFH%{rH z2u@5`Wx#PXza0ILY5l=n4)tfjV}toor%$8v7D0}%=KO>G_~L*7BHsl!9ew@vwH`^= zO5XTl9_c(M)wU1&b9a;cZjw3Q%DgchUN-Q;nZ1u<6x*cyjo3{1PDEt8<`>}$pUFjLv9(p^^$@LwKo`O}yWn-E(Jbb5?(z)5y zHsPjdFpn2gC`E!+N2@Evpv3NQB%4!9K^*dhEg3fDiL}RIYq5G~OK&any8{(cQ8PEJ zC)Q$iaMO`9Ji7#A*A}e{;st@MQ^NP}73!YI=SKL=WGXQU>~tHw?+u2Xnf{=kK&6l`K z%%73xXSHq<&Z5$sRJ;#BiSK&i>*S)-6S(>FoF*Dl##^(cjAMAmxFvt$JE?x+J9mHK zJ8L%-#2V0n+*R!%?{;XiKNP&Q_@cz4xYlhj*9o4gwqO*vCE=S~P~z0*on38tE;PWC zkjBr=E-9V3cUx0MxbL`Rg_SktxG&XdU2Jxdf9K`cJA{+F-jY{GWf0K83>Xg=oP((= zs$F6>n7s7$>*7@H>IbiJoEtTW@F79_r;!@#DY2nhan3w@p=$4CgiV8GBkfc~Z`#YV z3~g4PEvC2P)c#59zkizGEh&B3jNFtMlzcTUA3LeijjW8^*==(VCqwNn_2}*3UVBmD zGjl{Qj@NYx%o2JqFj-(-yjd1~P5faS(tg?tug>CVRa^+Y_Lx}f@_2`=*~P-+)KA#g zxkcb9{rx8DZ@$=f)e`wBBu8eY%+HtmQfh?qjlZCa#85#*v`$$CQdFxm55mh)rux(p zrO8aS>*JdKCKiwK1)OW%3efqmwSeMQkU&vk5Ip*S#)p4t0BasFoU1q;XgH6_Gn^Th zx&8&1PSZJOBzyh|v-4G!WvdCd^?kvbX4$C4`I$_<*}!R7BGjroauz@Dm2D2wZF+Rb z720ewU9mzvP6%Z0GtDe?ahs$JvpQ3QQEPvjYQAR_$pp2Y4yC?hPxsMsA?ul2G207; z7E`NvL-V6t%y~R&u*kA!{Pgr%`Rg~FxAA-!6pbKn~n`^v> z)v}T0bYcQfJqVcaPj?S~o-Y#vQIMw<5EK1nKErEWoJlsmFEY+AUima|Gtxu|W<4-O zgh8fvz5+6nyv#(KUcQVplXg(!xeC5JB z%G98noQxo7;3)>iQEaEm&a+e9xGLp|o9{ts3H(E7X z_0>-;a&8}Vs(!y90bq22tv*XbDFC1&<#Y$5b4f0C@r8$FY0aNZ7Vjp4?((e_hHnPn zw1k7>%7!g24rwbY_<)9aNO&r%Aff`&r?db&t7Tw<-aDUA@@X!!3Wf9q0gc66p11@t zrr;IfWhfXtABrFh-YG2h$MXA9@qa7uZYxrLb7NZlbS&B-+aE+t%)LK%@q`0|M+ltu zmKR0nWlv}KQ@{QC4IS=>LDW^C|B^XEERTHB)`H6XQ(L9xYdqjC6)4PqaXeShbED7S zyQx~kibDeZ>m9~9uPZvQ=Uvei>Z2m}=m#(ueu4lPB|xZC=Ngw>TTy8-Jv44%7gOxQX{3?lR=JFcs4QDY)qqEw93T)# zh&cxyo@vUm8_OF!xp&mfOqTd;*YN$Yf}im^*u^jp0flf3$Se@Q4iuH0I zSl-{@Jou9O#B!5iX=%wpr`+eBcRRXgxTw=OZ^bVXEvgx$RHep`fjq(rOmmB3?OE!lKLD3g(DBEM0i2&w?PqUk?JF=IaB{ z2?(`6`gvY#`xr09ZS$^80HeDe zq)L=Xld%77OkG=CdsuhG+HPxYla`K(VSSa+#(FC^d#c{Vmj zB9Z(R`NABb|6%37<%-8!Dd!xihPks7k1Q>53J+CN?dz>->tA^=r(RNA8^kA=@9&HdHxdb=f{5ubwB}CX?xOH8opWB z0nA2TFh^Wz`kHpykkw@N364f-8GcUKBERa7NN zzlW56F&I#%10eKd#~VMfS;Sd~(&~-pCM1+OIm7z=mQdjhAB7~RW$z{k#OD?)4169j z5Z+mU6rmnsB2E4vHo0iLV>R*OJEj%IsIWvDKKkT@_@O?G5@(7Xt2z@5DM2<=3`0BM zS!UH-Y07Kr=+T4Z=~7C8KQ4OShgXT3wIa+B;9YQQ*MtA1*}z*0gTGYvxC0tpWJ7IZ zXHYY*u9uh~02?vP&Dp{k$xFFZf0Ki_!jWY)rHgPON=6i91^Po==9+``0Oh<;2?DO! z{w6js%dG5NckerZ4`{^9sHC7)aaylN|5acE2QvbJ2)IL05Tt-1sEwk$Dn>*%Bv2e! zsXS#UKob5P9vU0`H;VsJ1a21;4beS5?w+zdWJb+UDZNfKEY^4Wl3Dw{UWMXWekkh> zu)LGo;r|cI)9r>%%v&PHi;%*RBmHnL(dSNRYrk?Q)%B2YX`;;mK?4~2ATyXo(r~Dz zGrX#C3w3_rGjpun(6-ZH0m*XI)8-p17sb8$U;g4RCi}5-MkIe9!aNauL`(ZWv;!+&Y^RK5$Z%>(mrPltyBclx35VB&s{1 zjxk3t-%R zzx$8K$8P^J{AOEAgeqc#;@jFa9lPWyzyT!cOW## z!7>$m{>#KWEKxbLFC5LKZo|ii<_eWT`MPB?aT@(mlLKJF>fo4VK)yMC&ndI;-TIwCBV|GQD--2clc%tc{N zJ74X+Mob;bNsF6}jMv8vz=fovt+9Py^?j4{Q2Y zY~iiQby{3Qj!V>AAf{@rs7OP4?qQ5k&>t4xv>+hk1D8UguLnM5OZk|y);a1pUw<5G~H!4^}ZAf39%PU;RQCe{g~u#!A=-uHt!!cTxV^Y0!C z&s^s(AxEVkiTN?h>MTbX+hcfwMHlb6t&MeAXo*}}VMR~Ex7i|8(m@-C7#cNl9i**u z?}jd{NA?npacLqEg^g5Ao~vOZ=GNn~ng8$%-dbNwz#U1}m=xrt?O5OC77>Q|cA1Rf z-cgJX&x0Q7_#k4@yGSS*x%=t04F(;EW|!{upf*T80w0xIUDO5D)6;W$4>!ep1M?iU zo}T2?;n!qgv^RW?2R>X*%p2u(|6NaR$}u(Jj48`p^ri3Tf}bsmtzU{pb*-)*j9|7W zi&CRW1Ne<9Dhn&Xv}Mi*`(@6rwYM{|nb~8JJi@ByO89p3H#Sqf_h;-8$9)50`(MF6 zpUTx}{va?st>C}*mKlj87QifH1a>`M1v+6SPU~t5PZ`7WP6ytDFdvmX{mevUr8Ub~ zW){MF!`zd`3v%lh+Shz0k0xix&&2m9XQp&^QZXMAFjPcPWU3r3EuK~fC5r&r--9;S zzsxc{5^1Bm6bpOBYZs>IOsD$=XRG)dGQfc<;ScK^Vf`x-Sz90~sZp3(8PADR_5_gd zMnpZPa#Qv)G#LtCzVT5zngo401?WLnzH2(?_LeFpy-h$HlQqm}gt$~w)4uGfNWHK-lD$wsdf{)o|2?b0 z#?`p>ue7EMYB1k9lYN8G^NcfEO(ywIZh|Qy?=Q`WL}sQ0##AX3pcT?MG6sMACg?BKFh^utr7dnoDuk9MV~xRQ-jNLoorve!t_74QcHCv>9@xM^3jIby9pl;_epfG@`Y~v*j!)P zq-RJh@%Qp)9e|8D2K+MvNpyj+n=z%m>I6A<+Wsnr>ua?B z{gc7~S?!JGROcD^wBMX@Pudm^cth%@)bu3E9(sn`JpzTmjc*wFOKUmD|Fen#hgHBvqw^~Z zw5qN@Cx>y-*gdyFaye-#44o1=*`Gl!3^c2RqL4OTw%^=ZWZD1O5|)7^Ge@M!3_2u{ z|Mt+x&i15>0y|9!I8<36RQWcbU27?&E(etf1jdNQ%+@9Ky%3+r|7fRQ`hZk359%l( zeFr^09esxq%m904gX#YU0;M54A3sOD(Ob32%1HwX9seLBr_K7C2Aku|7W!W_5++XW0IYZh;N50v}IZ577?VKqDGqc zZRfms0J!@cG>bhB)}6L9!k07G5G)3d-H#&+rEGNJe3ql8x^6>QXy2f>`GGh z>@#Cagc-)ZWIYiop|WM)vd)Zs>`Tg$ZLBjG%aE*N%})3pQeC%8_uc0Y`2KP|&L6Hv z=6xK;`*l2D&*k-Qov_yYbBziXLHKhw$Xgmp>3A)bXT1Odr7M@-@T-IZv&)8oBH^eTQ+Rp~GNgesdK+m6dzL~u#j-wUfg zq}H`XIcvQKh@6w&>C|o$pax&ZD}*2jtL$^whI6kzck=9oLXzG)jo?2J)K*qr72q5j zUNN^BSy9k0x&K{ycF%ME57hS7Wz9m70M)*FZ|cUucPoo>djW^?!yT^T9$`Y_DwkDN zUJe1|@>w>PntOY-&$QhvMjLuiuUGt~dt^`f{|}V=zV1q#2r-+H#w3faNm50{n@6`m zT7AFDx3#3bTakKgXKxpRuo;koa%>q2RZ!t>X{hg*mtv8HNi50F5O-59dZ_v_+2@iN zxYVjG)L-*|fAEoM>rq^^am9wsWLI`CfBK?{_lDkIJXCv5&`(eZ(7-m?0T_qJBgw|2 zQgqCdOs92Ws<4~K%uUH^R0Mky+XTyv7ZP%W39f9%5q#%#@?DY$_3R6<>?-(m%kx_j~?HFvi&)P?7F56sl;e8S6O9nOJ7bl=5oS+=u{>^RZAY6L8~8R z_yip?%vw6s92R0F8t@kcA^uU0>m#0YJ=Vwm^rx^8HjxZmfmwGOf$+GAOePCj^o#QIt7~c+ zQ_+CXp!4X+h>wpCwWePTpcp#F9zA-*$mb-m`FeVKdVJip>6Q*ZKR>+*CT+MUghUE8Mrn+f({Rl1}mHx{N1H2%gIw>?j6~D|u=Z zXGBvCb22k|x)WxBt(Z$5b)&-`8B)+ipT9h(kQp@ejH;0vH zJbwIG`{Ur?p#GM3!^AZ{?L|>q*9phpbkT2j3LT^hZHJGr3&jG~0fvsql_bDfJF6fl z0=E3@%A?9hjcE1w)SRY)H5KXC_1Cb59mUjKuW27+xC_+ggDxwvNVsYHLAmNW3_3yh zO~$Jp1Vs8LOb#jxbY(w+4#67~>@-wg2;^M|#loNIb8^Y!k45GiVTdHj%uk!C5t08A ziMKe%@ZFMljVz@LzaebTUEHa#es8e?*!&(A3IM2uE@+6{1HY441wmY)6SWz6Npl=1 z1oij&=%%ob(uZSZnRQ~pC@EX25bpyrhm0`Pa#+oDRGgS*NFh|Sl|O+=5Sd3yb@Igl z&X+G=YEh@FQs%p1(RkW-^+ff!va-IWTh5K}3lw%qWt5)n`8U?FjMAz4v3GyNv`XIF z>6Ag!0G}HNjSpZu^R)PlftssFPB6qUpacX2h}JE@xKVgeQ3P9jP1R4)D_;8=0^in> zk&!U~k&X^c=2)SPQ4uOkRTnw*6vu(Ynx51m0{T*2VpO&Ryk^RyOTlRTmD}Denpc#5 z!vp-_7OLx^kv+m@4-gI{eVhxw<0~{?Jj3q|mPN?As!afkJB&)+t6pGWgjg-2Fe%8~x)pKk=6>X>&1mST{pzESm{xZ~}a2C0L7a@H_Ihe}}N`CyE-IQC!h0)%8P# zUTgFfA_=M*3f}qz#@_w!Ctd~w6#J}+cYY46^ z{J(fQ`cF`)b4wlK{wMd`uYY;G+MzhsnxUO&Sf&e}5Z)`hrPo-Zul@7K7+3+)|A2$N zDpyi6FE4K;b$;}u98cLvQRMI`UB}8C0)e0b{?OHGG&V7DsETfDzhDDxd9g`147Sdv z^4r_>JDpplL?HGFn%qKIY$@aTu(P*vG}ggaHPDNnUSDW6vs|9;3k~xTuobhPd@GVf znL$)JM_gK&8_KwC64a&NbR10o&>Qf%=`gLG)sqv79m)tV^fVZKd%+2T_+JXn4&f$d zhye(JA#aH;;3Q6MKAH`IrJFk51Pyrxul-l~u}ML<%G)BVp_+ki{lg`$>q`UdA5ho_uo~)07%G0+EYsfX;$nsVn_ph?w&F|3D`IQ}vWMT{U4o&C+lx ziQ6%1`_-1?L#UgiDd%NpxBer(!Pn1VIZebi1eDxXRSh!lPLHyO!|OS0?d%};P%PHd z`^g+(Z?Wmdk@1A#5p7*Wt}IGfZR8AN$N#?0-!(O@X|$d7JVOH%bhTf_YoaF&>DQ0EGV5z= z+8;O8mrI1fq=>AHtD||}a!X2j)M+-N+?||=IkK8J!3PuKrF>6%#E82R71>qQbKo1W zO1BT05dd1WWKcx-+-TO+bHx)Gv^vt=WPwi9p>b0nbR^BRM>n1FhOYCWL)1=G=65vp z={(-Dx~1=-IVBl(N+xI}H5%@5Sv5gUvqT;=dI#7DAp}q#1PZ(vHQu)?>q0yDkHZlV zQn~B;?L)dADJM>x$aweyLaqzCY!i_Bm+EVD6R%c_unO1brtW%XwXfYI606zq?@L_J z-3{qMpW{0*b?qo4cL=C9(yjw(d$oMXPj1>6Q)!2Zf6KRGYr+?W<+wwu@ktYD0>}Jo z$-i)1TNheSvVRU)D{z?US60&+7++Z_M3@}H+}0%56R^;WJ>A{oPO6sM7fS|WYHY0K zLx(alE#r1U6}5nakvSn9DV2-Ap=o|gSyS^AhE>#{=VX+y9fxCS>esQcREBoAbOvU> zeNZ@#j@d&j+zfzb_a4ZHfQcu=}5|c$|(qUJsSKKW`FgRMR`Q%hk zM85pyG6!ord+xs;VsSS5{n0($1lYm4-46ihar^N0-_Z;2y?&}7HO~ck)qTnFM^e6&fejdGB zYO_ld=lV?u%6DeJ&u{LkqxUZDAQ7Nc2o&Vg0l!XDBXI2avIutxUMtN{7GU%z;03)o zWMVl={RPlI7Ys3V+&0iXbqhR`;}1y~ym*Nx+FRf9Io=U6v-RcxP_gChpI7sLHn>Ew zxDpFX11G|9O)eK&X+M%NhS$dH!XqKpNXAGhO!^mg2qLE;O5=AM&hpDBl+LvuN)(p5 zH8z3euVl-#TlBA_4#g@eHXjI74F%@1|A#gCu0Tj`D-d$9?+U~xs25{AP%r-Q*Ry*; z2Xgta7}ojm;*rBt0BaoJzissA{okcAJO>n)@&&a^e>}3>1_WXU5$Sxe1Nc8qNH;*$ zA6h!fi>Dy_m&!Ue+{YZ0EidLg7A)S zeZI&fNJ-XI2SoiJj(edN1Yra7HA(@5aeP*uB_ZACg)sufk7WDv-v^z-i~E;TjCBa< zO@C_u)u0KA4M7RenvlmbEX*Ek)P&||^t&OF@}YfK1Jxa9W*gW{>1Mv&4PjoyH%d=KTsr&ig z1wt3TWwE74Ld|TRI#r1fF?l0}es7Kb_g|_X6jwMFm~My+@2lUn^V)zHxb98!>pAw$ zx|T7=9F|a9&Oe`$l}6~g@7-8ROzu2$U)l40%^ovrXCix}3WJh*fGYZj?HS`jOfjF% zkHeNY#096eGt>qID222&A2$bl*Pw{U`QKHYH#?W=382n8km4vYNftTj!{`h_E=!$+ zC#LM}e?Fr>Xnzwx`?Iq>R_nE3ehJ71 zw1pYcl-v=JSxluNqiONTriU0otYX%X2lD|VR|(MS4tx<#z%KXy|4ff+ z`mr--d-B&`v6M*-(clbi;W%}qiK$-J4f|XGAX+BY zeYUBBZ1trviSn||T#c0G>NnU7w^sOWl1sAar*eYrZ8e$L2*QT0)0`LCh%H)J8ipw? zcG;}tHAB>w3aOMzoz^!p69f0v;x0+*UPbL!RcN`WgW7*0m3V1g*QwelWm#f#5nRzU z!iQffifv3Z!xJj7HFlM2YP$hTJ;x4I`2_uLDS4QKR!7AkMZbBo81s-g$oM;>?9{zZ zYe@|-(F#O}*z`C}XEc0lCTNr-JKN6{s_vO~$>dFjy5}J^;~*?QQ~$quD2$mrkPtbFY;*cww_2|UQe;XLd{#T@AU7!ldWb=$ojSE6>nn`rG+Ol?EFH>r;Wo|W}pzsSBVmrN@q}i`5{H`{n zg!Q8Ry=zF%?321hX~h+!tu<0_(k%jGwR|TBQ9{O=`mBUQwd;PzJDhN%NZ&1f1qe6q6u-fCBOe zzpWCuLumNW1FD9Y{_-;nuID7WA1xSNI=wvHNwPX0=oZO3b#3JJ{-K)5B1!)B=A(uf z%HZC#;~0j?SnBhmyr(Qq8YR$Urf;~+FSREys+`$XC4TyU8ltfQit?lKvQu%^S{SFM zGyZ_BRYLX~@Jo3QX4gewI5y913wiFL&OJ6?7P5207Q0H#3!0x?IxQH2=C3SF$_;+Y z%3U8N=Z>#)y3)$g9fv1Oth&nR=%&+mZd?eGF8k&xqi1i9eUB}3j!Vr~yZh4w^nGQ1 ziaz~08eB?v*$!Nc_!LVEFgacqLv*Jxl;7Rg=93&cqxlinNb9eds*aE;a5{aOF`8KF zjmDFAYu|N85cDs|9D0U=2K623PWgo4A3Gc&ZDbQZ7d2Q8nNdd|Sj7uLpd zB??DUxV7i6!Tt+tTkkk{5`59>u!DJdc0O%teOf|=U$7bF*VS(Qjiz}NQV7o_v_QDL zF&6Mc%;+~Me&rcSy zqTD_`!ODKE-ZQ0`>Tdp&c;4vj`1ooxaloK*%}?uAk0`P`+hWvXh`G#*B{>_wv#FZ3 z*6~&5^?+%&xi2j%@}8Q-C+9!ri8ohUhZs~Z-;J7~bxjjr{j8^>m97KxbM;c&`ii-n zQDpz+UQ5)aU9ek{z6`axZHs6qHA7BKT&=BuoKt+dQ9H5$m=%TEgmLv$Eh#$@4Kc%%X4Wiy&dPOv!y~Bzl<~3B{%jl+n4T z)xPEp`)a2m3!}9fT-(>DS2OMYPDu3If80pz`p}nljY@Nkoi<+I`dd$Ze z(X~chPwE$UI-bBZ^#Dpx&z#U(Rp z|Ih&{d?CZp=fq2oyrZcCN2A<5t#l;XVGiYAYnpnLDyYkIom3an>95m|P&d8n`gUzD z$}VzcS$uuNGms*PXL$as6TU8n0g?2e3asL{8GlTPZeBfa&+q0`w*gR)=C+6XIV#AftU46;0(?%Pyx6>@+7bb@3IwT>B5w_ZKc+ zBuPwQCub-6=%g0=+L`DiismZ?GtlaMY?dIC zl?IrV{wqt)9|^u>30)IYr)-N?6J$h_xIQKK>VC2I{kuxVGoGEg&Tt)}gMA!rgwYY( zs630$KXs6qj*v#o*+MV2q|Gk{I&fVmT_5D3kL`E)>W!>0sHHC$DgCTlvD!5uy82bA zo-5>L-_}Z}eG9$yV@y(BG#GSvZMfx`=8S7La1&}FCoN+C~N{jsX&@~FQKs! z!UoYPkyxB~h30Cmm|5@&1QnSAWVuUf5tvRB-cDHPO!cMH1r8?KGs$8so1Y7i=Uh1^ zFF(js({{4KYOJn#1+uztZWNQ5=-gl1w6xqJb37(QIO3qsQ2Yf# ztWboxg(?n>U1+u$Q@`vC4W=hC9V!=GIAb6CG%|2r7voJ5Vav<^R64tSchK#o-!76M z_(weBm)I5Gq&bCy0B?0?1}Uz`l=g7ocCU8?Qnn!ldOdZfya{h-5NY9YfH9|!@f1p0 zSvSm6v*_H2J8W+Y=e9jFVLTGbAinZNureCDNOqqIFFBQ)^;jMN~~3H_xb3nq^I^iHPXkIF?pp!S9-7!2qbFk zD04Z~A@UKCyO^kJxCMg%rpbE3P@mftd0;M~ey&!1EQNM|p1ZVyxsK&Q^YNqFGGZmJ zgPkxnx6@}ELWY`!hy>tU>rfHLAI9PnWJEv{Drd?RWrOSE;X180MwGa5 zCgQC#5aJdYv4IA8&?{qp8dv6?v6;=~5@cex&gao%-e@@o29$quiej%mE7%qCdaVu+ zd3Cpk7- zrulAQ6?ohh8v^55VQ4<90cXw27+U7*QE#0OK8?7XMZFYP-Nukt?!;f~oKm0W6657{ ziMzJov$K>1!gGeU$3AYBO>c!cui4$hT#9=6J!E&8yt6~0=3IfG+SFi}^yu#lfxIkJ zB&?!Y(|8nHqYjKBm^Ah1;eS%*toCi6S%!H6+VR3huK z9_#wC4+84#7{9AV)JEX^t2YyC>rrp`D|*E2`jWF2jU z)+QGmPM5Jv0ZG3QEWG_M^OJGlTe2Dt)114$VIP@W zz7cAeoBtTqWRBiH*fo7hiO8SOXXGWKv%gjr+!&MBtc1!zHCUsToh^G9>)WDf@W7%Q zHskiC{4Y6(p>D^s|H3ps!bD>d2)xA>utSytJEU1E0;mME3KE~7J;LrsQ-Q76X|kj` zgX1Ytqr z1L-9s4wvh4Ue2;M$5vzuxLCGpe9dg>T06|pwkAroF0UP60j|WH8QKPXb47>)=Qt%v zyeEbYH|k#5pSqiBzj$}q3BkO7Wt^I#qNBXEG=TdWicAbbmx9jX^-yUMb#ww_`~|Re%2LaGBhDw` zFj9w;prZsDBQ)(h9Pw`P$q|Kf-R*BdSsMEoKI#1N>4`}R{h9GTx^gQ`8Fs<0oc=4j z3FJdR8VX}VdZaFGz)P4saoZw#5+6ztiUpG09kR*gzfoD)<=3I{Ie~!Z!DglzVN!Us zo1a*j_viPhWQxfs!lKPf=(?>!rW~^@w@PSrq#y`(^MV;Mv!QgDo%+h_8He&vY`;sX zrS2)02kfDbMjY{>SQb$?T)#nIpL{lI`rqV5{0p|ek8V=Y-RQ-MzL^={q%hS1-XOwg z(fgxwN|57PjutR*IjzbsmBnjJ=SI&|{9XMdm&I*Or1O3`>xHWEr^35IAo~sotTHAL zCHUZHOl2G_JnpDD(-cO4&{HcM%4f6zaPM7CwWVXg)XL3Sl2npY8>Jae$bruCGi$|k z3W`3go$qs(UGA~AyS@EDB~*9Kw52VDM$eAM8DXrx7T;vPlnSh&Cz{;;!H@=&8j*p7 z)~U`u<8wIYQ@j!{13m{-xdeJPDe09yWNq!t=%22m+mFAq?53UYu%8|n`$q3$Cv*64F@B0xt=M>AYsLOo=PtCPh&+!)V@oVr+FJWJ<0*rh<694Dx&v|zD&#nS zbvUDyI>fcghL8@EDRM6&Cv&ZtS}suJkr0U25695AOi76NvCi`Hw`&|(&dnJ$WUqj=Ii^p z`111BO)UFU?{KTH83m;E4m-!eH7^qj8kXn0#$-c7Yq_ZPhY(lId!SHG@OHOvLx8H) zNokTd7h>Ys&Cgxa$RSCj{ACr<7aFO{#}SB@mjn3Gvu|EdL}^(CO`Uc zz78z+mY`rxZYvraoDYpN5u0%y4;d(ox7Lx88+#_kh6ow#ZYVhX&hl2zotrkUED=x< z8cm#ofFkmYcBj+ILbZt|^k4HCx2RI`Vq~DshBNEt4PuX{%;&~#yRTvE4_%kTJEzY? z7ED94Y&6e@G}|<012^4|EmkyNH*ndAUbfb_;-b9uu)#G(uBb#~@ce(l#h)orAnE}7 zx>ies&1*Q$)^;_WAKEp+cqHH@Mhjb$ju#Z@L1q-Uhcaiz+-5{EAj?7?dB5*-cB;!_@nSk)n`J+c6P(l%jYcz*B^S#5wl0O_DwcZ%!R)(yAg5gLmEI5$ zi(Z#+v!^PsiWw3s7dz_3)E$lN)C#xiEb`!T(fjGr&o)8vrNQ|nbgu5T+>4n_`?uHqh&E%ge6!@DyZzlDU zsRat@|4t;`k0=&gb%3+P`m>c6>tS>A_+>Sw0yx#bB_4+g!>j5_P8zf4V#CQ2)Fz8* z;z>?-94r+?jfdY0Jf&W{?p7{?3+ZAM>=LaU6`F`0xc6WiN*~n-8p-D_G1bS>NqHdk z%F^j(D#-C$?8^SC|Gj$OeO%k71i!mnH+M`4)$ahN?(o?s#7T}T#uN6(UsC^?2iA9~ z_^^%@R3jzs#_7eU^@6c3*RKiF!;O@@>a?;3iv%rta=M}*D5on)QFmoeQ4-h=h^8UIcb9hQG_Q(x6yQ)11Z@Tae8g&^5a|=b%Z|6!=#hUP5vYI!riVI6 z1>=!RjnE5no3{@Iiuq?C>@=eiShzIu;2ClB=}`65vuU&HX^{xTAWBXgYbYnt-x(pu zpS}2$@{NZI(Vp(52|d#G4CYd%^5@U~RPj27UYQ@<{4*APD~GJ=0l9bnN_8uZz+q&= z890k;@sl1T*%DB>9400rNXqp^Cq7ZNXz*dSEseJk&bOEjjtOo;nc=D`=SRPms?^{a zPMoq3&2!c-%H|sbjON6dvvRZjrqQ#2lFhpzaQc*tV52-Z6#DVq2V6ae0X2G~Svfwq z*(MdG5>u0BEj3&A`d=qor+(KXi{atDUtJnmj>MJ^h~HbYZf7}i9rl2BSLc4kcSi&O z5&sC;74R9mWT3==0R$uY0gFr%D&sce_HV8t6v!W1*YE(huu3>cFW0FYjN6}<8YALF z>~m|F9?5R@DT62X&9n45JaMqx@jf+W_b-iX z)uq))Y{bT?w>Ca)bR!c?>66GLH%@Agz}%_k{Oi~ zM$nF2<{*pDC}5&jGp3avtElz3xLTj}fFErsOYJY6cgv}rTq^TZRNMY}J<{%j^1Ddg zqp62~nrWZ`H}lZ0+!(Z-d6N%-#bw_BM4jIHMrh{&Uri;0+x08PK|#49dfrd+syAQS2kM`;w0ZP!$I~&ad{_R7qbl22(A;g{EUB&YUkz=$IFa|PM&Pc-WSQHv7Pugd@o95Z+6_kHWPytaB znX^!WcpX%(;>-ANoKBK)@<070J3keOFyqc~E`3Xf1I7|D(b8e@1>ABK$%V=aWJ}(=j&z&Psszk+UMOZv(=k>W=UyCp3E8+T&_IcVVZ~01>3KRwj^j zZ775clpdY7(`)NrRts)eubvX29j=+imV%Q&vklh<`(n51@e(d;pFJ1Dkd!du8lfyO zbQ+~UvS*)^f7}3otpPS+60skT?;%Otym3vG*e=*bXw`vf#_|SfXTud9$-&SvHV7OG z$NB(x`RK24nB~>3>up8Su%Ip?#N4J|IF}vQ0+P+Ro<{goyoKZE@i2F59(%V@k49Y3 z^F96x0hbiNW&pTVzA#_|HrQR-20Td}00%$6Ani$W(lo%|yf6aPaO_~SW+vi0Dk#OzV zH)-D`6wA)a6FM6gXf%$fXOTKCcVzf%`&S=e%LSh+X*yZ+>=van=hl)nqT9q`CO0+UUuWA% zWt+G}Pym`9eJtC9zl)?zT0g*AqA6y5tS&1K}z++2@VG(}Puij274g{X@+u1|VkNSgM-3`;~WHFUXtfJDp z^}}uHH~r2|K>->f15RQp1_C0i-iYE{kRivjV7L)e6U@)r-+i43euL#e+dSynkvF0! zt~c!SqQp?p>a0_XhAdK&_csV94JXH65{hMr#B$tg6uhr6;15&!O6eq^Y#CGe73|AA z7C(nt7C)?!pQAzomqweI>*}I|b@^QfWn&%w>Q3O&=T_!*oM=ocFh@sGdWTo8aaaI8 z2F~#5)nY&iJiUnb7d7i zgCJB9B~#~*+Kj53Z%jlS_E%KXea1LXpY#S>%qM>+oq;dkcA*fV9XZA#?)o79^Sp$1 zXF6-21ZrzIHn>Bqz^usi@>#$>RA)i?5ViC4S1EL^A=KfMM&2?POu%*<4 zR^awigxuO^y6^8PUS(C7F_n7D(m`sozJ^v(ZqM?8=5#dccn@dW{sVVYv?-fLfy{@A z-(CgE(u|y*jPrc5xD)`YZ58rbZ`m3UEi~fEewTIq^09i;s3Q9({u(WA2Q!Sz@&fuR zmA$KmeAcX#rgWS)Muwv3Z$A5R`Tu7bLb!g3UE9(`Wm-O^R=&%y=lw)vk72hN!nxCP zkrR6x3M497E|ZtK#CXdag@t^hN*zE5F{<sA9wUeF+S**T`?JA+B zR`7xS-hpF%wkMod{7ByU zzIcU9GhB(T=&`AkV5`KkaE!Q6Ywp=wyMdzQUmh=hjCcbXV!(8nogLXQ#vlk7&GHEq)hU*&_>A+4Bi+Q-;fS#r1zH8G z2sL=LT8>7aE(eaXpV*B!q#JeHHr&KyTkq`io7mO`6nopne3g2Ft&u9}%*QkOwz_U8 zq0U}?iVWnxCSY#7T(KaFvzqE$^OjO;NNujb&I;rs&v6N3Th)~(P|iB0O8@SwZI%{< z_~WMfeJgC6yk>6eNcq{j+ttovvbd~zTjcwnEEF+u8#1NW?6sMvs{H^BLI_}UX_|dY z$itDr45?1Q`12lP)(~GeM}g5NZxuMOP!0$Rsef|Ry1$sbj(@`~$@#6oO+vS@=Pd*^ z-DY$qZMr`sx4CygYWMOkDsi_Zm(RxV~kn-1+XM;n`k68?&@w%psU`YE` zW|n6 zl!NZh9A_I}dN{43Q-`&Ck)@sB!6>St=u0 zET)+}n0qu>>$!ehiz1Emb(Bf17E6WRq5L5>^Z9)Ya??WA!djyPb2j#}m5BzLQ@QDO zSuX+rdq2v8gYE;^S*$B1gkH4-2Ts;;dptv9EXG)~>&ZZ2a*B8D(s6X0n0_qDflrMT z!=Z=O$p71O3SKh6(Oxw~+^;=2m{MVL0#|TONG&$yKUM0-fBl3cJFNcNRg>sw0Dt_? zaHT_)w&D<}Nm;;BG$e@r6aj9m_?Op7^nGI23~t4Ku6MzIW(^F-rc3qB)zi9eLZ?Xs z2rg8Uxf7X=!G+k|#p=50`@DOqbzG-p%cfJ{%QZgnN3PRi)|ZINGLv`py>kiM%|-@` z%h#T$jg#W3c@S6@hq1<8cy_LE$KAh$mbLB5x@oJYU1r0|RIqHGSTV=X_zxF~bL(ob zG0q2S56^xIm$Q%Bo2*a^-04M~8lZsr<1(3=W_}jy3Fn`z>UfDj8D-BgX}}eLLQ%fo z;I7~G3ht|jrUKr77j1xrO&gx_t>wr8nr(jHpv))KUL*987(t#{p{^ zca&cb3|4=r5KnbbcXUJi!XtMs7$s9Zj1^9Qrb?eQ~ zx=hSkroW= zK6Xov&BBZmeRT~@yxs7A3f{q!d*IGh`X55dE3Ey}0z``_aZ|-?Q;2WP%uf@8#4~mp zL~81~0(p`l4q9I@lcY(l-BCR3>XAF$K^9~!>CaQnwKOVe51QB}G=DYg zmaFTNA8bHL(>d=#P}|h{DZkK{L^y0wVEKca$S}3Q;5zG8F$UF)gZupja~w}T)&yPr z_yo3?z7C^wxSSRHr6eCHl(l+gpW*OL1VW&T?<+l0Y(oXIFgfomMc=Vh^HE7#vv^(I z2Bw*Mdp6Bp9($6=--u52yoUo7(AT;3fKmbmfJ*Q(F^Idfnrbx*s-_+=e*hq#6*E*57&~ zGj+>QM|*s51VvxTx@-nQ4qxP`)6OjD042WwW8R9z&=Jip9+6v zv)tLzh#B2BY6<9?jE^0{z7XG*=NIGbvM|wQt-_TT(q5Jr;NcBVZTdB?-u!3pbkS7( zUmZQ8c4q{yksX!qT^mCT7{o|;mxwB$euR(WDFQKZw_BOcYV^+1oE{zZDi|K@*=B0$ z?z|P7_5P37*E^a1ZcM&<*=h2^dWgkzNL)-{F%e2|LGXTuCuxqR4oWr*EGau7>J9xtR*l#aOSBp zyUo9car}vsZWD^HfY^oCB^*@nTYI35?uWAuS51ZPyrzp5%`Br?X!!>})@>UoS}~LA zC$_lEY0)cRFz-?pU2fS|_I@>*Y92p$b5E`QIUw>4HuShNw``9gggoC6v#vf>+&b#U zSez;}E!k#$aBl5MdgL>I#U}Ia?3r&^hbe}~#APhXy{dWtY05nv!_i-<#~(lV6O{hq zX0m;s2tvdAvqnG(k4N6wNzQ7UTahJ1s0OErlumQI{>?oPE-Cqo>6D!;x?=9v1l4(! zTCw;*BHy5>8gteG<3 z6vf6K1{Xksiwr;c`tSeh6$s)pXSB~AKnGs1VTPZ{QWb+{X;V4EFZaKEoUx@@&$QGw zZ`^Z>T251cn>UW9<(iUPR=@T?r&r$K{^`U&#m2*Bq?`l3LmPWdZ2!^JiynZ8+N>1Z z%GfsSa*W%lPa%pNhvJ>3nkyEO_ce zx|cbbD>Rx4I$&JLqZc~GHyeiVC9cEYAqC_9=_ebl10i(2eM7y`{W&;ue6jm< zEL5}eTdsl3{Snb?!i#_$Bb(z%&>`q#T*r1w;-^yLaq}k=ZTH}7o5}$;gul%LO{^(F{QxDGm5A~nRHY+@4D;;q>!s@*PGh;6i$ zslJ$#JJSj4`58@!qKhp*?cUozhYj4a%XKcDU)B2N;|I%iHuV3;AhL)a>GmKr?Aq#w zng%7`X&2tPvA}jD+PNa4bhGYLOncLH=HntgZG?hp*ON9^cje3_AMCtKW0w&+xky)p zv!Ho(iu#~!CYs<^Yu_YKe;0dM8w=p)ad9zz9FyBq|%2TysK73*9jHBg>%XR}WQn4?G^p)9M2rHB6h} z^sSeExu~tytU4V(M@p_bqyO!E0)L=3ykcYVDT{^ZnU^vCHecEj|8CIq^6tnbjJho; zjm|7ftPSPf6uueCskc>aQakWr;Cb}A&*d1m)yvBxxEax-RKB|`Ny@>$v`JzwQT*l& zaeiJwF%ed$F@4heS&)TtDKc@^Yz71HQbS@U^o<=zVDRERKY9`f%hEc1e(DgEQlG?^ z5SgSKz6LfJL$3gNWOnb@H7_Mws_@O%b<-Zl#Gt}XZx+Yn@qmP91M{-ioWBf)Jcd6r zkO-3FMqr!q;eLWI-Y7-=&A>04`ZbN*$DJyl|Hj@Vx#mMNiuFh{D|#kctB1ezF5QNj z1fRxpC9KS?YkIEBnn4!s!fR6$oipv7C{|jYo3iXUXzN5d--2Ek!`WxM`T}M2FU^`c z`fY$uQsssEhu8`JhyXCRhYBU#0Meg36Fjk9G%gP&QeuYaBN1%Qc;|p9`&BOnS0f>9 z+A`sWsu$2s@Bk+x)T<@9mUz>YI)6kl9G0wNNDdu3G;5i3YdlA!6$*n7XQg_0hR8KS z9iq+bS4W*IVim=-X>qy6W*F~cI(I?WS-gfw$rFb@OlA$2kJt({GP)jerTPhw4yH;n z%AC&`Z=L<0qwPJaezY|(g0MglB=RrYr3KUFf2YeU2Rzk5{@3UoVDZh_C%-J1$vBW| z6#PMi<(qyIxl#L(4I=d+|Mp})m!x&Gw@v|?;2=&`$2d-&1QLxhCjfjugJw}D?zXKjB%2P#*KPGC$U0dqDO$aYq zOAZ}i_s(GW2s~v}O6vEE7SLGxM;idR=ujz5I?YvmR|*!cwpx)Hp(73~Cp9&*;`hSrUsvyUsmET(~U9pEbCP~m88;!=}5m68RL9R|j=30#r|lU)V# z@8<&g69l_lklogOw_TUxRbdWgS{;{Fd+v;VEPX=eHd2r-6Jc|^*(@w&NkVIp-y1x?3szp@{zuGY@C$kTI84!ii-DgoEE0-|i9SNe6!oWgrtQKArL)2!I6$$~X|eKh z1%7C=wgL&CIt(V4_$tOY;1>#}g$!yj%9DkJtg74WLj*yrZsDfnjawPMv76J24lMk6 zc`}y;0Ht-1LrXbvq;e>ubbesYgl_s107=35W&A+hSQnfr<@t6Kg2=WZJex~#o!V?p zw9(MWJVbI^Q&+wpy?6_$GukL;$0PcpVn#SquyOsO7YVXQnTi=W^DfJvEOPON$U$d zov-_En&ZYtY(QquB~!CDL?6}@KIKf^(fA}i^>CJjb#zHe$maA}oRw6njl-I@cbLVd ze)E^jG&_eaqYo@=V&x6j7HM2XOuU(S_}4|@|szb;^%T|==GmmsYf8SXbe zbC^hYjCBvZPKffN)q(Ut3LPMwo|`52|0{)dF8fzW;)-$^fyJjk!zZOlB@RwecEm6J zV`&^x#4ug)Pcs0oAWYEH<`qS`@sZc|m?=ye1v$s^WVCfOBjWZ;Wjdz_{a%yHhXR6c zWZd}2NNeFcCE=r8QspjXn-e z?@n|OU%4cl;T=R*qM*p9=Xqz#M~} zw9R%3l zDuRhn;h`8&wOc6IwyZ`ySVZZenXGF6tN4iV~+A|H;9MBXr>!ep5*$B%L5 zu;!)5A7?3(sb|8lc1CXE!jG{<=8hb7Cj#1lN7$IR^=XK~Az88;U;xAF+J z;w#9%jd9#q1BV-7;;~jR9uQ1D z6!1XjtsCS2$oBQt zAAvsH)OR(BnTsTAZpG&(i86=7w)$CPxGG(z`pT6xofRcYq{DO@2K|KGZXHGI1>0oB zb8<=-Akt@zZ|vEzACk)VxzG*`s*4ULs)W!boH3YA$r#~gxer~uc1g}eN`y`g4!(<_ zQ}U%$Ok*z{TsE)4HOv+7cNKJTDtG6Yq2jRW1OwLequbbmxR4!5GllTxc+qEhJrEqC zUcLH$ZZ_(4RaOzY>iat#)39<%s#0Ym?VLUTe52u{g&Nz;AzpzDY)vis;#7PAf4RiP zu9=)KcmD?T0JQ(B9m>zjA>aAK%Aqsd0Y$vXcA%rDm+>ml2)dmpV~w4MPU){O&8KlB zU2A~~Ou4Q)+jsdopTb1Q<{DHoR@Ph9MCX5+?FVwA<$Uu>fM_%Dh@(y16N@yC1OPvJ z7UWyrEP$MTo0=U}Yd96dKqoJfQaGK12N4SI`x|N8U+}hWFkK6yYwd|t#z6yswUyc4 zJ(n!t1IIDEWe3vJS5o<_?Y>8Q9a%~D*r1(aCkJv^6IlZ#;SXl7&WAJ8knLRqh6s-{qAu+N%7fd9v$TjyKhk%n{{HpvEPIeucHvVMg~pnK3NBxVV5D zbE@%vLikF|8TeG<0Qj;jOaRwBy$i^lQXD%Cc2DUm)@(a3WK+({eF`*VS)tI`QS%Zw0h9skDQ_9!y(=dBCZ*R9FW~(R5T?dx3T#NBcDXq zn#dKIz1qR_ZAfTgcuf&w(nR1n>}x}jVwLkQD9n5T6C!JHuaEwgoG%5jbmiT3i&uOb zH;Xe)(0iMvdu{aHKx3HN_gv+9@7*_QbGs&();n*0`{>1(tA=7z@3x*c?5Va+Q=jZ= z)@jk4iZ68Eeckrf$u}l0V%w_4iEfG0niS!{zGZ-MA1{teLS387a73MY|D%>Mklu@< z>)A|^#OUUhWr{hRT6S|Ls0V)*d(76ITc~n_@;0|{)r${?%w|`AF5TfOa-JWtW5{gy zfrk4iM(rX*vI5R^4ojYAOgqK@UR$Io#I267!b@maa2SvEYzmjFSq-^)8ocyrsx|O z>SKZIi0c|OH2gHYe$9-68n<6e2%0uq?kuH8(hJHpWSTM(dh6HppDp8L zqB}72hI=C_OR?{M(B(geY8JZ@oSMmd-!}{$6Ia+5Ukjp@gcz!cRF>xx}R1L}<`rs@c+BH{}|-(gvts*L*x58D)e4tvjGAN&3$68boIm63@i4@}RUEG44kJ=w^4~Tv9~a zM7ldD^y}&%SRzktP)Kt8-EKa3U3U@EyLeigy?BfBq|hRwBp=GdD=;Us?rn3r1ji?? zJprAGy#0Q4YC-*)T1fq?00ApBX*(>JYr0>;zL8JXV-(&p5mMa!r{$5HQDu78H@Npr zms3Z}q(XPDcVlnMl$HiUnnyJ`xs#j5UmO5FslG%s=SedkV9I=v$dTJ3M#HfeR#T6 zSVFPvetPgMy)nPX*Z`ewBK_7t8_3Z;k+8UZ`b0WS4^mZD4=}G$FBzB7I)Bk9_l;6N zFGx8(e9*8OYq|`RBC#f&)p1{|b%zYw;@`W}v1S*;5_I(o^1=s-(ONfxm6GU1@!)j$ z=TJ@2eJIdpd999nhAnB&Tri#V*08h`+q)p!R=`r&JkHCys&1i@6XH`n<+oa2F0#j~ zlH_LhO@ATgtslX)jZSlwU+gI#^rqozqQ=#Ko>ypyoNJwKJj1I}$Qz13?n=aL6Mb|e zppeFKXu)XOf+8_rCubYwXm6dV)vr0TRBWVD96V|bB2FajstFfpWrMyJ6eCrm3JY8J zxolJo5y{_{^A{sQv7zqG-l8(PMf%I5B%GoAax?IF4vqIv5f;&5dE28H6Y{aiS#O?Y zuCy|V*#+f&r?Y;OK0}?~0hK-y9^%yM)h448Rn626F2lLlP(5nDSI17!OCr~{v5-;q zvNm)4AyUSguqUpq!P??HC^p&0ws?dregDc%J-os_y6?%vM(8M*D!(Yc=QMIJJ2uno zw!|X=^eC1!cfZYOXh;D;rN1y9D#%Qz%T;ZlMsf})wXEp7P1`+7jAhZXvnp@q-CsmL z)YGE)r>!}29@^Sd{e{Wa3V+;O{~ISpi9c>Vi=m?|TczCYBH;2KD^v~ll3N}9$ z!^{>p+RBCW1LH~612BSm)rxP@`s#CaGb`<*%qS1RtHxoK5w7iP&enAs#WH013afR_ zbJAP+3#joNQ(hB~OLLVwuHO6BU~(rvWNUPW8krq`qyTaxbQ_!)NBR^R{@M!LsBrAS|r zmB{4Oash9PGFavI_-o|Mq7h1-uW3c}dl@cnOebmUxlEohWA$xIxm{HZt%$axM*00R zB+C<$gD5%e#y{;YHuTW$C`)k5*_Sm_3szN3re;RStZ{>i2B8mbO5PNO(L1vdc=etJ znq-y2uz_`^^h60CpJss^`B#xWwEls#ntXYhmj_X$FjZSTyKy$Eb@;qHbxL_zUuSqH zf9u&$iWoMg%Ny$48A%?iJCU-2J%l&zQ~r`EP?9cd3lU0&qoQ(U>Dj}n5;T$~(BLiZ zjiC=FQXCC!FZu2>;I~_ulS&YUN|GROGhR9Ov1mNf2^GawF?Kx5M@62pQwamTBWyT` zRihFz+%cTYfmm63m&H!_K19ZZh58NQkDy6iW}|;-g#J-LEm4jMjY|j`RScEUn>I7t zzwUsTHX7Yu^eYGqsT5_+9y)^%PAkVbhMXx^mf0Sj{yrwjK?)}`HDVCVw!_ejof=u| zM>mKeE`%u|+r~$ocGE=?6PGRF5AKb$y*(oN2{=`~l>anU)1Ph;@WIA(n$z+#4BEn$ z9axfpwx#io`F`@#>Ttsh!%>OxcCc?9zLE7YYCC67M^E=2`=#rsVeDYP7)vE^w>0V} z(HS6PZdbBXpve|Z zNYFkGuTSW%ow=~pp25$NCUd2Jvtqd)_XEDhU zHrW0;x}NSLkyVv#%S?r@JYlBQzOf)u03iagNLe?n+bXq=zVC+dP1RIy&l^7em7anBcBnO{&{=?!|6g8 zo}sSAGRO;O@eyV6J%Q6z6c)}D(dapkJy-bhab!0&+9`z-@h}M+7NC!OA45{j%?b>m zD4rMaJ}qcz25g;JYTlWFJS8=-s5v}c+{{HR5EgMNYzZ^K)qjx>qG`ac3g={bI9Z}l z3SfwH2F1l{@yku-6&$H5b`$9vYcsSox^*GtkO;5Y#x^^K=1nEYH=M+nnWm`ySQ0 zK2wS+9X>#Dn{eKo*Xe2KSH_KO+YCKShW0ub<+C(=xq83ZZsQev=#t9*mZQks{V+AZ z-?sBCMZ?XcF%OU}ejfSQF>Seiyct>_u8eFf*ixL{)9FwMDE@&5+;X;`(!F)#O}iB} z?mcPv`U;#s3;r1@Kb;hm+2#h#R!Xrd2d+LP$92)l;nVoU&J;*Xe6*4a>RvJ_Ce-8$ zM1B0)+`ycnA%11^WVSY^Zu13fS_ge89DO2<5X}O%C-9o_enV%W!M3)-K}CPUfb-QZ|qeK8A|U{2s1gtDA6ov zoyB&S`nM99{r;DdABj*bHR=`k$kFPObo!@PD#L2@k!3&&P*5|j*mZnj)Yifm=KdBb zj;t=|Lxq>HJ{677WT5!x89DF!!>{0u0B#Voy#V#nWO^ucL(u^FJbS$oT1x-uOGoYr zO6cZ5t(d5n`9U?z6hj0_IJ-7RIK}VdT&R%Cv1~l0)IBUV6G8 zMInDn+R?_Sx@^jAZi7HI&aNwT<%RZZ;S?vng_oJU75e~cnOtiu-c76ad@O%4>n?JS zKwb!c6N}x@jVM>U1})#b#66hgPO4LSE7}(|LiXiQ)8RVWR_O|eo8MxF@Sm?IN4%~@ z>y!vptd>e93aXGTSF;gxlRc#G8hkMhN=Uj>67LL5#=HuF^0gO!VYiy-9 zJN}u$1|!`VsPi0Z2~3x8XRoei(bJ0t2z^ zDWBm2E=Cw>ha`Lf!hkZjthc46t`V5WEWhm9Uu=YXInK#CdWuk!Sd|=_x7>8u%TM7f zJ7KZfYa{Unm7b`&41PYN(mPx0iG%kAFUeZT&b1!3=EcC2JWfYBrJXCw}{i-5g?5=W&SgxX4ZLJ z!t=Vt& z^Zh$$WcUjeHcNPtk(qK?vv>^1u~OY4Q``sCZ=|j^>LIGqb1G;dEndqxau++;99~J_|iC zzivW0A8N=skGHT8+)!}ws(Q^y{i?n2xZ?r+pdaXN=|5Sap9Ik@A>Z-njvvxBm}%$4 zAih31JVLdyO-pK67yx5AmEmDhQNOI!Hr+V)BA(c`RpQnT4v&2IYB) z3-tuhx47`4gE>6E`so|9k$kS#MxaO(8`C8=NBN7Fb>edRlL89!Z>oSNeg6>^AN=X- zc`U$Krf%aHK6cPe4DsX%p3na2%Uaq~hOtYAsTU9aMH%foseGB;Wm%uKFxi2N1XPFj z!`-klLkhmPzkcjbnnzwAmKttBbY><(+1*!=ixpc3KdB){aRG1XbJf8g0BN(G-&~KD zt494ru>D-hLfT)J@=_m$&;7+|>&)AB7|n{J_qORCNqGPtC!p|M&TPXUn4a4-3E0N5 zxTL*ZvXIXzk+Ir)u995}AkKI?3~cghgjxQ%Q^0?1;QD2Vl;yZ*BL!=xemP%12P5|> z`j>^dg2qILTyUrQ2NwL_pH~h|tR5q#95Zdhh;P>G``Z?x;s18^s16<0yi;p$vvgk-r6C6e$*y-CM3w^Aof6DM7~4a zp5Hy3#BWo(7U8*xg*xAW5-}ZZF=eMz~ zbb3kj07v@3BtIJSA6wmU1P#7=%un$vCx;diMCQ(|B){_Zo5Ik{Xr(_t=LFx$%$Q*q z&L*KaJ=R?g0*fHnROr8(qlV7kU}<3aYep|Vnf$7*NC|G`*{sIPfNXF{h>pcf2wr_y z?hRAT!Idkt7u9L&!=5H_Ii2}eyS-8|c0(-?2zlmKdg2ifB(yzo@2=zLO7dGm3HObU zU&fnbS*?9_Qd(qI9IM5m!-60**CVj3F3BltMej8Xsi z8;77Ifb}iG)yh1az4Y%N`IAmzo`s$ZdLN^?D0|B)SgXch(X|W){s!U4%AC~a54rK% zfj>M4zO}T64F2&l$Z4u8Chj{#>1!$;+q;)2dAuh561I_icEv6E4*!8|lEpIe}5`hGPcfYgJ#DCiMt$K{|2ed8mPFz`jQK{Le z?>kNd!$BuA`cRVG%-wy$+ECjQl;A6N&o3RgT8sbQoH2uiyue?{>CS&!^#?SV`H7dYMsY~+d)2gA4U@#^#ITT{cer$HP?hg} z;-|=ljNVQ}=6|2Ndu6!(HzstMZ+S8cN;Kr1Sph~;wEU0a5*GA|89wYBJSgMDEE!g2MdH-RzX)AqW!i#$|g$2AB9ZDHCn4=UbwiWRI;$zD4yn2sWC_l3|5qPGY z08_n@pI-(4TrB&s+Ux&qKqNfoT7agD-Ba;-D3FSuFu}xweNU{j&RVe-FtPLgw_B++ zfa%kl%I14?RMjM3pt1q{8>_fff6-k^j(WTQ+zy6*i~ zC=Aa2_}srb^CO4s%sW^EX64xb5Xvv#ltZ2P^VT>Cj2w#GdFTH7{C`g7CsN`^NAMgu z27UEqe0+R);%i5|xMK$@Nph(|q@^>MP1=0}FRP-WqPr5t!+!M6%!C5(Z~LRF_!pA0 zo`t$B4q!fizH(DkTH4xWVRAApc-9Pf)YAFGv173><$Qehw^nD^1IYSiPP{7nLWUKy z0m@j@CiO(mW8dRLnA3k7=`Gg7P}|QQ1-u*s@P?Wqs|BZ990U0^3s#LMG^M>Lb){w! z$*gY`8n2m6lP)t|9(4I6qLR(fF0w&V7j^0H?F|=E0kK0mBKWn3YJkUpB$X&ARPV|K zf{YVMNPz2w8dSBlwe3{x0m*GEqJ%xFwCnNtUoA_{@xtLX2!xKyO);dlgti3ZlSzxI zj;C7;q`i0zot(WbjCZn(AVy%zjYKcrbm)#22@SSgk+=z80g`*jgVzIDL+!26aZw}p zp18Vh)=;xM(dpDs0xw=3r=|veeQtifxt5w*C$c-w_$$lU`)!H5dv|!Aja~WG3Vd-t zbHcz-=1S#9!y9{Gagqtt9^=ky>%$(?3{ewX!=YR0*kO;2uPT_j(0K!7r8Vl^QKIum zU!NU^`~s!k(s8o~C{?+;kYPfl^CDECl<8;+Q&H7kOmy_S_;Dns1|9yCSq9y2Km1o( z>#08SHR2xiBef1*r!R-lwRtWE$93C`|S&~mr9CTKLv<=h%LdT z&;;!Let_i?PQ1Py-(Zn9g2wpRncJCjJ@#!ELPy^Ts<#2ZJ2I#1;_PO<8mywrhhILu%>n9DmiD2j)j;|yEz`DVU|cSXvj zf#n_>L>4X#ENxX5kEg+oz$50Mi;@mpb8$*a<1_1g#yBr01Q& zch=4+8zA3zR*r<=_QX*LM)dI0huSU=_MwfOiov}<_7V~jz- z6Y_bP?(gOVza}_7qh4VRe}-jxOf`9tA?f3BkKIjs)ie;+vRJf@qu6kOzPAA+&D)X& z?_#}A=GV(oN!37l)0B;a7W@UaO~6C$NClKy)WMs9&uE0pqQbljDYc&MeY@NBxpMN> z2sN9l9Ni;EgR6J+QKy-mlO>gg z8LU}vh)xndyn((PmYCx~OtQEd>UJRA40XA=esd>VW~|x19qtdTQtsxE#xqXbl>}iRg#O~%hFs? zN^hPo*^S`m?*aQD5w<@#P?wG%8a72&B;)TG?tgD?R(@VV zgxR$7v3VPo%4n}vD8K%4_csHxRys5=rdgn9!u?S^Uodh>Ui)^@Ft6sJFc{I)tBGZ|%H$l1ysiBmdC@Cc52!?M5@@qbS^Yz<=W7jK z`)JvDY}Mx$e!KzxfKkn`*W2n^I2^?k7=#ziTxQ~s1l!_c?PbdI;$YynqR-(vsqb<) z7}if@5k64q7rUjE?8L@7-k*-x)^A(o3q3;pPv!`Sd<%JgT1qp9@7TgijyxhukD2k# zY|V)GXgD7OQgTYN5AeS(cGz|7fXqQ$W%&*d+=~rtOZ3F%bXSf2162&v98qXC z`mPkwL4=`kuL_Z=suU8De|Di{G(}VZJY^ZCUHB@}Tqy}f8al}w?sx69*|SiM7hUcP42dF>lk$Lcro5sJyB}ztYsj}jP+#j4Luqtns}2rGZ&5(kBHG9}A-GV=B(PAuQG%=+Ax^BWdH^Nwj@ z5#A)DH9>gH8qsqX?(I|IJ|WqSc0$H9ZFWOOG0iafdrR{<5kX&Nf^Iz*NghFCAH-jK zp};uj7wIz4>fb@*Q-S?@UgQ4dnF5K+H)p(yG1rD57Y0RYI-bk{i^1o?lqR`&yq$mH zX*P4six0m8GT)1b+|d~zb(3k@`U!gP?h&UUMt^0Ojk7v_HOY@zyT-8qRpDaiYE@CZ z|G{LIohA69?N%xtx)5!t)H?lD zm87k;9pi;c&}Z7gWFpjiSd(T@9g|J z5V`|MD;Y*w@_P^AUjW!V@@}Pao^{Y-0bE$I_Ok0;wcAgYV+A;ncfNMaznksH(1B$` z&WYayWudeA`N>m^?=zA37rHE;np}{81>ZX!yYit5AWN)Us&$~sI8VyUiqyUHYT z^abXO*Q@)G&#JUQGEkl!!%i2HFG&34PnFHw0^wV?AZEU9MdUi&5Livc1Wrs$EXQIm zw?uL$UufX(y1<-YOTlG&9o`w~zWtP$UJM6UkPZnP_uDA#)#(m_De`gZiwPr)BAgjS zTr%%`0W?M`DbOi~qS^$K0W4)kC>DnGGBfVN6y4lD8TJ}r0qEO+-T+03nOwA#oPDzh z#sLIhV2{maMe}lnu=4M4LGH_iL-sROA6b!g@2GhyznSU`qjqcfNS;e|b}zuHL>cPq z7N2=lH+t-{9lyb%mLwOS1B%x&y*C@$;Jd~yG~hKiRO%>PwyXfx;9L>mmzI=-iF3OZ z@(GbMdv|)Lfp(NzA-CpzunSl;<@^YMF^70PkkXGA#VR_y_X{%9aWrswd#;N@c>RS? zPDNV1_~_`*&1Cx8<{>9ESfsNt#cABtI>HqQ-?FivdV#B zRC76S2g*bM$%XYeRw*YWmRp!(Zs24`uFU~TfpvNZ*$n-{H6kvryp=UU0^WR90;98J zRy$Vr);iSRL}%aiuzG44xF%U$GdVYRcd8_uJ@SdbD0> z@a|$ululeFEm0Br!;$a0Z(Ck)@Y#JW-3^_>OBCCVFS2_9<=22)^Ks^4TtL_pGtPnj zr%ev@dqt{qM>+}W8i9Zs1a>X5r=Jb)P7WY8VWIgxV;~oRp-D&W1{kaaLl4;fNmiX5eeu52; zc9|?_lkIXq83VK}`U_)CaeB|hBM?t0fZjsK2I_OS>sn`^))5(QY=d*zY)7F5QlP@J z4uYosu^1LeB51Ix9HNX8#`Y@@hmTrsfqRUKveXm&3CoiQW=L*-uFlW((%{4ee()bv zuOj3z;AK~RD%rF5rE0B3${?5TQKtW>kjWY@cZsvmPAq|eP4(8Te3m{lwF2!nXbh0m zXwjROa`)_>a4G|6EAG2PbK5-bbE081bBl+@3@afcEx|+gmPJfOy@$U5ZM%Gxg0-t%Ag!kPyeArXiO$y^hrfZvL zgCC^4T!vxY1o=7H#Kw?+I9<(GxG5El>z@I z1bCUL3?lmUb|@~Duh~}=VR2(>!F^#*)df}dGE(=dB5l-7y{U9sg)3eLus`omaQ0Ql zU-R=X9OmwzIEUL#fqN1I243 zh1Q{zX1+|;_*nn>{-4W)5$Gi^wzPe;AU`GBcn-!GOsmp9zl&6mw zw@KLX(8scf~-p?XqFt^H-+nB>r=q0#*yDD6=k0oby8H!axR##hW8}QWCm2?=dv=#yGvQS1Ji9hDcWfth zJ*vpkJa#0uuV}Adm6jYg3l-pdBb@b)5C4dPC8(Mr&=U0y#adf#8^J7$J@RPFr;)NDha8faiNa`+d39`GJC93crCm z5J4_Fe^$%j8du5p@|a@`e_oVc;g1IAC=Z_%Mt_V%7YzWaAv?V?JOY_P zD!w3Kh2gE0^Lz=eD}B6*=%vaJ&5+FYO5{Um+zkncgGfwxrj79Lj*G&k!t zSZ+HRFTArtq!eYG05z~{>Fn1k!V79ar#XHpk^NL?9)s&R=LZ zto*}+h#+3(oxI#!btE?(>;rH2sY<2ENBdVZz@BO!!%0fsys`o^8M^r%bB*i$OFr@z z2B(8=2Sfc-qvVjOD%i-ZdT_{Q)Tz=xq@=8#(5GH8ETFQxE-5eY$R>p*7;m+OqU1e^ z+6=;fT5b+5tEMoIhHfL_(aokp1lY23MR>6Jv&}K4UUUE~Er1YeQn58%aF2af%-hh9K5 z*n%#1YbcwSev%Jv>z0-`!$S47UpbSyIUWhTZv}n!b&JjGIE(M>{P1jS{@F>)bll5a z)0!+25*_|%udgq9M4|(P$X;DrTiX=YW$Luh6F4d$D%f^P)!=G(&=s#mMj>$8wXf-5 zI6mv~8{rSa;MleyEZ-MzF?W3PNkNpNLIp_{uz$N^^*;BXrF2!|W7`E0kZO><& zJnEC=-ZUTE`;hw=vism9_?mM4WZ`j7f(s!uW6-$etM`-iUcG=wIlH*<1UFD5@fFHT z=FF-7f4PwJFJ2s?_X~I^KNV9B8<2tkdOZcaNYN>o#RLWhva+(GnFDGOG^iW13)|c7 zsbz}4lsWww^!aj}|C1vIG97;?LXm!UD0x@&&bRAaSMZi<;FAJRgRDlJ=DO3boq98n z$Qb(gx4`*@T;-vm|2+s;CHslqbH1Qs9BAwY5Isx45d{SWTDrl%|IhzbxO|7c67PnJ zzJ^(c&Cu4}xZ(?3VjZomERxOy0E99EY`eo$^HfVzgpB=oJ%M0#RPxd9QW5}Jjr~55 zO~qpZzg^cq|@~dm)_lGKW2hq_IXM-S zD!BUPhfaU%^IwDXfwzBUmHt}1Bt<&TH^5lVM}Jvh-(OobA~SjsJvQg^Rb@%@Zv|I_ zqDJ`RV$S?xg>pN8C5teSQ@3=at%(P9<39kQu;(7C!*J`n%0|n;}jAxGS`i$=FYU?Iy*?Mm~YAYkeRQ8-( zIL%f7Dfx-%4WD&t>iM3eeWYj0=_OC^xn!QN*Je(geZ8Rf<8-ghFsyL~>)TG;@ZJm| zu2*}Kv)@OxMiy1A5zh%TT>W)NI#&)8QuWT?Lw~AmtsR0l{uHmYhXkpU)@YvN zLx#O5y(r(V%@K%gh6rxQA{5MP52fzaUH6ZCY5^K&YpSlT1`?MrktK-WGy(bHd z_BM$4Ep3_2P-o*9187pbUbori53NXQttX68=_*uBoLx+zNOtn1k<8ZI5-0WP?m^J- z2~RI}w`us-89?x*i%>y1RSY-kO<`c&GuU$3P3qf7cJFy?zzK7CB>(j~{l|BOe?8JD z<-cVkJ&u7H59#QBrg&Z+q<9q0siSjj6_m%IgFTqU%B8X$1H;oGbt!BS^#C+93I_SYcI>=L75|gY!8OYxPyIptUI}|nrrBBx!Q~;iT?p!VxQfn17 zsm>+26NVYl)~wZf)kbDQCgSwRHPn{YDz(q4cMR6#`zvI#S=*&1)N2>3hc7-2GyPER z?3V@++MiC;mXQ<4#=aaZh|0BPlSF0u`XJcm=H7sZPtY%>jAWh69s}xtfcBjs300r; zN>a+Jd?(=AQX3WHZV^+FU3EiZ4Vt~U8}b}(i9jg`@nMQNTd1bzf!q0?lx^U{-$r{4 z3UT=X=v(LzfH1{dZ3b$hi;2PkY;;1bQ5mnPJqla~;SGFiQ?a`~Q~jRsW^HVAzsImfzA3eD-)_e1O`;$nd)Gdh%7Ai&s>Ppz$zSGN02w6~Dd~Ywk z0uAn-f_Vk|-eyvg8+pQaEYEc`;15F*-@&?qyH=Hb>#4&d+pkSWeO7L@ zr9QX5R=wHI=x<^~{-J2uo`3+UK^=osqaA$KgVY^1xHc+4fwv94JC;4k6G?6d94NBkolpL<{Dvyj1Qv?sRFH2BMa3m zV%*c-2C?QYJR-U^tTr2r6FT_yigRSGXR)uoJ^!`>*8DaN9_N%+;r>m?3iUg*v0Fe& zENh?)(qJzd6&?Kr!xTA$hI_+XAaCcRo;@rAYQY*DZM_kg4E7=DmzY_okhD{sw?8hW z#HDHzqm>qqD$35E=teD5-_yruQ9@bxDL)wz>9(R(q~ z)tgl;^y1jHNTFh zzWM6U)pHIwcxb32O4$3Fx4aJWKD-_c9l}hPS5?8__{TmJuB5tZA3)1$9-3+JOcX^Q zvG)ZqBGpTn@hFsxL0i0(`+UXI-Xo%6n9-PwLTicKS7q#+NH=Cl!wW6axvqv&u7;H> zHBmMz)yofSd+xLxi%iDsSA}~unr45H`t`=w{0UV7T^j(j`I%k-biqmdm%8Gzm2rAiZDV?6ox-8;w2}drzvJyy>ERR*a1(JPw>@fd`X6 z#&{RMJR^tRR`K!c%SRT=?3lQ>gUm#4;q}X?JCQxtGSWVY9z$vf_DbYD0GJ2{(6*@A zL*|DIhE=b{D)XnEE|5m@iZt+i%>>L;oHnGo6oRLz!F!Nn*#zsVC=-CvGVUIJP-J`&Qk+PpVCe$P#Zo1U9ceQ~=-PM)gi%uiHy=)?-y4q@EMX=r(w z$6JMS3#r^W2gxAj!J;g<<$4IkiD<1gx)MlgXa^>-Y~yH{Qo46s@$9X3e zM4BQsKKX~=9(U~EU--BLohTd0k5GMH0h}4|%mjAe5uNYLlCPH2Y4N4y;p24dKt|5k zXQd~Od1MQT*%=fOLF90!kB0}|LqVK`;*f50!|YrPg^QVBp9e>P{RI$5ePCRsKzTI69UIor zFh})V&yv~bV&)#~^dD~@?;})}FNn@v5D@pCZHiL$8oXq1@6no}&v>Up2S2jS1;rot zbKV5z_bCUFe__ z(z!AI)yTD6EB9p(9)9KE9+@(#0lcRN5xlBXaGvGq>k3&=_pe94>$h~z9g%7-7(xzJ zR0yttwg8gEws$ZTKqygOG5f%0Q1m3f2cue%cZsw~vhfSj7Tjo0W9anA}DxV%?&X!ui-eWK?DE-F!8$Y|KaL7BqEQM&k z5$uw^vm0bTEHls}d#UL>pmh|zB96nTwF|^pbS=#H+uFU!EhMR>L&WB3oRGV%4b`3K-LsAxP?d1L|fhXs%G@gMMv?*=ci@mN;d`{P{ZfAwXMIhJncf@mG%Kct!4w< zsquIA+*=8C7(w(RR}sg;LPkxUdSz_%UFuh*fW1xvAg<4|{bQ%(cN+Fwi4O;}3NWgA z58Qk`hN>**7H`dD=w^TTzD%?~_Di5?4*r=h2!}<869`0u!6Ez8`Ui;}?rYcVP=4HR zVF`L8fAlzPZHM*SK5uZig?A@6Y15vz&J|umF-Dh918a{zahGENB z6^Ug$J3RKGpVg}%7ESyfyn3mLu|~_VcIn(*SxRq=^cWtXHlKHe&6L$}Fyi}~8~ z-0zgUf7>8|M}G@CxX;mpzm(#D%34Q%|C9R2r%hu}`04a$tBYXwq!<>aySN`6%ANFMsYzTD9ETo?=qaIkBD4Clhdk=7Vch9&OP$ z6Tx9xKB&L&gj4kSE<(1x?-j>XW0MYZ|347#0hPn^r!29du?*1KU7mr`W3c+qo>AZ} zbyiA&%KQF`{Lsgo#j6LU_vGU!nkvrBA^bt@ef2X~Ah)b~Pbm(FB8xV@(&3kb)mr5WL4PDsP|ycf*4LIK^Fcmg`9*z|b2KgH&98E@~BL!dP2DUQ!`0<3;@4yO-^pfF$p zbER<$P@p34?+HK#uWpAWk;i_ld9Dva2B%6!KVjSg47-9D3u7^Hph>HqQV7ZD@%>>} zAZm{a9h*h2+ucsmxQ|DaU;?6ax(;H`3J>75v+WV2*mHUDcekEuS9b$<8vx&AR=2<6 zXjJq)#^srpy0+aVqCVJ{zSU)TR$Jg*@2;ny2KyTs8@U9yO^EnDE_;?ZWUq$PCgR z6D0HW`L^5mQG4c!1xnP0o--L%+LI#ydHJn>XtnuIp51gK;L;Om z`;%c|r<~lrj(CL@tb?y!8P^}?EIXbKF$(nOcW87WdG?0f%}|Tb6aLBMmGuo?zyA&WM<8$SB>14-Y#=H zcf*P%{2^$7pvaBr23@DqM2NGI^Di4LeA_IHG?&nDuxCitg1K9x5+nzi&?HQVch1;s z+k)eUw3RlF`}IAKp^_T!1fHkKxen8D+BgWM0rEJhnOat&ug5ka591~QC;2lHT&0q^ zs}vM$`0Sfp0MZ#~{{d9W!doL7ym!M!4LpB%gc|ZH_zIGd?0a6@4)sG5LARpFu&RTw z;X?azMfYnLO=!o~Hy_q!Lkx-2rGL?Gmw4J_HSFlGKGf~dhvpz*i|j1n0L6@aPT)k(eaHAM+fB^3DJP05z zM80CW;w%ME@Yzes-QVCgu0G2fhC~P+EdsEb=RqSy8^fyFIDGZc7vNnFWkid5B9@sZ zfCzmy0=(Akt*bW6k43@4Pk?O<*HBH{6DBBAAyGbQiB8r79r3d~;$o&Pu2E*Fu__|x zAC!d-V#zv+^c|d(3Jpr~f-Xbj8uGYoD_vC`2LP=owRdm-3NWCPUmrmpIJ-+<( z%8*dsdbE~??FKU0DE{p96sqoP{4R9BTm7Z6X-LmfYvi@i0n$^dgXOR@X|uEV9(ax$r{it| z{Bdy>XJ1?E5YWECo+p7}2}n^*0S{-YSptAs9JXGw?Uq=+hikMrwpZHj(*;X~gfxF{ zr0)&CWCehY)H^81aipH*r(lbpd(syMp$4jg;89}xpv_#GoCSFqOj2SQ#>ocsTzg1G z2;5Dk_W9T;?HNQDAsxvZG38w_ki@K)*5ny8MO!?onmbp_$XgIq8&GYG_ie9gyL$;r z4*3w4auA_4+$K)4zT{-lL9t>7@RCzQhEl`I;aWpP^fP^crZ`Kc6$odY-U2H(m8=oJ zP`zZ_V9zSbY5Sy9SZ73leCa;#2T%PG@L90g-E58p@aEcDrN%}dl<2`u1_pmzA+1Kb z9Ep??ou8k6pKC`$m`;14$i%IfE?N{}73*x+U}sxfIW0swf~7};f~Gy$;tN_eG02ad?Q9dTheR&$m&!PVnfMuLGwC~e-bGkKSwEf9%EWYuxp32Sz zbI!P@=IU(zr+yC_?#OMm70^at^b6HQNr-0EZYzgtQmsL~=7yT=vo+10Nmsp$Vx<@# zBBc?5CmA0W`JCN6CS%*oase7Kyd3Sr5Kdn`rA5-vJZw;idF1MeT`ev-CWfHeO7mgR^_<8(ge_ zzcuaU^esg{bLY3u5w12ZA3s&{Lq)mf95YJ%T`b-FXW)NCf{M2TH!B)!{Z^-P(PUh} zWi;s41;dHz-f4FJB+4ulKFNfr^jUmSeVg0%f_E{H%ea41%cI7^A~!ge>`#SD38wsK~#c9urU@IG!V^>i;%?KfK zSI7amE!Dg%4p&EFRCa&GvKNI!Aq4NviA3t6<6PfsZhem<^wXEop4tsVDPB#cLN2Gz z{!^df2j*lCkxw3Hdq=Ps!?P7bXPqLXT2Q8r2S=RM)4I-psWy6`VJjEZw%&(wP1Sj>+^c$5YiCE2#W~;RY;jzQ z4|l*vx&?CX_Qrg5ZeRy8;O78VibFX=PP+D@q{M3~jz7ZW_x0bj6sCAtrZI}^cgl0m z)yT4&ws@;*Of`FQ;@0!4o386z6RYC6m+F;FUgzequo2XYw@n$WbC63F;MET3tM$}g zXxH>~w>b{JwrnJhH?=9A$uMgDcYgKq*T~WPpfNt&d4&kJb`=&#+4vubZe@?{42n0V_QJ@05cy%W;eDYuc-N5Jg z66at`@b%kJm+BTU)`DE|U}ERnENeM7?$N_O4!eK54_@4FD$g%7GVL~|6P-QkdTBSU|EA>x{`C`Bds?aZ|6asD^H2VyzllsB zgf#)vO(%p@0QKQ`|7_~FrWzn$O$%kXZyaecFf!}R1*Kl46{CUpS5t;f_Q{Q zS$>3-Ay^|&LI0U2)++k5jB6;J%fuX5)2j=Q*yNCG4c(6KG0NI0- zc;$UR0Ja`9$td3DRLQw!FglLU;r-{Yy(RWnpCpLD$;ZCwn+1(au5RwLTbL2yQW%^3 z-z|tWOg+!gsfk8s7@(Y;CjJ_K~vC1fx zMFys%|+@(xu|%xBhkDuv3L~cs&F3=<$E^`)C!w zUGB-G3~EQ4TdYHTY@B24Xq;yv|AoIcNtPq{4>t{-FApY}oVF&BuW$4`A-N_O^&^C3sC;GYJ=1!sQkJQ1Y3>Z7{)j_jM-~NjQ^=S>| zup-|Sqb537tWOpilLMPtJNDGvB)U_g#r0oU>0JzWV)w zOm7#M`}l(Hbc7wn)RzOC3r6I3ivRN=_WQg;afIC48~Hb2H0?Vm2?}8V^?Nu{L+WRg zudFIF?Em)@2C$gM?Y{2KNq}c-{u}^nWa(6`{WxojJsio_e@asFj?MG@cW$GRtt^3W zhwJR@of}1$q^DIw2XYN3OHO2thIoh1{TGG`bB4@)OrotFJeb*( zD#`v!Gg;EB*Q3m~J2{w5Y5)?X%-=YQm+t)+7Ao-nAsVsTTvC)hdP+~F*pg|(?5-X- za+1+H;&K@b{@o|9N_1r)5(4f(yA;QxCJU0VeRP8H^h zH5mB$D-dH&)rtb=7nUxf$~tqe1`LKQPv&kzBEqh9bN`!Td36<};7Pu`N=`})(4^Uf zcsYbP`31Np$AsNlCNE6{&4(gJBwg^27@X>Za(Lcl zpf`H3!E$&xa%DF?b)iIdb?7_uXD2}<^(oQ?V%q@wys`)Qzv{}G>NDHR9G}{p=Nu@O z5i)`x$wGsqwEW9f#xy)xxx$aVj7rhxd~_ce!>%T1Q}+0uu6BY}Ts zDkMT_4;+f_f4c1U9Y_#Tzg&o){VS$;AzvNU+mZn(`|x@#bMIBmKa)wnG8m>4Jmcp~ z^3%4Lr?AV?Zr`~pVX8k-{TnEj&!&Ahk4t?ohSp;S8)~0(t$qI?h4&8pJF6`cdb#Xc ziR(6fcy;$mSozA}sMM!?OQ~n$^l;Yo(joaT&bEIaN=(gO?2@bZ8O6~>wok;nD~Hdj z8>bxvbIiXE6DTC^+h;;@|4RCzGXI#h(wDq%pih4YcuM4U@yAxm$wv4#i z$<2VAi==A({k{swDfZEjvr*74`+Y;tDIR)P$JX*^KAWTY!*b7q>ju-07Ar0nYZ;aW!ZjqjP?8#?{wY7glQG5*Qlpn}3*(!;HM?I2k0 zUX0AlxRk9&gsPfi3t*N> zgSm+u^1l=C4VfE5qsc?n<&tDS2BJBg4Irve^iEPuFMH}@X zKois3O#G}b=gQsiS?uljy*8>V1SLfCGlkc-D7qAE8yIlJ#Ku1Kz5s@p@ju|Kfxyoy zeX#AUt-I8!E)$KaA;2(Pj@gi`;h=3}fq3>#1?It=6&WSczYSMH>j${!T82u&T@}%e1!R^u||U^IL~~+`V;;zZ5}%t8t(5 zJpGU*2;^V#rIN;_fV&LKu94;S!B?5w9tSY%mKk!dTpUepVPj)r4%hS1+Ei|EReyrf z!dHVPn|daLW!ttxBO(Ar2!2nxj_mCZqgQemS(%wF*f9!ESEDxo(gD`t_Dmz9F|bTS zk5cauQzZVO_t~Mgc|5oBWArR|-X?aRX>c9g0_bRK3f8sa6wOd^$CCY_pRN`f}SH?MUyr5n-cw=iT_CWy! zq{m4`nx(1o^#|zYyGyv0R5)q&#sx!l8s`^wWy_&>)o1ZX$2)VYFR~Z$PE`bp1!vX8 z864D=+HrLQQqAA5+qGs z{NNAL(+dj~+ywawjzNfC0_|;$d8HJgiQe9Obh)sR$LaKuVBxYJCaPo4b==lS7t-gGbh%hshh5+!iowDap`Of0IO@frtNE4D;#`XW?GK^1pImnVEDqC5 ze;jP=3ai;^eyxM5C~%k>L%PkU=n4%JXcltALe8Mq^Y3Uzupsck#uvSi{Wp5`t*i_e zIUX;>Cn9XnYs=7b90y z>^9+0)Ag8=I!mBbgW+nPJ0vVDOu8!)b~2!m{(;c|-rBGCu1qi{io-rf0105xy)S#_ z(332v-K@f;bW;yJQllOvDm%XT!3O%R(0=H&DCp)+B3HlK=eWZtD^MM&a01Xw5_|w- z>TsLO_=_Vm5Ciymi9j}l{AU{Ypc5Xh)b&Xh{)P&+>K1Iy51jPqb(ma{Vp`VnWBax& z_2H~HsO8aprJQw<-QV=q5?s&SX(DRcO?2wCeDPgrGE>Xr-g`e=tU;$%SOdrGG|T-p zrjf-26iC+YDmsEc0ehL>)bjM>9`PYiiJEKxlO262cyMf4aZ%bczPk9RpTWbH@67rM z_58s5H%f;p?b(D^{dZR3;8@WsN%6!j5S&}1N^h?iq(CgZM$z`Uzv5|$0DH|l z&FI#-5yQa@#zBoa5WWJH+1V>OMB1Ori%no}b`L2}hnR!N%uC{PeeS3t#3eNZeMPWKI%)wpPf-~R~NW5h>O z#T}$CeSZszI|YBlH~*zvW2{J?S{5@Z$36}p2AHv1_YzK-BrS5*Mducf;J@;a5Fm0* zd=3g*O}*U{3grYgz>3J>jwHtJpXACcUcgVfB2|pKesRT=q3Qrvrqb2gv4zPl$w0U5 ziSS-@yOz7^i`-Utd{}3)G@%s!gm+&F!bXA(1Fy30C%2j;>6%*&ZyxOFAAqV)4pM|< zWQa_-0n~W>>Uv^`>N7Ts^mc~RY*i%(&0NB(iG}atgI;pkk2gWJq?1f($-5tM8$YW6 zeO2<%&?jeZ)Tk%_j<yl#l+Moo#nwv)kWXV(WzH~=NKn{cK^#eiya|x)rr*JvN7?y1KGC-e}xBEe#hC-o; z?M3!n4K8-ebmg(p6`-<;RNc&dg{yVN&&PNEV^-?Z?NOFCiV(RqEE^I=C};QM?MOi5 zrIHlmb>gH) z5O@v#zsvkNy9D6U09Xoi5N!0;0MuUcda2Fr2~;{lLmrps?wK!(DPk&JxwyF}B8kyn zhD}9Z2A8Sy-cX~jc|F+O9ln-)sZJXi58tRnbqcek83=`{ulY>4aYD=P7IkC_FinG~ zHH*q?PU{)gZDfg_jl(b9tZZd3FV2oK6ZN#U3*7rgM&ds7D=yTa>tRKQ4~^3PyV?O9 zF$i!(MZ@fpz@OctfAByAAnABr?KH3dd$z@vXg9iWmMNum26PZy5i;^5!u*lc)*`>f zeeO8Us&cjHLy!a3iops3pqrf?;9z3DoJgWVJ4TO2=Av!4x73g|`8eDU3a}dWr`$zu@8UN5meeGHs3V z?hjb<%THL+G5thVdnZN-n35UY89yPixL^6Y^Mz$t7FnK@v9WOjLuzwm@r=Z{;lY(D$5UkE&TZZ}iw|rQI#B{||2v;Y0}4p4yFv=) ze-Dym+D^Y`k?w*wbBsm0(bWbb6qoEEC9Eoe{S$c9)CMEu7p-1qRSG$6Uo{~LwB;_; zSC1+*AR`|`gXi~a+u3O?8y`7CyuH0=r*XdbRmhj@cz*%I<*X!Ba6f+hxRV#CTPz(TS z3W`imKK&)P0r0=2G> z1PukydeDWZMgi?$%pKF|v4`5!Oj?7xyuw8(h*eRh$-RRYQw=LPr96JyB9$xF_I zl?_{%j7V&w$!a#}VqU(6G=OOFcts1ys2(XO$k8PPs>jZDX$xCV18iQ0&>_lX=&d!- zoZa|6;qFERLL=KMat=t*5&~+D!zhmn^FW;f{QM-@4=>dPds80Hds-h9R=RX_bO0Rb z^gd74DK`i&?jIl(pkX%+T0@Dk1piXux`vhA{@<|jL$*kmnz1^>iS;cQGSj8vtu7`e z7A#j@T@C3fe@cLA(4~_3B8`DoPJoLm#$LuKCjQPRDK=-?@fUl=@@&*8$l+fOD$=*_ zC|bw{Z%!nX)J6**mVv$Z!i6=RG$2ev2hQBD2$C$>N(^$KE1^e*H9H9SlG?wKEu>rT zZa||go_qbA`NEt7>Bu;9c`*VLb0JqEsc(Us{&<|0-$S?+bwQ*ov*9!q2$i@HM}Z@^ z5DpZNe>aHyHLjtSL54BeIr{5j9H7fKPFQ4+)xiliW*h;H0f=Jf5p|k+@yz!Wapc`x z2gs?icnA<pt=>~hH!A)4?Oj&nG7T}&KTA7+1FZ{IU{#UP`-h^Ny=vv= zqYKtcK~ux_|6hM3$PrR`Y(b9P4L4i2J=*Z?LPus2qns;dQ7ef(x@huUJBhh+H=ioK zGbqOQXY1i9p}*3h@7NW-*h=QnrKuy9|73~4@7JAR;{kX?Gb#If+1pPiCUUUn0RDAO zulc`YIbIy_E?YI<*6{vr4p>)I0*PjJDz$IN-Io=W>SWCTsHmRS^Y8cjgdKR|NEo)a z<8O%j$EQZw7vKA<$)>JsYB&Q-@lee$p??ztQ(J+wL869lzw4)NC*WlL6R2efSyZwU zoj{TP3CJpdffB)pz5hVqA_2H7#LIa(;itqVpvMn<>c9@7RC~S&rtrz`xfmkPg~WPJ ze||i9avrdssULmXovLId!#gX%>!Vx%gHDtKL7%Jk_na@~9zR1?4QxLBO`T z*;Utz<=F&XS)4^Q?<|vWa8D-mFBNc`UoN8<#HZB7y_F<)_dXp_BG5mt|8OR-(`~hf zlS?PXS~O~^X=MDlh_2=C+WwqvPlafbuN9+9Va`p$So?}6sm z z$)nAy&2x>TWqGM{HT&>9(zT1L5zqciz7L#TdpOz9&A6NjxYLk>?V>`4%bPb^D8^H+ zcT@GGZ=ryYWL}Eic}98nokS|Cbv9E*l!JlI;0Ap{TnSe~j|ms>XSQcl)7iq7u;RYU z=V4#oa+_+LdiI;YCPAQpfS32UV0X}z`<(Mm6S?n>5j{8K_SShP$kd&eR0(xfOeoFv zb`ip@TP9W`pdM8JjH338I$Tby)cX0SFTHCO$1N5W=a z8$W!eC3?~k(8P&awzbe3T_zkfez8z{-vA+*?IrsXKaBF=tfW^&4W2HgYB3D=3kX(B zB4fb@E*b8n`XGm$Sg!rRO2WayKce?Im$=tWV6q(_;%7GvLd|HyNcbq)hAYj>;3l-i zSCzEMaZL8`Zutk5Rt$1#iJtZIgKFx4AqtImbKh4;X+`}qvvSQ@edjF(_s=Rkg*k(> z+0U#ShdYRijXDk^YShrRV7gJTOLBoEu|1;G)`el;rC2$!PujUuw^Ev)`!C-%_p%HP zgxgHSM8+Cusi|myl+`t~!)_?{m)h*{6BQOZ78bfr4=2ZMaBl_6q=53^N9WO%23}$< zX|PKP5jiu6=!{fUV;*OBF25C`g3TJM#w(b`nVY;R5}sV8&w2pGZu#CIWx&%V$#i*v ze^GIm`l`mF1IZOp!3Sqgxb5TE6r*sm@h(wF02+XUkI23{rS0gW2LD z1k2}*`}Po#t6xvbf}O|tzGn(wK=SFs9phX2!w=1#2oZ1Zk<#Ac`p&QN^s14{Ai7pT z`2mHBVM4tIeXQAc1K@l7NX>q_hsR++M@f4Gn?ff>dM#Xo*vM4L=XUx)O; zB+U_X4-KmR+*{Ykk^PA_GMzbU&k{Z$4ZHoo2z%hYXVb$M1WVb#Sr3sKfzlehsrKW!S@j zhdx(RQJs>WF3$PtGg7XmuPwH6@Aq@!w&o3{J>QXU&KsO_kd`fcuD$kol5Ags(18DA zU%e><)_aPt;=T`CS7x5EG3TS{YEbaj1CO-nlQnbwNczAd!KqqW&Ajw>n>BI)+vo-Q zv7lzv1sg$-EO8sZEY=vjfY`+c3!=MHbbNuO&J-Z(D^1047~;psd|1YkE5j$tMdXlu zOFOO~!-U7PTt-Ii9}^Lc6|om3-u4Ssn(!`NTyMRf3jbWLj`9iaq016(m0wagJG{f; zd9SQJ7siSm!AV^?@Iix z!_j)Zepu?H;Tf2#X81Mc$DFDxzk$a6<>hq?;qkbHnjk7YB>S$PMDkfw>)ZMU_8A-8HWHG(Nn7v_z2;y&!7jjW04K@(?b0!rciv8m8RI3?~f z3#GNAr=(Fv?Zi?ToPD#)V|gWi&_X_Z|fdY>#NZE+lU< z#ww_jj^K@7&8}cZNJyw(E=nT058OJx4LD>gQ{Z)cnqBSs)E0)$o<(Y)+u-!WL<6TyyKh52HD2(?W1HSM8+LlU zUS+nYUEDsGSiJIz8XQoy;mvQSv#np)WiIQNQaee)nE8~1x@u`tleS#Nz$jm;3c|so z`Kab`U8Ds{-e7fr9^q~l)^^!i^6-}W+$B5sfPpj9UuqHd#6)nZP6M+ghy8OTzBIT zPR)5-AxqW8QrzP$j~=W?z=gt9t_Y9mx|tKgsVfhGD{Q>Yz5{!!d2@H_k#T&cnA( zI=Cgw*|IkU&Q&n64uwdP+(Y}A^(lOh7YXkL-AH7mSyhyo3}2gbbWQOVJNx=@s5({{ zQ)!%~qes=Dd|b)clTH9pQOWSJ#>lwR-4*plRI|QXzwLhOts;~vhXBGtT+?qMAP$3} zxlcX@5s&*X3yiHqL^x*$>3MqFne4ZPKqFNWFYtWZ;DowXZ97!!-diApi;1C+SeY0V zsw@J@28qnhEkS4?_pROSj}eC~d&&14P=OL&*7TZYZ zTJ#b<`BecDo0n|o)p0MBh+}HNWDL7bke96iqh-uhfCTEafbYK&#`|?%MX})i&cXN5J^+V9a>=Y3CAwQK40w zXK%DJNNZgfo! z4&fhqrFlHF$x#qnDZQEgf%bF^~&>bF~skY#?orK&k4( zE|fKBjjkK?ToB3jIII?wvwf=r0SJ2RGx7@IK&o9I>MWeN&7X>earR=LRF3- zh0Y2RdTBsqAsDO&_isV@7@_0Gy_T7H8)*7Bpmo$e@j zQFM|=C?Qsorv7`?|I3A)?uc2N@frT*N>r!8K&T$7>j>P z_h;z*AdaZ8*juEnM0hQY%^h~;H4C3$WkId$$~eFQx6JbJd7?xJ)-fM%){-W8du%lw z;n5+7=N)v8bVOldS|7HZ>2N3+w%_uT+<9VPW*pRDmZv^|{K5lJXDO&uGTN6uKyER$ zIhM~>G>f33t-wsjT+?{)4Y<5Uv`sn_9f#b%jhQS$$j)plZ!k~a(hufm>f6<M1vU3_a)Irl)o1%&NiN(L`vgf`+73+Au_daRiKDg?2CO7eD>6kN8_=ghOScD~_RLY@1*7+C>o~kX5IL-V@4RS&SHY z2SYz*{V?TySCdSb(CHx5q^d}2jy|UA*8P_skUP9a_PK6X9mQ(rhIJ>pF3{Z-oC#G5 zYcqzV_|}vZ66xHR>RsaBtm&h8pV z7=Ea@qY;pa)YqqzvV9<{@%1HIsMkC>osriVp~AtYng$(GCiz)NRO}XW{5QYtVyCPG zx;hHFVbG8Sde3bx5FHlRbz)VHGzo7Qzmf5V;`aI~L#!V<#)z3@Y_EgOgCPdupiEnj z5@aHc@$Fa6H@q1;K%&|fLQBy$D2?gtJL`RpyV@;0qHQ?b?6G=xEk?aAJ z-qaVRO;dG~(8LQU4=xqYuD`Y1Gl0}>t|nW0uQqRR$Xm?JemQE2t-aq})A_3JdfmMV31PljWJ0uRfgHLsY2a0P*I<2zP|;$Z>ytN7 z#y)Sb8p^rVkjbV-DKyV<8YA8e0`3|%3}GO3KM82cSa9}rs0W&+ROrakk8z`7SVjj~ z!A-kth7hsRCDiAAK~HWB4WyO6NpM2YlL*c#IDXNmx-l*r7@{w))1}3rMS8gRA-CB@ zZh!M+Kdp3^_KoS`Efcw>jmJ=xgflcKojaLQ5Qc|2#D8FI_<>;G*V^VeVAmbyevv^P zA$qgVqFYT%HIuF0$=npI1ueb$@|sXrzNty(>@cP6%5R}XN?vPizhX+uCB4dGC4p-^ zt0L+VNy&`0F)ghfL!960R|vaO<6ibGJ=T=Pinrx9U&f1*fy}Fh9T00@3Y^;{d1~i|(tP*19OF%toqYqb?a&}+DV0Jr2_-;-s65|@CU9GIyJnBW@{h9aKq3S zsX6@Weqp&COUTY#@BLg4A4ofUJUo7x*6m@xn-@FH@Mz`2*@~%1LrR;6T?0d0iDypL z zY`8UkbVq+iaCiP>?WWkrn()v((yayluLB#P4Q&R&{d%X5>z1V8$Kbo?vk^&7`%m4P z^u1J3wN#<)ohYzOI!$|7oJUXU16Y*V=avY!E+lf7WzYlJJQDPbt%_BGIj;{6OFtIK zlUS1*kJ5~7>Y8O_?hcGU55~`AKA=MS`sF2latj&75?P0MdZ#59+=;CG?GS0h+mXN#3uM2+Ar7}!fEf)!UgLj-;%3F65np5q_lu&NOAz2~x?sKKT zD`M5vrWJl%y+mo--esP^OBVr~o27MZH@twO75c0of)~DCJze3sK#mAC6GCTyd=U@w>8#vzoH6$fxw_U zFVv`_QA7j+SN?t{(9b1&is|cIsbOrqy(LR3kDXZkRD8r}ChH01N35ic%w@Q zIgN9pT6+B!rv$A{T)jSpA$ME91L;xp@X2pi&{J zw8wseKf|icgO`7f{6wQo(mrsab1jf$dJDXLI$G7ACB z04Ci!k)9u>q4w_9^MnhR5uP*^m=xsw9cs@#8GOOYUcYuEZ%@Xvaw82EV)@0i&3*Y> zR9Dfw5u5%5boWOF7GjyY9JOP_jZR##c9RmG6p%+8B}o*`L`6W!1-Cj0mdyz}Nz03F zq4T6D$GbvJIk-bxq*wTpi0yC;9L3aLc=d=L{y=9`$PH>4!<(a!$$=91G^64!B_g;o zmo(*xRng)7Xrli){K;5)5D4e zgjUEmc)9ahNyd>Vm(3_x#rfG#_1wK|)wq+wB*J@4V~h<)}!19smpBY|R&N zBH#dc;S^j_FRdQhIzz4ZC3`qp{#%_t6DPT*MYr4S&>r81yY!`zQ9M{NaGM@IsYiUn z?+|)OCnG48Zaj~gxkSD>=G#UkN}4?y1a5n!#hA-heX%9?Tu5b9p!Rr`1&*;;GeF1d zw>6)yXT?{%Z!f0J2>;?BUU$9!m0|uG`E^kTGI)Vk+!?Q992?8@*SST*1bie8a&n%k z@NzB+c!rxQXXqp68)xKLCtDvKyij!xT5oRqq~ZP*+eIjipS+wvWwaA(j*Y55+7Iby zyW^dB7}^o(3;FWvbgy^Huw@8<)3j@Mov0AO9t+3fMhWngtj=4Djw*{Zrwm0J6X@A@ zQEH-^n3P=~+MLHx$yx86$Su4PlpOX76C!k>aI_;1jrm@Z*}n8_+L4`Z{hqC1n~()r zn5PBLD=eCf+71_g?X^N&`&a1<{}d1p_PZ=}rXPv0sz0dgd-{Y4h8JmyFNec8OJAh1 zJe>e{VJ<*~cT0f9fJ9B?Or9v%L@qf;!N(&=)jNZC2e76VUAmQ;qh^7sjrgu$m_hxtGWrWqz3o~TLN7Y{6zVmmtYkKe& zbnS(}F59Dl9?Q)X+c&RPMo|1uC`b+lMLwx#+<4K#RYx7Hdi-cPXvM$*XFc$3Ya zYC$zeT0fmipY?w>!V40~-y=fxm4%Lw`BHR+Pn2$itUNw+zT+OpqjDw-VV%u+fhHh7q zc$3p!V%h90cq}+R+~y?`^n7y%_0$ZRjC2CruZ2SXuBKMMqAqVRL^knJ`LBQsz*Crf zRT#JOKsr<6jyDu_C1@@i8vVSU3+SmNVm^P!z%v~S-?X%R1a&?i(3VP+PPhx$F{ll; z2-hhJcQ@*gT`-}hX?cog(G0?YQ)J*id7hXcc5s+nE4y=x6&yV%GTanC`Js|i|juq=m~C;KQkHk+gt_kBbSk!wVL0m)aVzEa!uY-fh|YwWryI?1EyXq-+$ z*SQ~Zd@Sjf7pz<-$a`E6`FUqAMASE0@)b}0_RT~h+xq)1lD?0p5XK~~v;|=rvdYbR zT-`5I@wEf%w6ct9ZWQqUeYiF2ehCVXU`19nx}FUD|><23E|@5=*q7$gqjpWZAsVqh6s< z<{H!aucL3%%p-<2WBWA`rc)b*Td69~_Ak_lr`_7~29*iIL!*&V2{+YPemQj3DHJr-<;a(|h%(k22#j=8*n~$>7}KeEpt~Pc{DF zsA@*x9yTnAZXZ9rAOYOxvJd!F3c}Cid%wy_Biz?%OvM2lG7qhRIdAZXed6a|9Yr^H zWX8zx4{9@z^7h5bTh*;fgpI+#;sjbud|!oqI8)lyT*8mD7G0 zgzW4I6$#FZ8wDz_ong0t()fVb6^L%YHX_jfvZS%vF`pn&?o?tZz|FWOF_xvS{}S^6 z)!&p?iOLt!y7ubn0{ttj4xbx}`mTmm+r*-KL>yAtf_GQ>p{N89IpjO4LjJ` zsg{nz@#Id>g*gU9cdo~P9?3s{i`TGwzaf0?OqLyIK$!5_A`2CyBx#!djO}KTlh&#% zJu$sflU<}?B}W5oxPc2xTpLmT zDox2hKkzZ=Q`;Gf#Ozw%;z4+jDS}VH-ayIma#z99Mb0rK@n6nmwYFpjq=(d<{e}_gV`%OCynB2TYQQyIU3EkJ-=OPFtiOyx` zurH=ajYaG%6n$lysp+hymY5x*^OB=iFH0fbAF38vSCgCih?Au3I+fomn>}0{UJIxvz&2Wlkig#1fR*qwGxPS;VLO4*Pb4=~>9__|N2f5V5ey%u z5(iVVZzlF+5g(DV8EqHmmWqcLl-mx6MvkLlyzLUq`XY{WGL$m&JLui z56#aD@?^OMTFrAdw0i?h-mFcZx6XplW@Z=rM}CkL0t!q@*~Y*Fhbr_3jh061|3j20YV#L<;|KeLAcD@JMIGtPDig|@P20iw}cj^&Jk z58}`uKEdl)^u8vk!6N-bUf#nnQfNEL>AM%~9R7ND1<;<1tRN92V@-?8mCI(jbtx1h zIU5~m1-s&WIvA>!=aHf0#7

1Zt7&p-*thuhCX_1~``U+wd?y%P{=1;hm%iJy7AM z;ylsKHYb)sE>fI8$Mirem5fk+#RJnXnLFfSTXM!5%Z73KtrBDVKMD+D%rDG%e)eUa zXV#lQnK3_{bF1vFNj2VzPmgFJ(3RnG#JzAHnS3WrXckY1SoKx}YD`TWFlX{duIn@M zJp8M#x?h5>{T<|uSl)!|eW_8@W6;iwd~e*bdxDujWg0eXi`0Iju222KQPcvmkd5T| z)FgSg3&}uwTk-8wOu}I9fbn;Oj5n-`aqz&;^sz*FztT%JL07-e1H&yhD*rT1by-GayKb35B*Zm z=?+o4q+3x51(BQqh6Vv)=MEpRh7<@BbG8dR%x-7*y^JzgF(;+7 z$hELUdpWjYqsVN6PI1m)J+#(I9@u=CLLbpF-XWt?g1?MC4`TuRSYGRp+*{shrcz8otZ+&7tB4Gvc%+A#@!JT)!_M!hFA^a zlZ|itC==IKorAqqUt+Vaz8>&yN+|EkrQTnl%N;*UyFZlz`7vqw3a7A5y46O2HAHrI zA&}q92o7EucpF)i~}#SY*TK$PgRT+)v-&}?qjt!34x2tEK$zeo$Cim9q5C$@9fs%tqOekb|myFk$=ssNC16Yiz}i^?PsS z4>#9b(X@*<{7rm_H}4uqlp~PZS2CWe{;u^uGbsdW{ck7PXG+22LJM9uOT+d%??8SZ z*1&d~8a7DBireEJe9lIQ5GI3Wh+lrkdKzI0+A}Y%9|-i+zF=*@NZl}t!c@xw zg?*Z}*EOtPwLFc}AzonDWJTspkkadjuUVg0HGim?>h}E=O$eq&Ae<5RWXyjZS>R&e zD^DJps}K^Gx<$q7!dVt~xoonPfRxZy8jT3K>e8Y#iz-+`&mnaQY~0e5ON-+iE7hEa z(3jtjx@zAL1Lccx?eGrD7R!^T=U76+f{theY=9BP}{TLA%+QF%# zgi3+{C(M^6=ba&k^0R^BOuhz3%28b&g{9z8EkXwTHQccKHnEUyrD9uN0$zc!z9QDy@Zp!#fRS{x}6ppV_&xHG4 zLln=YU~v9*c=U|Pn?UWm*6l0) z0r4M~*ZntOhx!d<`LFN|=0Tn8cYuN~x@yr=g1WPDzFdup`Ld=6Ml|Yfn4qR)O~pf! z_XIz~8kvSlz^(c2>5X>dm3PLF*oIhvata?%$P;f0FzJ31+6t8R1I7>9^8f?_n~{Mc zYaG@0lj1QRTH6fjL?_NksrxD?sroO=)(TxQ?R{34&N-V))5xcShs0Auvm9qW z!ZT93Dp2y8M<}~9(3_bv1Zsc1!DlBBIs=SSvdWeO6KJP z@TRTbxHaUn3`w-tk$V@Vc~pw(ReFyOeIjwQ6teCE|QmOJNDJEv1ao zwKEQH9e*dO?(CYhOX+m858^@t9f3T>-AVf^*+Iw$L7;Z_fNv?ySO{?L+FwSoUFn1el1qO$@2sHBE0h#hehkygOIdZ`-{e zpfIBVpCmQ1K4%tPEi)X&A}=dfn)^_!5$kNcl$N4mS_sNwI-q1=kv&iB)MwK+QaM5B z7LE>d)QW}VAjuPBlQeS>brnhHrAYe$!E>HwuX`R%lb_Vqv3R5O7i8VMmK8PsyN*s; ziBb3lDKXv1mpDiY<}V-U8oCnrB0I~O5o9T~Gg;bEW!Dcn z^pLa+A!=}9;wP>*epI3S*>fE&&tZ~C!JO0`851S_*=uuVs!WfjiXEOXe#2=g>f;{W zjt^6lrAD*tvgY}IV0r8nI%O7bdHK$fu8P)z_X+#)Y-A=B3>1$O5Lt%SSjUdqguOL} zI>nsi^-|4RXbti&P{XxCXhRICxL9PXoln8?sE<*5Kq*E+p=dY~v1-sSwj1HyXmC|h21iPBe^iq9MAx7qLh$)+Z=_g#=i=($> z(E5?+zUcBQB+PUOJO+)cP_Ou4Y!yNw1|ow=kQVKxn7?K(nUCi+~Xm4Y@nS z6XoEYZjMH!*4M92K3aU!nka6aSQrgQ+4(SLYmi(Q4C{M|_NlG>pk+j~3x7P;m5T3; zZvv;^5jm^DU@0d>KT1IJyXGOk`uK?5vYRRs?Inc*69rthCkzn7P=SwQi=XdMOD^+{ zw3D(NGe*@ID)R_R;j33=8$UIIq*cd9at*d7C#thO$e%7h-uS8G$}XFwJcy10bS5L2 z@U zt_&N0@D&I`bTjh6A}+Ru@P=@a>?v{@wFI6fGT4{&c|3~ZyLisG$lV{t?i3000ee7} zSy1COOvyYuYx{&Uu6y|Eiu z{2yW|vcCw#tjnstRtlDPv;NZd(C)q#GMK@M)R%?w5{-bT^WBB>3W@byTE850OLjT# zmmU~D)T#nz?Dugzn+|zmbHW-2Zw5S^Uv=Y1r~flOR4r(@l{lt_d05!Q@I);rqY2G& zfUf*dtg~&uk7^_OmA?6n^)5?OXtr}Q?pQtks6;?J^~4jMM0NIdyyyk93_h%=C9|quT#Ir4xLZV=eRdjUva$r&0W7n zB-wvR)GD|is4(<=M@CTWs%uBY_27BP$yD_uTNPwzf$P)t$<6HdXaZzUwj{{u_~AY- z|A(_kl`=?|!+ouu&L+Jy>O=bb*7rD?alTZb;6UTpI9fH&emgif_VE zAUB`5d@3K2_KDUXMU22G{=>@0CU+FPg;&BW!uub!+d)rWc&~wt#X2c9Vo-qR%LLpw zjrbIMlAqsHUQX^h`7|)k>AwSb&vt5yrHLfCkWBl!>ATNJgW(U=btz4NiJ@HXY5;bG z-voP!$;J8^XnV)?@Zdwwp@!hd{xFQ{1382p zj&w9(m5_0VkYY*yGd)yYBq3!{pjL*T+m@Cz?y)ZE^GbQKn< zt|C=}xM{lqKFHuia%OntV|%IE77ct)WHU|)%HKaGSP#Z(Z@8W4!m6_hp8qq>B=+}l zQ{6Bp(%Izd0uC9!W1kzBn`zVa`E;vB@Si>M_t-T1aV~k1uP=Wz-9&P%mIhyxX8iH- z`XIrE+48B@^ z$cvuAl5&^_7=1ni*n*`TK~mRb>!9UFfMa=i(R5+v%-mcZ_seeoo--KfncbC-=IN== z>7q)&jTyR8&Ob&Xe}^B00uaqEn2Q*tDBdI!S9ML-#nd+q{rlqpaQ2`1{_}?5LX?}4 zAfKk~DxckT!nY%3C@++MhenC;Y#gI3=b#n{&umJK;3;}UW)6nGuAi4-WXY<(-I8dO!Xc<-r>F|#xZu{4f)JQ-48BcvI~0D0*2G#(MCq|MAm;o5N{qrQ>;_UwPJI>Q|V4%zwHEIknYJBv>o zBObdd#KJCj8zhKsdqv>=9=WyOrAH{7gZuyt5x z@&n^z^FCyu=JWfTi=%lD(d;zNIR^|`=7uk)X=^j#c1aTU3p?Mq_5;rpCmjzM#z5sc z{Wi{$SE-#gGJfQ8IY$LABC+e6UhJ!6V6cve2fjK@?eaWO_#jdw$(o?;eOiY>`1r;+ zlV)Gi)6t-_Yeb$#m=5GR826M7w*V0-QNW1I_7D*59}ZP^0&gv;{k_Lj75>w2l5Nx5 z$HB_Z&c;)i%Uu9tW9J&|<&mGSlq8(fDR=fVeN}W(>b<-sJ*naBX;F$uyXGh#Z2Y`D zq;bQDgi?+et$iW;ZHn53>^-;}*|d=GJ-q>G@W(hJdOE12+vfQ!Ph(cVo@gH=?p}$g z34&@N20JhJfMOxZ=|?ej__3;5G^9)@c5BpqkPDLBFBihA+I9 zi#l|z@Z}t!S~ACLM^<7~vq{mbQP;~)omWUVC^eqFv?v7wWx*duP}9N?W=|aS{4MSG z!LvyojX4c<75Q|H47MvfoFS@=l~B0)^!*mxNH{y*$Ezi3yTRLD5Mu`*MQ@)d9VID& zz{v!+M)tp}lKqzOz99F_oa4>9%!h?!CHCoBzL0Gp67{0aXx)RF;g1K7VD|r++*iLR z_f9gLECE!jHd5Q6n>;*BD%%4jk)$7)AZmALnQiG&-bJ2%qM~RWs>t!0oD%i(Ls8a5 zDg_b^?ZY95O0V76xISCh;5K}V{J&k45pt^~pCPJ04We*L5oPO#HYr@Cik`RpLId@$z zm@+AU_vJ^~u|3Lv*Vq7ksrXM&)(_$5E9WnF5mEj2y@#r`Fk~NWRCw@s;iHS7+*CQ@ zGA86%=fG&yP7ZuClxNXt|Cz2?k~95Oi8*fE0a;8I8W9m; zg|4OU-|3r0;MWtMo$0l{b-M3RJP<8uJ)k&0;}oYU9z}@hxr@ zZm*v!r#G6DWj_}ZVTggS%N{3gwm1dXRC?*EMXB?kig~t#`jl2UXFZKnNIf5Y+{r9E z32aP({@L)P!oCRJ-&ZQH#&2-%Z9haeeIrQGyba>;19vqRXz3G-Ey06I`goyecVOiP zHl6BSxf*Y&AR;_@V5I5xOD)@&`qwWf$>*+r94)&@UIomNSGypl>Q4`5e(c-LcwNsT zc16vKBfc(0=aX>_4shARAcln=kw?wwxij`8W6mWbyv6UO&CJs3$0I+i7uT(@7nqY+ zb?ATN4{+DTId#Q;58wa3F}hq#*Z32)JFxjObRRQ_8M%Ns&iG-Y&{f=zr$z7duNfh@ z#f4u!V};Fm#pm`59d-2KKH6zfF0SFMA>EU0;9)iKS%bAd!sfsnN`93YsY5sb{leM zeqE$V^H{!4YijYZaN`%R z3#HyySb+^+)Vw4C>X;U+hWrQ|9XE$HY(fH2B!8S-RwDzwjcV^foqFt^d+RY@$64qz zJUheWzqlZ9_#fvlec#Z<=|vW(%c#`KLjh9olJ-lmmlCYvwGJiQ6=@o6&6jbiJ;)0^ z(po5a<5{*m3EvKb-|6?@fm=V|+R*g!QtMNvuGY)nj~mJB+^3ngq%p4{>s!NspM2kd zZ7l3w6pJhAKd*qDxTK^5i_V~qt|Q8Ks!pkBhjTl)R@D0+VrB4=U`#}qlr}GU(mh4J zGD>b6gU()9?8XI){u-O}zI6k4qNMXN!Gqd$8qwLDUws*I92p1gLwV9#cWv~G6&AS@ zcD4m6^q)T|WAzIT9On~h9}BGV$+eiyazXahxNcEl!z@Q%8?i5X2-e>!Lr_Puqg|%B zYhZp*ak0yv9tIkHh&jhlmw5_k6+}d13Ci+w$ayu^dMizD^aeG{#ly=gPEC4aQ0_ebQHY@xOpsSU?Bg?*A4V%DqrUUknQrR`EiFWFIP_jG%_6q*YxvGY2*;@OG}`h zP>+GWk<xI>gOMVJT)dp7EX9(sj*+j{Z*oTE^ zs&q{^D6(gNUSn^dXftPjBRz2;6`nMvfOkMzvQs>H)&p}onH>L!bgF+NJ;nI&@IVlj zja|FN+~N`G^r7TAK`8W+=Ar#5wtS}jL%p(NzQP$KRG`@t`GMg&8`jDpRD2el4Lb82 z@wp474sob4rm-K_f^a@QxxGtgTBb!!Nk+3b$s~b=lLZaxL z^Mr_ASV^wRsJQECPB1djKCC0%RAG||XS!V!d-WHtxGoq{nSXfOR&|5rCdSqUBi%zHq0d{=lkQsof}eDx^+YFca(11EgNb#A5EwPtXlww#n%+YYQ}OcVtus!h z4;yRL>+INw{zIG^EXW*#-jH6xV@j&%8;N#lX!HeWF{MF}SNSMYpJ=H*B)aX7Ww{6! z&7@_9d9ODm903)BdN~3{cKmu}9%0zFQPKOy&TJYHE(8I#I4GhQidSnuJ1Q{&7bm2d zu@{KbXNKYS=M9q6hq6rr5JQg8-CQ!!b)|5*0j>rfwK`!uo5N6rQy=TMry4aSx&{U_9kE=BZfSDLQxwILHD zGKNEHgG~7>(4T;!^7d!2A_hV>#DZ7rclQMcs-`cv?_D_vcgQ@D zfQQuNXcG1wr`7PmTyWG>UP~vg_%wS3C+cR^Dd%28`lsSqHSA_F+>4uRDjurv5$+pp zy`X#oJs$sQk_mPtN^kqszyPl`FX)5pdyhv&Uku^$l&M&sF39O)$3kqkLg8DJqCx+P z*`-*2CnkZFxy&H)&m!?{0=I2{98HBYdU&ZROj)!SV!-0`?!9}F)3{YS|G~AnO+OWm zuOvyPap*1PJTzSXA(2&YaJ-nE*A(XDdS0rE|0;Okvs)Gj@VFC^^=Q)Kd4eAbqEbKD z)HtSyl;ApWKaPJ8;wNqNQ3{*JYl6A&4R0gzNrEi#bOn=z1;e2bw51f8ZXIJ42sjcx zv9M-7SvZ?4xQiG^X0TC?PLW}N)ev02=$cZN)n?E&s$D7`fRv6!SBH!rl!nHqsJsQ} z7vJ%Q;{y3GO6Zs(qY7uK$mlE1SL!U~oxR5V^3=BSHhISSELZt-#?27aJPn&*<1j^^ z@XiFy?QciVEJ$l&MVH83ng5$&rZX21ckqZB z>a$ocE#+rX<)`1@8=I4`?T3$LxnWC{1tCz?d5F|x(OzRr*e#_1Lk%WaPSj}1?{r-; z_Zu&$dZ2RWZk94?kt1=IU04<6JkkX<9u%&F8-e2f+gb#}>)5>8(WHtFVnk*g9KWhR zXoenFxH=)^U*5Ua4G!o=uHRGq4`mNGlR75yjd7uU^zxW)L-gJ)WnWZI4#2f9Jt@eV z&OwqLzHkiH;yI#DisWmmNS;zeK}^s=4Lb(S?L!f%J|GZa{oi=xQLL4x)Mzp5W~T$FJD}qC65r8|GloFnt6B zJo4OgVWay;MDN=Z^#WZhr`h$Lessn0%ou{yJ2i)8_V>50QQmn2! zw&Qx3k+6jhMxM%u7=C6PkK0vA&XshRNHnhTh#ljj7m`d5!!y|md}s-lp-P1hUu$Wwti|ap#p)5sVf%VVn6nOt!S`+`vO-7<}r?v!=8*C?^Dx|syKDR=i)OYtYI3IjI zvPYU$Q2Dvr>AtGqJ`n~7BX);FpWb;BmwUbw$1CdBhg3=)0~p191E9sLplXIUy95>z zgj5_Zo^2OaK{Jo!6cKcLdB`CXn)g8#ZADZP*=Px#GTHEDP52O9_+O=ctiXW=YOB|- zr_iUR|9afe+x-pQ3e#BPE+AfkRM|x8UgRSY|PC0AfUEnO6{1=+6wzm%Vc9d{8yXFa}j z+OZ8iQp16^1AX6;M7v8shAEX!mw<3t%3U5H$&{#O|Uvp z`?fcq`VE%*Ii{{Edje;taPzfn8-xqt&q$kkd&;EkKv?MNv%8{D4phFCHGREUe1o=E zWyPS5sl>TE8dPEc?$JN@k~1tkNPKZhGqwA?dm3RE%j{`(EUqJ+W-c~+^9le?Q_Q2P^e?5m>Gm@zT1U^rs$^mnM4XO7i%6(f*-mna zT!}AKK~-L_<+monV$Q?gKC9zi6$7@z%*ShND=(+DY<{Sf$j4WK^jlY5Do8+o8b~ze zuiMt|0sCG0@04B&fLnq57-&sV9+^64-sh}IOn@8ZJXj)WE z^M*k0hI5z|K%`5T6?_TA<^0ss!JV5+x1%Yi8)eZITdx{r>4QhV>& zIVgdGFTEYC6VYwZD-VNTOVy?X-BQHw{-n--9lh@Z{-&21bJ=E4Tv<{k_!U<2i>waP zpzU9ddLgT)ZM3v4hU>g2DUrH^-grK2S_C5xnbcSlF=){R$a+Kwe)gp!X;%j@p1BP!l@dPTMAwOki`&+S-;3 ziH2$o=F0mm7eBJhKW=tYE8J_5uuziSGqVT6um8E681Vn3T35;tOJy>3U=iYjfv1K8 z?3x^RbQ;znylhuxU{2ntn>Vjd@*0)T966w# z5YiSrkstMZ{DfXs$61BwXs=#!a=OI#EzG3aiCz*CLL)S53a#8L>v-oO>Qhr62Z z?QXT?$BJhoybbsjiSSlBY&z7H0<3wrinE034B4apm3MMcq`{iv1h@#=+fVQX0jJ?HhBKhV#% z+vFX(C)$aL5mz6_rRX^L880kW;QQvc(WO}eYDDiLz!Mc#uQwa}#snG6&V=p@t!MWF zv(RI~u+s_&?;MYTx0WQX#bwU&IX$5}e~5UMUxElQi`Sc&_u(6iC#ewUC~(6x1Lv)+k48aiiZMl^t|E@D?}ks;pV4KmnLwU;Dh)+C9|eHa!SLRso~Oaq8(2Ccc#jT zbrHEXlMkUxg(82Ky`^4~{8pY()`H5KRxV`Nc0zQ=mGwYsIuO6B5_a3+8ToWl32;Y# za-}6x5{`$42o5WIVY7a^R*!%4K%0vWNZ7cwJZ<1KLy~fzlHsO^RDA3$e4O*_a3Q}0 z{Gg-Y+ku3ysgi__e+U4qR4C_7#UNL@oVIS~d48KSeApScOvqiBpEJd^on_yLqUaI7 zLAK&Q@P^L!WHoxS^yhl3#%BO7OgKEln3(U<Nh z3O<~cjE}YXyAKX&aePmsrOK?YUSBBupR<*S)_3Fush)Z;2g*$Rfnm-s_J{8KlIrIr_QNy@}p}t$mS=3cUg{k;$knRSHun z(S*bY3ijhTwKcxB99=x-nM%({$#Zc!n00rq-)L1o?9YjFm=|VHH7jI2IXZ|j=Qx@0 z*YI;7mH{%nrj2^9OS)Xu_$_2)fodB*4$I%AdkGuDsGP*ddPGCBsnODL7dX!-;f&9 zb4-wQPs&gnfCm`M!Rb#WC5@0Iqh1a$wZ`9xs{Ul0CI7nj8sEa#q(MExZb5lG5kQgAdB{u{PaD-iDoe4Sd}#4@x^I(Zzk8oCq}KW37ZF;6 z&h&L8a<%PwWIrs9tCZEl=@|!{cz-}m0$n-jc{-9_G!mooi3w^6bC%Jq;aH+f)}Y&l zTpaZ3$HqTvNFGsgrWcFPwGBALD{1;tb06>3s3{<;4VT(g2U4wZ>llJ@Gx`RfL_&04 zASX_>T>n_xF*uGvv{K&0}snE?vxOuiO6hf3{M@+CDeLp@7>Hi zI9NL@5N|M3S2bNNrcB{Ty)o==2E}4m6^zrze{}@EP|C4IgzT2SFAhQK)4|*$DV{yk z2L3I(wvd|p8EX^}J6Vum8!#2<#~HiJ;Y!QPQE}$1Jp9|2fcb|07sQo5Me#`GNib8Jyx}&CIlRJMr3T|2pH{8Ry3~16K+%> zdQopg`l@OF^I4rZr#c3b6*0TLsq0J@Np`ZyvyLa4z3{v5xZ6tb(DGP)Q-Sz)K!kH;`L@Fq&%Rip}(d@hY#g38mRXz4q zf#~pJM#1`WA({!Fof#R|`9QV#`)23V1a!*73j#q3oc_$Z4f@Om_v>@4fmX!Ej_+%V zyA6{pMf*E7F;%B$&LaB9s?5L(U9pjWgYM1=@c)4DklIoGZ9bZQ01f|32knjKw#C=a zpHG0}GN{zQl~v8G&~UWX1V9g@N^mwR*g-~)h$MEdb?2ZQsgi>GO!6F76IgX>EfB-S zL?Wv-*}oZuW}G(V-_@&Dn4mg0mFyybJGm0@S`SBDF7cDwB+;(I6+4czjbst*OtM@O zTcpn205?a50G;Sk3yNEuR0K-m*l|2&BGWAi-=XG0gSPusodKC~f=XNN@W0|eWU#^U zpLA;<WcHWut zXoC%t^tVH@+M0Qx;Wa1|p<95Bmm9m7ne};)yZ%r$8v2E0{uWV+4@N=Ek8KGZERfG*gmQX)uol zi5dD4w{hJ}4gQyC4W)eIR@NCF?SjZq@(3q}KNP?Y}E7 zw@Uyn|KFD^T1zH9O(lIVh1|p$I#P~e2WIeR`U?8@+yKA%{`--p6tLsr{zPpD{RI(s z3n?wy_?&-1djtQU(B9h;2PT>I5^)oq#`&K#H;01pt(=*1)nlGp=pjJ4- z+URy&prQKBrI*1X5%@b%$&SwT&)YsFv&Jk3;AGYU5bn`(S4~QK{(B|S7o(z&7;P2) zwwZr)^8T@S!xC`Gx6b|R_scWz>6W76>y}%+Xq)}jXakNIg#xR&XJc{WLfr{{xAlK% zKS069l<93M2bXF!>b#?N0(}RC)a^KS5 zITH#zsN#HlqK2}%x~d*TMHj54rVdF<(2s4js*@wB5%rpmNJVcqA??!E0VnEfmYG0< zI_o)jwEl;>TR<%bFJHtNcYN8{nTQdQ=7Yj%(B%xWku^b3q~uop>~ z@kcm7sbH18>qHudUE~1%C`txCmq-c1IKdupo#Z;NK9NWBlq0Y@QVafeISou$e_IG3 zNxYD899~D|0vDy}FEk+?Tgc@WA&IKYZtT9dUnFFpoe;)xrh3cWk^RHn8K%f)I#6&C zd;w0acs}N&&3KQzPL{Q0pXgZxrK$H!3Ta9yTHUE2-qW)-- zmfS{$&Za=E3Jn5iGH2)4?jyeEL+nV~NCzNsL7p1l4?uoE)ox=V@jU*j z0>`t#K^HV{pS%5XWbYZZkkw}*2@Fn313MWj4;Ou>X9XTk5iqCzDJQNAaeU{rIjzef z{QG+{llqGC4MJknkv1xBiXJXJZB^v`8rbnaCN*@qgMo15q{4-MZ_~z79BZA0n{C;r zI~p%uXpC0W!CAvYT=%#gvVF2EX`kH$&otJ4LiPvZE6dt^GhcIlbum-m@H#TtVdT`2 zR@}3;_)955(xXJboy4ZLo#%BZa zKnR^rtXdQt_}ra`HNlzNeeDl>$W4OUAb`3y5$yP6n<_HA%CL~MKUvC=8vl#U^NZBZ zE$On>{9{ZdGf-95&mJZsn}M?mQx~c&bj_m2QEA$>h^(k6N9Vj~p3=W=uo9|2#JFKO znEblNP>re-F?0D;l&P}O;?iI==|cjK{?Z?WZn_`q!{f__)2&Wwwa7B(6%WHgq&Q*% zy~k(Y&3YOSPCn((H*IgyA9*Jn3?L<+(xrJ^OGeZUHci(`uJ4Tf(Iz%qCldh|mLMAB z+^zd16SN~TrtqD`)<~l_cKSY?p4)xDbV|@M5ex^oGu-c{$^e+u$n7?s7y@LU{XW0A zGcvqUf9a%W@_M)0&g{t>*UKGHK5Mh@p{EeFzQxZ;4r8PAAw}su$r*=>YmTi)CnGSIFA9BIVccOe(VeH)1@txpkLI*MTY+#C^;evpY{iZG`_xMFpXwbI^V(NO z#i3T>Q=&IZ9yCCq5H*mx`sWa>QiTq)hkX`~2ihTZ-OW>LQR}o{!pdO-0}>aI$G3TE z)t;QOI&GQc!>an4mUO)ufqlb^>kRIuFj8v+WT*8;oAj-{t5K|Zbln8|guC_zuv zs;Bd@_vA`4gl0B@n5^tV6NBddH&Z?Ji_~HZo|kd4g4Ux@f@fw8WdADP7{LDv%B5#l zSdm?#cHM=4sh2mG^zX(*B^(0nUVNsm4w3~Ggfd+^s8c} zN76s__NeD58D660T>=>}HclXu5i(`k0LUrPiQ0j)nE-C^b8SNio`FR_DBX?(IQ>0#_q_gy_>{= z4dn>bP9;^`;+K6Yn)vCHKK}ub6uLT`vHt z)1~IMesRm$Q=AwX?3bMN@hxmc-1S$_K-1g}xJdQiBZK5cKXN_%6z>pNdajxPpL12e zcd$HKS1Y)E-JL1ot+n0#*)s~Xlp?>-<2i}gnHLA+d5mro^u`@69j%SX+w%kB(BF7H z_TTs!>xD>oTjiuhYZVk>|EqRsQ2hH4f58Zytrkt)fdJLNwcIhM@FmqJDtR=6hQ_xU zRuG=MjIFMWS0s>;_*H-^M(k`29zox9)@%nJ_)THQEJX0p<~G=3CG;Q-96h zIL=HLl}0dsiM(S6U>p%oXx8Ht>8;lIw*&zjeF$)IkB^c_d`mfu-c+KsLN2%?Tcr^f ze)P;ck`#AyYI)108Uj)mcQK1)*VFo&)c2_YIpMbUf>`p?@Wh;bxxGh$Z1<6<+EJ@i zb-9uGff2WO8DF;a4*fKH=>ElBWg!TrC?C$}LBcj3-_o-cFMD zc?5nE_MBFMR>LTgK5{Q`7z5q?W}m#mv2AaTmA-JH5EJNswMa@YuqLWM)fC1KJu0d0 z7(@nuy!1bE!XrI+52G?*q{U+&T5|3?y?hYObNj6hn5By9!iU*pXO25E#|`v=B(H6O zv2K7vRxCNUsv%`)eRJzCX=FGYK!e{Q!L$J(mnEbxsn_nMXq|feP$wGPh}VBrPMcMK z2PH4bpt|MGl9hEtQlxZRw+I{!DS2C#)pWVAT2#oTfzhEsHxEF_9wS@D# zHd2wg~j{1LjxJ?bZ8R7U_dQW|H z^0T&jgt1g5R0eku<~!rsO33=2L$L1R(j*J*x3I5&lKp>#eH(t96?QF!!l%anmRh}2 zpWBF2=-&h;Iyw2Ok&zm(9GboCn=9Y@0wi@&{>+$PlvHc0<%9!cGrxmWr;5aL|seb+NaMm$UaumiB=sA zC;-_Aqci}0gIP{;YWUfjf0Zo@3M4?DRNHP(B%-VmD?=!hhng=`r{{SUsNUAreTyQj ziBZP2@{;~(0{Z1RX{P=u(RFEVuW@OYJGdgvoDM%W5{VW5zA9$E>JYL)V!by5NJjN$Y zJs07z4A`dr!|W5)!yqp`wqUQ+sXnvL5QzvLfW+ch8{T`?xj}?3l$ED+?%1ut5DYt> zT9M@S-^hF_ja-f8b}xJYVv;FWZ~kgj0nJz29MAn1q-S6^F{n^PHO&=0bPVA1E5FG& zy#HUwxbcu5A3yFr{d^8Ro}h^A20CQ!cJEL-d1F3SQ2~w$%TXL96nVeq!q@IVZ9*@` zmtB?J&0K*;N<$7Ic3lQJn1G(%pG8(PXZcshN;3}DANGL{@K)6szp4^q$jQ|EzZ@B^ zQt1Ke^*AHeT8!DXl~1~>k`zx2 zUOM)?$|qV*MeYF(pDd2m1ro(9(**EXFZSy-H#=8I;u+Dp7a zV(M+$2C28H|A}7rrkhfR!XyOEbymA6zwKiw zc=iENJ5lJf-L@rZCq9pQ38Bmi)>mZ(M%U%ZFxx=LVpzjfo2|Hy9 z1nBmwmW-;gpY&AxE=5DomV&Dc%85hy8u<&ud*SxjIY3!ahq$^N(Om_1cm9_<<4gKp z)9;{b{rEGraL~ItkBD~~fhOJECWQysC6XTRTAu^`JDi7Ox>hB2Ku-c*@^F+TR2B@f zLe&klL1x;w5CIySZXr3mO#qUEx}IuDi~-nS+rp^2PN0uQXP}#f!TT~ET(|mL>v1;M zL-`i-8;`jBYW8SL<#iIDkG*t!Y8LM=zPqw4{TS|snBlMtPL9|zwLhZK#8#t@9> zpET&iGg!(~2HpMMABr^I@}&PFp(`tvZi7t&fhF7c^wY#z-@F6A4Inzw4R<>4mOJLUVVdbnq7 z4C~FQMos9z@psV6Zb1n-+pr?ZCoCe6M!Af;qGa(#R)S*4VGzurTjuklUwGd8Vt03%nA<$YgN+YSD+!f*;b!u} z4de#Gv+_c>z=_Sb3=52vRwL}{zro7&KKWUjM9@)$1;eP$KN{kJ2C69<;mRqcx#%dC zXpx#bGRvlsV%p3)^?6k1eColpbbR?NdX6$3K*#2X1_rp&-GTbMG>CzLe^A$hR-(+G zI<)@T2VDCOX~RfP?gK!x)ojvr|2Bk_vEtvHk2cVgbiFEno;g@xQWwSQ2l(8ah+Fdl z8o{ieM02nq)q)!8;yMz3!*rDq{||3(9aYuZ_W#l#h;&GYfHZ=HGy>Ayi$+3NgrsyM zA)V5pG>cv!C@I~!=WGu?YX@1y(oj&a5r*I2>!sP z&szV|jIDoo8_y*BGf`%&Yy~ku9T|bsV|H$e(Gb`8Z`s)YO-lH%OBRxHRSqGT-3#j~ zg0;_Jq%kR1w${s%2(ZKNCes~*O8YU&?|uG)>HPnK>3ri5)J+14a#Aq8doKn+7LBDu z1o~-g_9bgW$tiY9!e>V{ieVlBsx1)n6J_6U^2rl>=wO!5+Osi~ zS*Png`Gd4ELZ(rs5(H|QHI@RMQeu*;B=k~sN4Yq+aL>DGVB6Aa9|-Z;>EGGjS?{^` ze3L~0jAEe1=`^}v(r09uhVmmiUVyH8rGDGKNW8%lBM(NVraSF2PZ=x^3tvt!q+}+S zD2O#5CM80ckEz+CaJ^tkLa+U zGJz2+1z)z#vKI!#nfy}#0b^xSVZ&NKZLp;qLAx4U2GVoC$CivOJx?ypAySC|f<-G7 zDHnR4tooVw{DRs%_8n?;p3vUuq{pRe)7vZjtr&QkhCYySk2GBuynCmP{db_s^b26` zF=Hve{`lb6&UhkAA<<;qXPjc^z2TTsAFi|07BCUo^O%W37#(!}!BvC|v+j-uC2)H& zRP8HQtG^DT&lWF95VX~TevbEDEiIj$Zl?-UQ`N%{2aQ(%jnC1q(Swukw4gj&N>U}& z3BtBCnn=Md88-*CI*SDvUaTED`rZt`H) z|C&cDEaDiyY!%}$#H+AqCQ)9dp|*tDn>~|czck-QMNrj8(m6i+BX}>mh|U34AWRF?f)`V$Kuw(e2*mMl47K&>J3n!R zAB8~ISJ!tQX}SQwE>qKv*XpHC=Niz*m)GS7t~Antg1~wZEAxxEjyY8FQRS~vC?Thu zJwT8wUN*~j_X++Ds)`^0b~Wr+3pUavIKSLa&qp!6W87U%^&h#!)E=>FWgU<=d}ZHzyWyu zL4_@oAHu6Oxkf2P{YJ%qNuq+?91;7IM+11{VfAVBDH}XTjMcn^)(VeQ=#R7OOYkXl zn0u%ZLDubS+AP*OkGx@&()lmjp`_36yXwdQE_Bra76c7D&1&R`L_Q#Q)0Dgz&`!5 zJL7mvHmdyWMY9COqvyH6!G2L&WNZD!9g9pz<|)7%ZtGjD9Nn#s4t50xeCg*3Z+Yi? z>P8opR8|h0-X))a28S5s3AY)S$;?0byLy=BWLAo+V1=EAq5lY)swE|zkM$O_xBwrv zit^Tt{#aU+(QE?sZ$Z;{1~-KUJs|+Wse>7)4+uv7QVxFEjeNiyk))KcMc!qJUI8op zBoJcELpg5+y8}#?oM?#*_<~75>w;$O40R2k5w#xmq(JCtUFI+yl``iW=jzb*bAxz$ z9hhu8jdJY=3~c5#R=fFUhHHOji$$f;ePhngFPa`ko2DKz8&3AaNT*=;6K)Un8}nQi zf&)A}^eJiej}%Y>A|BCC(+70ZZ=|*GSq%0izv9}MKdGDVRC85hD+~gu@{*skumU}Z zt+d8z69jK3r=g8((U$g>Po){}cXqca$UUyF`$=~lf*$6Ze3~jNNk}t8tiHT`%Wl{x zL2kb_UaSSN!7^KBVmq4Go?o zR`=b!hFpQ#JImkSUWNwdt}_&jN$9pPp~6nTXcP*+wvbdn`bP6DzNfnx^**Pyqbx}M zdH7ls^72UQj37{eKp9CFT70>M2bP@4xnT&!*cT2P=HI~mo+2$a$+qOIWXcXvX;Y+a)tbeq6;_}^)b1VXWS(Z#d_QLQS-N~96Pfr zuSXn@8LyUh*8w*1+{xNr%k^i{@N}_5XdeitH!vLjSYxsmwX@0TZ9s`NxJiywL-xZ{EkRI_|&zGuUqv@)?VoE0lM3eWKj zGV4E}c$h%u-kE{a$@nwjy5?H>lS$zF()FQ<%`dkZaxGwC96HqH6?^))dVsp5`+v zrV;$Z1zaRThEqgl+4JEP{T#2&%;wkTa&gWh_%1CqRZ5RJMq^S51qHl6Aw-@j>RbHW z&_IWp0SBl;3f0KSCw`>`9nRPKVq4#cT!v80REOX#=yPN`!}4Li?u6)3&YR9Q6`l#} z55fsjk;Z;d_n!R`P43BlsrwO1>~L@AplmafE{^iEYhD%9m|_C%h-$PrE(bK%9DiF) zjUtwCDKDkM@ApS^p24N+zj(o%?%nlXunI$y8_=pJCiC1+#)|H-Ib_|r#?c2acJ#x- z9iuJMH%;fC;@Kk0nT*pqdgH=cJ*+2NW52c&PL-x4A$*A}7_2l;dULSKeh05*3Ew(Y zscA|)>FsztUtlXdZ%C2B^$7zQ0t6TWjd`o6gW~p+@_p4^f?R}jz@=4KcQI%;eKs<_ zbme?J%FUy&)@lSRLNjRwu`U>O3l_kI7ljeMkQ4>afW|xSwim>+CZn%;y}fJhW@P@?+|loa#C9hThKQtgcf2NNqmRg4XQ<2GgN21v$kD39{H*{@ zNs=En>)&eq!NA@8Nsej^4IJGvuD5?D4eA#JxzQ?z0)#9U-@U!~m({r+IZEe3G(nQB zg~+flC|8yJZ9AJXqVOF>@=Tk7*5p)hR{)1ovi8JKw^psk!9zsZK&E`Dp#E&GQr$2i z%UB<}ULBcB-i&y|%x=={ByZnNc;oJ@A)I495eN3$aNW>g3ARAUjowspUi!w= z@m((b3N=37nDClHMlu67#oYmhbHbK>1=r`WHK#>=P5pf}(+s7d5d@;>e3XLzplKLQxA zsmP<4h4e-5o`s0Oo&dM((o>(y4{a81Z(sWLe-u8O!OpCQE7iMly7d$BLkPXpoBBCx9JusQ>(u7SXy+y&mQcKCwd_m!>N5b`ZcRR>n@z+=A1*zz$i^s+6 zm6Rr{eIlG4)anV6OTt$<~biSp_cqbnfQD4LRtqc=+4($6b8Kx$R9 zy*HiT^}3r1T2b8TBC~tju=Du%n;_M(O7J384&!a$`9zBIir0d%vwD+&R3~0Cbik%z z3tK9}<*dm+Ca-e^5ht3;p3xT|wfSIMOy zFj%#ABZnAC1`A7HXsO)H=s_W-7yd~^S6@Ov9pY=`fY*+4+XrExr82iRqGX=CKv{&$ zDw{)21Q@1NR!d`E6OONHw$^l0Q2Wn*s_@D$Ohz%PGCW=>Hm))RUrY#t+h6M51Uplr z=s5aRX?}&a=%NMNk~q3+r9ocUyQV!Qa8n-CzfA3KSse9fD`e`!%l9n^vk1(7Vnen; zHUyLEo-LkzJZP3nkk0fiK@e_IRlDQ`IF~-m4++hlQA9<``cEF^QUl(1G_sTp;cF|< zRpb_~hc4!gk=I*s#)YY=AZBbWrYHc>Q!jq1W2lCVFQZZtsD$yUcZAb18afUaO_P=o z`$$>5tz|weg7+Q4d{W6Tl(Nl7v4m_$s%3dX)ml8NOaA&J=+Q5fQmqrtUwX~Z+?ZAm zGxLO&MW@uQD1su!!SQU$qGt+C>n+BNgLpF3MCzt{T)d8R`>hDQnH-B%hOV~aO5b|| z6xDUYN`98joqq$xm0AWg92;azK7!r*O}SIJx`5@n9<2VY8jf>!CYn+T)4K5 zYaa1>=iSO!jM~SwzqV$K8al?V*qfjw)2^{6N119n!95-;JKM$2=^hs&CH3_CwIA~0 zn2|3v_&smikr{UASFtE6ZK2@o_CN_;JVlRe#(=6!$cv%{0!`f=VBsS&;U`pX(>PyGE@Yq+(2*<)he(8l7FHatlo(XuXTsKwT%tq zOM-lWMWFzF+(fdegf*A3gN3Ygin&E*w@dJ!7Zqe@32TXM>0k!jK(l-kHG^bB69JsB z%Ps9*t+^E!?TkJ-D2ee@JynTY!eXP&BOO7rO3w2pgZVCN^&9(-G?Hg4)=LLA538A@^}!`?0u zeC>%?c9?v+DYCt0H;B^;Qp3>Gf6E0BIRXUtjTkNK5AuY=Q!ZgP+07vPMdtgV9_6rt zv9`XG`fX!f<`z&zXJXk=Mg5by=s%#$<}!3&sQ2YzcEMNm4#vFWPm#8Ta(T1n(xBCf`2rBTa0itLZs!iFi*uB0s;J$+FO zueF(vQ%0_j$3+RzPTW6A^&c8QjClk+Z?Gf;8#C2LaY)lIzkJpY+gaW`x0gj)gsr!< zainr91^+fQzjAfB5`u@l-F<-lQ@JHp`DqR#QMAiewumx*lnJGSwlF#es@Rt;Jm)F z3jgO(n^pvO3`L*i=X)vfEnx*ji*EFB$TS(6QJ*7TfN-9ZqW!AeC0=tp^GcooR#cu| z5oA+Q_B4WPY(oz3kzIfeIGWkFp&$>21tCv}h!7JN=nN*BFXrwu&o9?Z{@zM4-4!mw zPh_sPzV-WKu}jgZJeoVBDE)Lryl>StRT$qhSHZAbqj0##(2OBjW@w z>)9IK_DQ|d8o$en9dg~zQ?8BG58f;7?BGFRr7}vPWWB;7JIu2w$=yrxV10N;ZPzk) z^OIMRTm4@)t1o@eddZQi5OBUni`y?JmEI{ysyR(qZ?_o_-0sXKC?U%ooG?6ba3>mN zO8p{k|3=XKcSnsz6TW#oW>^JhS~eL;wwGDr5t~YRF~@v^Pm~;ZXGiwx{QOCeVUxGQ zT4s}{Sodu!LIP6i(Zl^UdyckQffCPaZE#y-r0a4?b22k_-)wj%0H+o;C$m4N__xj^ zwjVm$eRA?%YfNGcfi)ixN;Za(IBWuK)sTQv_MQ?^DuTlVGkY&hs?P*ytJ6!3e;T;J zU!uk{8G^sYgknLk7Iz+c$J$o^!(*Uo_UTQU^I zJi(s**FQ3$U-ndf69z_%9@taAV*e*MU5>Ey)z{G2JX*j42JGiF80TORuzq*{`+~lA z1T7T*$M~AHetXt+y#ermX7s>XgvLsi(m`~jDuks5kekcDh6vOR{`algi^$SIsdJuw zD=jPD7DSe6oz2HBQhURjPEyC^fURZ1f0_I_mfEn={33iHihXaT0K8YT-ckmsbiio$ zwBRwA4QsM#Dx=CgD8xHGb9-`jmBAEJMj-D{wRCk>x0l41;(@Bb^Fjv#^>JOB0)BX8 zPuRKo4RBt>sOBz>t)@T%|ZjL%7u;r`u%#fE&(rVC-@4SB_R!w5>- ze+@0^d;uf)Zbc3zcnQ@wN(ZIb$>z;?8;S`ps4;BroRDpw2x(v&GJjqL8D)3wn7DVF z;>C@-GpujLBt~Q$>BLd6s56%qW;Y)1t7{)-z0?uDr6POMSMof`e`)OK3gzclr_RNl z6H39eV#7>v#~vB;64a{91q1s8qgQV?Id zPdYOh0*>xG+q_UiC?^+O>7QhF);|ju@h9kuZO5#lj9hQmKwo#i!&;^?jI_7OK3zwcvjue4puy)d%VayILs# zQbiJt!h-+^3h(TfP&4FHayGm;8bQ=FBzAyl4w?RLR2QgId_0M|RQPbYREOcCd!r4%or!V}3z(LG|m6=Zu3r-3=tpYGPk_bKW{L zc^qX`M+A-znLv7GjQ2J%08bn52DoAgFn?8>Ub56bnek zGH0D};A9_2Nq;4B1G@mc<2@@9iKu=uHjil7Vhuuk_8AisfWVZ|xlk!r*sBjrSgQ__< z6jW}BHwga>)r!lw_UfKHG${uzq$P?{eT-2A;L^_&K}n=cPY&ZTajs0ezVJ{%ug@cTDbxJ(;bAJr0xIp8IjQ)8bxGg^Z?+9K+Xg5#&rk zB%{V2S8L(HFy7w$LY;j9D62DMPLhDTOW9^hiR0y%SwM0-@)@~ME)?oC&H!*J00}#+ zKR@)Ii!M>{^GoY>?Dq)-BAeT1vk;zr#UvpEh4A{(F};cT1eJKZ+1aUL_|1u_E713* z$$-aL8V>SY;rp;^otoa*cV=X7qb66ng#U^n>> zt3>PX6KTT^W+f`ju3z_N?B$DB<5_v0j9fFUAZkucwkWchC*VDg`j9qcpkzy20?N~+dPK%c%2=+hIX@I2T#*G zQFOfVYU>g6^Zjq*lT&0y(x>mqk@A%vEq^+G%w6=k=&|= z%KRJyh^J_==&@X{o%XppNB37~};Y+df|JiITIG-55FwWuBzl+Jtl zzID>$V&ibv+(Z_I`w^?Sv8Ga9QpICSe*ru1Opu7YgGI_OX!a8jBOAXsYp_4P8eaT7 z@~NfJ4F+)4_B-Pu_STKp1o)*s+uwOJi0kCfR}~4)(Uy5@fi&AIgI~ZYUVDe>M262W z#okXmS%s^YwUv6|PerQa&{&+c6!`a`;xvqAi>>=aD9UOdpF$Me_p zL>Kc9jOfO|MRUTF9+);Z+|DXUTx~}JvT3@q@ie=rVb-@?iyG(UtflJ+y@33h;QPg3 zGfZB_0soCd~-qrgF98K zNc@TRp7qT_i_(4{Le+_0N(>HfeXRQ-*;-n*-zhg=0*=DJFMbhm7&^b~jCroPX{ukY zi2k=gGKfDk@2f?!C9bzK;B7DV6hbYQKZvg}{Ng#E?~>~d>2s4Ks!=_Y73cS=uSR{S z$4q!_;){l^%Ep+FhV$$}oQu@77v*DmoMFz(Hb8)2hw{9uIxyueKexi(bWIM@Qkl9M z&vj?g4{rOfhKL&|Nv1$=v*I!Wj+p$C>D0KP6n1mt6So>3Lsh$X=K6vJ7y@>rGnuZs zBp*mARO5y?(KO(-6zwRw<$SHN)kP)kr$rhh6ig9~`cwJd@m17gJveyT)wsdj%l2?0 z-kaLTSx+^WpA<7ey#T_r@cY7y$Ct(ixf#uPs#}fykX^rI0y_m|+or!pvU>quy8lI( zz976<=vl>V@1^T)zg|xW%D6)g2eYQGcml26D|t z+yTn@)>&@e2@}Lx$(}tBz-(~xzz;%8$?bva+jE-b9`E@OY^@7LdyviC?e7pY8xMQ# zT+jr}U;Qed`magtTZNAF>dD#8;XpLhppit<9@4d9g%XyN@=jG-wRE?@EYHn#3hAwU;Tg4Q@;s&L=3{kYye8QZREZ}@O?@Sw2P3^CWt<%Ti2 z!(V@%aSy`ohfFl7G=(VN-C&6@>GuN4i*50K@f$S;woUix(xp zaw24@03vZ`W=7g?Aq%ip%9HP97Bv~Pn=#ixKjkX4;wj9|pIe#qi1wTalwnIN*}Oo` zrB2u6;ek+iy7&OP$@jMy6mUrMaWx!-7xOf~5L2FK--_yj z0wuj9fbM_rqojk25x0xp-NU`r4&*Oi$3qd!_it5LnrU5%j_E^ycV6pXK+n(o-g)5& zYjj2PD1F$iBe@GGja!~3o%H_n1Wt9hM-WVKR_o3Dx+9>FcH)|y;clRzXWRwQZZ3Y5 z^xuhvV}4(Qeo0`CQg>2i0@)Za?CWk%)FI%5Q!JYc;X!;ocNLWLXgZ~kGSQi(9JYIF za=nJn@|?L=xrFa4EKOV3nDJ6l1Bid3DzqJxzaP2$mZ{0S{0u&+jqcXU(~3mnWpy7& z3rmC=zHpPo%|uLZYr+bNa3ne~yIfOQ(~2}R9%Vq@re!w1raoWprgNeGx(&2TIdNygtcck3ll5MGy%OH1oZy*AM7r|a6C9lG&S_iqEM%?Fq-KF3JLoMA;}T@K zfiKYRqgD1K^qAYzMim6dv+~nXc8jT_$sH_5JP?fo>k`az$OzXCNjwSkUmt6oNtetF%fiR)r~5J9|gcx%7OM^(h> zph_^*XyK`7zMlStVZkt*#adO|Fw=+Y46sj+$b~`*_zuMy1k3aMBvj@FKR-_*ouSp$ z@i!(hQgv0<-WdtLt^Og`dUDnKaOcFE`o0*xJVA={fqw!4jBF+utG&d)w#LwGuua6G zb&fS-55a%OjVD%;^zdf(@U^ONvB^W4;1UA(YT0^~MHZUo zUR26>6`;M7m|E>}lN@$%@yqq#hPSf1b82xc14rJ@s9#v)fgFhD$70b^L{Ox%sTh}N z-+Ly*acF>Nqf3T`z)5Zm&@-t@-K{B^EyjK=B@AIRR;lA@C9ERX=4jF=5*-J5k!CfB z9|U7U6izp1?i%ZyIU7hfR(4)Ss1OcJRF!sF^8SZyuhl)w(y9^sQF&yp3VfyqxQvMorVSiyxVE9~*&j^o za|Duwp7cW)-!?OPP?%fY=-)%K3MZhr$ho-~C` zAC_x1KC@o_dFiF&p`mLM+x}|4%;i}<9kBW*cBc6+H#4e>>~@AVm1X_i8hJ7;%<`7U zfHMz>CZ2>b6$)BEj%N0Jrg(s9W5mR?Smo|sPz|(3M_>kX<{yZ;ZL=cfkFV z9+S<6NW-km-pGy^xEO+2?P_wv=_FG3DuqhE&YLqH zid;Q&$OVOs{|$Fhl}wxDk>imqypX+_8t2%o+-dx3& zw?s(lel7lACn_c;;SKcMMwj{P3_;NTF)lTCER!;H41s7)fD7fk57)+~w4dR!er43h zlV9GvqvjPyX3B^DJaqp0I}aQ>!?SfUbBKmbZS{CGG~s;G{d8=HLU(Kk0X%MQ8pLZo zKxh7i6hAuxAwinidYQP{LEd6VAk*h&?XYBwnE}{jm~JJ=F8feA4hN0jfC; z9bpfZy=8z%^%xfb;o4#23(6vP(n*7`+XR41(L%7XlT=P%uZrg3Bgk*=N&j5GzdyQ%!#=xh<@tJ3V-k0L-QH8jE1Y1uMDje zflU`s$dUFKo5?MXk?uZjl4$v&O+2?;^Q$CX%>r+o^qCprjNGutm+TY16qQn{8DLk4 zosmyHn(A6%s^3r5%X&@B=>fo>z$uC93*{dvb6KsYO9YO>dt7jCmRdNbSkZDM;QWSV z15$}9G*z(gtTv~8pI@xk7^2(pOqdgw!**VXlsqis#5=fhov4ceDH%g&g<9d`LiFJL zH{CgPK=yt}$EK+d1WkGW=y%&%tk_pAB;l4SxOwcfZyGeThRyf$n61VwYQcPh55%pK-w-4_{O!{Y~s@vvc{0fh2YPjwdA6vLs}Cs z$WA6Y&;&D(GZu9!%PuXUx;Tn6;VM3B2EUCq5L~vW0B{;mNSjH&cFh ze6Q+g0{Qgau5d^R#Q{Hl&W6Kh6UE_%p9i}rB{=GlMGIk{?3Ikq7Zq)02lEk+02cw( zY0Z~{R1ZK>sz+mOj07zCu5?){ZNx)d=DSt0(Du3iYYck`bqfgKd<04~yy=g+mImQF zq=1~%GVJpDmiY`%jWdkND*mn29v|(C#!F?kC3W7~x@Df|{HTto(PA26?%LOx={+kz z;alys^fh3=5=Q)ag~Ch1n|w94Lm~hpBS>n(iz%N7wWCP9WaZLT7jN~G(<*&B79J?q zSP4tJU5R=j#`9xs`x!QnJ}AKlkjQAX%h<~IZHy*5p~-3W2eI5RkoSRK%?@epa-rca zKW|_K_FNNl^?4}g1udo$8MRm_LAel_ixk&qmw76(`@Z6G3-+X2u- zhF9Lk?AMsst-NdymuSIpF;n69j(u&4A0J4kkge%JnX9z1wRw+rTw_w??!6i_@ z{uhkK$@Gf#SUnn2V+w95(a=P!5hE*Gy8#D7&6Ds)xJHEqv%_JFW7A9=wgj&|-Y$7O zj1MCFf1+UOU952Cit7qvaCMOo;wv8tWP`7(qXu1#M*R!~>L@*?4F*2Y+PEpwUkM*cib z-ynb!n(z*YlgIo^Ci|ZKy6c?R4!?X0!`y+|s2sa)nkqHSlsW3GLBy}}K zudMHb_%`){OXI@(Uq5nu2Gr=9d5j>52W8~2FzTm^US@~Kj1WE42y7FG0^%ct>SYU= zIZQ5^u97n`AO9MeupOGKvAH%`%T}P_&HuDs`YT`>f`ptQ;=@)`i7T?u;3LA%Lr9t_ zU4ko55Rd!d?BcvoR=hAX^KiNqS8#iO=V8rFr}c(e%QmAxMEHe_Vl|4<7Lc0~@wvY} z-_!kI(5RPP^__I>($33EKb|Nl@lhu6&&|ov3A_0rI89B>b~uh*p!PZ^<=g4@crmOd zxKh`COa8oL#4LMDr_&xHO`oL;A7mdY^XmolUdPs2#i;EwgWGZQ?jU&+QPs@#g?Eo$ z;IV>p+39e)0^<_96c-jB5k7HXD0*NA-~2TD*MfBFqK$^S2VeXcGwjU9^l3@mADd>ho7PbfwyD|n&ApiL z)f%Dp&gDo(GlsPK=lBc=%TgUg3yBgS7#-iIlmZ3^lg;S=2YQmdON-nhgI*_X<*p}= zvm6L1Tp_CP^cH+MMYTP zuQ9WLy)P~g=nKA>LFSY!Bz14CcPKTd>wChs8pKG68J$;0j{pTTVxE$uUT94p=xAQF z11|STOMSmq6A(L&{i{p7ug!m;Jiq=^y{!(`QT$cigD5B-QwcPWHZf~+PU;1HZnVx0 z%ue!!%tri2AV%D}H%&B{3x4>bH=e->9=B?x`&YmpE_P)FPnz17457-=(LWDC4mMk+ z#+PsOxfl$#!J?cigHV!CP@nR1amh)K3W6mCw}%Z?pqu+kUpiY6n+rD!9=hSKp25Q8 zs4C_f8Zfa)HrCg1w3(S!7 zY_xM`m-B>1i`j}~D=0cGM!Xr1z^ST8)y!CM+e3ElEQ*~vszB9G(F_(wUzN&UCYf#; zL7E1l71=|K8v23k>BHc&h4oDqi{LRUSn{La9xuZ|#P<5i-I`x3=l|by<>hx={hWlIEF7ki zgse?~L_Qf!(qaC*ed>4-@y$p1B;)GkaxJ+Uqq?mT2bdc>_zz_XARB*rCPGc}=&gel zNLe;ZU4_k2kxm_&Q;9yDU+{3W!a${|vH;BE-L?m}Z^NN)H5}vCQ_)8M5L+8N47(oU z=tf_Eefvjlh;8qhJg-!BCqxx|{Uu+S*cNg$mvHmbD;lXDj{L<_Qn)gn|Lke$=JgLl zU##eADflGcVi|;b`@{Pr@t^staIoNsY??&2()PJO>6)c?#c?^OKBoHh5ob!XgGN~OTl@NYh`AbhYO<`%s8mQ|4qn#!Kz3Y5J2A zn=LZHWx$#ic)aw@;nl8uZSX&UqOvx#X$vbMsJoW9YUtySoU-Ut_73X}w;DmidUk+5 z@hdW~H5>x{>><~m?l92$bhGUNtgqo33(v^u@2mXzBzHDg({e^du|6suC3PAuBoW6^L@;IF^s4M6#eoJ5`D-zoa>A|>mc zc;>ZDh?pTDNFKKqb%h9is-($@HlOqUBr|9_TI(CcGKzd|AQo#3pH{~`lRfFDBKtQ9 z+V0{XZx9EV>gCr=jqKP};4urMG5-vREUNn|f8{7Ij?aoe!NAA~)MiGItzb@6^>J}g z96Q+B!%OlsfxLU2`7p@N(&&1GFdxmuQN#buH;K5xmXU z2vhXtfKKN}(hiQsi1R%ZK)j^(OjVbX4vxNnz81GeVZE^JdfP=I$m|2@J2;*jI*~>a zPOv0BDoLuf=*jlS;Q9=4+s4m9zX(}RnT9UFO1~m!k@Jr3LiyQdak;FX1f@a3!4BUB zV>S=Y_HZJYbfBD9jCk1r`jWqj3Yr{a$tCy68BJ9bZ0W31cX4RC8i;2;q@-B>^Dv$p zAfmiir72$p1zK$yM{{5Bg4=5mT``u16y7QxgjI3VKZ)vIJ?`{(B)T{E=HLXUd#=;m zGm-*%^?A#>+?^^gk4CAbb?=NbpyM_khw-~eHkt~uFHP|Q%!stGbU(wdB|Cp*#b3tX z6J1s&4bSibul5<@@K=ARYFBYt2|;Q3LgITGZx<)$7$4&y;%G|Ix@RM3C3e=&*8~K< zAEGSFt^s`&lgbPRyRK81vZTXI=w?CMdWnh20VR-mBgUZi92r&Fdcb(mw2gq8*wEZN znymQ>qZ2kIvQc`9n$mlTLjYEhnS^wfHbnj2jF6{w{P2}XYwTJ|{`4jeBd=DbS6*7g zd+wTYR^E6Wu!(HP>UOr?)3XYkno2rC?m9>!oU*{=wX= z>+1o^c7Mm32>N8Zkd$|FzAqVXS^eZveeo-U?-MRYf7Ui zEyD-*K$ak5-|Q1QF)l%6%WnxFfZIt!4|xX1xs6|w2(@~}z(Effzh{igV+deYHr6l3 zT{osU^mMSWukb*;!_`it2a)4K*>TO0_i$Ic^{~K zTl}S>hS$!@{ZU8f-J(Myn@n374C-7flH4Y&lTm2}h(j7SJRM8YDZV@=r!BLzmYO)T z5h8qJJEV1~|JruuI~E00B>A1|WNNIX%fXiNSj+keS(w_B=Fg8*WBh(8Px8N^*S$uG zd7sduKoVkTv%9}Tmf9h#1H}7#^3>_mKNkYhch5f1rx>j|@9A%nSPHU+&&ucPF&JHF z#gI(tU>wZZ?30V0>pwe#9Bi@`)#Z=1Si&XBW4pYlkE1JJy&lW}*vUyUYC(1AoI78l z`UH8>?zJtKY-&c`tKXgc647r=RML#NrOS>t8-tk#2b4g@fUD9$;%}F!5asrWrz5374 z3|T7Le1s~^)E%LMl;42K04p|g6gfAWT zzuFX4UxCEL7#Q+bd&~$DXa<>7vEeF6%3;YVPMJXgKD2QKxC*lm$PHJgt>Vjsh_9rkgu`uMrUI%`~B^i$Z z&&|!?Wk|h|Dx?peo70b{1&gUUetIs|sHx)k{#}R_CpdyE#Yzz9Xi9xoqkH zq=X#kj}%tBC=yY9h4Mf?Ob&R|R9bKzDhd!vkN^`YX@I9oB$mudgTG7<_L#Y8R50_>PleCAKz zbW{dP#y^X`85sXBR+$IO(64j(S!M_4@~gFUB|_xSQr}lBKeU9ZuE;u1*HxTAp3T>5 zk^VaHS5xNyo+=|`gfDQ^YKzJoN}gR0`*Q;L1MDpCUNN`~^Sj^Q_9n6C5ESeJ?%Q55 z-8Lz7df_ug|QA@X==PIox zDpx@1#kXpDYP}^Z4-eF}GQ%eN!ORB|o_>tAhl8>Q8dUXY^QTv`|(>VcoSY<_nqjp^kU}RLGJs(}1duUgB4j)yAo< z@jq{giBF*?Lf6%k?d=*da3}{vLc4|mF^79^>T(}oftCXdC0Y1R6+%mi?^;kZ=D#+O zADY&px>KPUB%kL|0{SnS^3CP(#uiXn;v0aD#KpT(QGw9Nw7588r1iTi1q@JBg7G~Q z5wK<*L^;Dz`|Ke&XD-X#l?d6nzGW<*{(7M3JBTm^>mR2TaM}w3mCcxMIiGr~0ljbq z%brmD3R*t@@58bL=l_*j{KX5<6YwQK>rak>5aj#n>Y8-Wt7@kAPyC_-DgflF6kOT? z{2xE9X062bGaWx8O{zoY5RX405TLiUxZhCDmx0Jc9p);{lI4~{=u|%uABEul2NxAy z=f^CRo?{ddiXWa~32^ta<^^PiGCn<~1G@}v-d>IfN7ln^!4$W1Jv!+M1H3?$(!lLp#iRQ4 z)ohGl#%}EE7v#6{C*RiOV7rao?cGpBGOYUR61PJzbLYm+#PGB|MLh-=aF#TKKJrnV zGIZf>B`e{rHGHyks8i#q*L6+J7uxn%VRagQ6~|eh9<<^uRU&rscyO)TqO`0to0ocJ}l0oXDs47d5G@7tkYc_Vb1725B~4yMR1J%DVUR zWUKFVo#k_tCn|@9lS?3;R^31?-4asBWR`=m0{&UTZoQH6Xe-^y73lr*l*=;;9SQ>8 z-&aC2Bl7D{k^BPF&E>Fy)EBjCBO{!ZT#gP^dTy4o;2o z*pqy6?Bczt|0~WpN=k&B!8!!~^!nS~QfKB5?#BnmznD&beSvEr&^X*ina|^F4%?1O z#ibTjSQdDw&@_YFVfQ9H00tMv=>n*ZqC7a9Q+A3+Ge|}!j?M9DgaI5@j(13OxCFN~JS9hf*|#WnNob){z931$%)0stuXlLzoQd}8unB9z!Amd6z| zhjhH=cDZnlR9q@}=W3*PA`kTFz`RG^7m2LBWK$|1O(x1y?F8(Y3xSpCj@b89a=w1M z_r27MJD!**g?h&eLaO|K_8lLPfVbE(d*_Dn7>ukexeaFuC%O%{iX<&F*mr`DC+ocC zhX2O>L{ksw)LYB7ft0xJ44+5S}|Rhm9!v^V#6ZAbe;k-1s0?@=aw;iL3m?os6J%TOrj zvz_-i)MKv#$dDYqG5>yyR?AeUtHxCs z#hwr^1%QW01VafNEdJ(eayTY47Le|zuP#yr1kFBJdR@hrjqfpjUl>!-b9)#`*ja7) zY8(fjx3M}=7Hm9(>N+ zk8vT?YyEAPH77q`WNCQJYIkoALAgB{NdUKj@PNxUWoqBkzB}FUc@l;I1Cf2DAzrf; zq6swNXIb0d{3z>NiSyV0@BC2uBu zv=vmmt!u$+*h3p!9G3ECt|kXNuD$VmdPQYHEkIMpP12vNV>rC$L- zQ4iO|>-O|%x0|x9Gg@D-%r~};$79o6s=7SW?r$Q}eQivrSq^-%^yAtbgBXxPts?9y z4rjSI8Qj~B#bh+}cUlKX-T91n8*Ut3UASJ*i3{MWXDoO_oFeF5>4VK-RVSaI3a^VL z{>DuMWzy1lo0K4w^mSbQ<(j@d?ROuJd+x3WUR~e5doRD$zBncbhPV!WJKN6W19JT6 z=o2Dmb2F;pzw}M>f9ad1&Hxe^Wb8mo@VZcwx0T?*rgu^2OOJB%JX3Ua*gC>9r|5zR zuM--QYxf3;ysq4RzIT|nnI0*#lK&vWB1jw=>)>+TN!6S6c*uVqc$8H{XhO*xwgPOyS@A)8k_a0e*Z< z|7)t^$C7^9$O`@4{Ykw+>B(`-rT%qRWgoplxq)Wik9Z{8#Cb%O;Rz&V4krEg<+9!y zp~s8c(#?@{mvl&X!=$9UQECFxDcxP?MZdMyx7Kg% zefG7lbN*7N^MZH0qn~j<_qbK{7G0%7-*)!4T)Of=z`>2jb6o^dJCEx)G1*Gwp?%pi z^_|hz;nY(C=joq(rrRSg7cBiVVHiT??kD6XCAgucn~}qUu}f}`SRj|?sTO6IbEH4s zOTBnKkbsWH-&U;rGjW(SwduB{#cAi~w^|!XbC9jf-0@dt3VfpSBkV zEV<-bI3^B{WoQZxE{|Pp3%Gf6^iW5GzV@nj%a)>7wey1j!t1QXXrE_T#`qOw6v)e! zaro?!GEXFz-yP^`bT(W-a7I@%{c#Gdkzy8~Tj=PueYS2jw*fGHh5voch*jcObH-lV zoK691@#c~xSGkiQ)kuyp|BzGE>rjT|8?Phz*(L%EB&7dfiAQP@H(fZJ!!iy8(lnb& z!Fiz1#VePyQx;zf)W0$xwETRE+929;ox)HIZ(xIPTc8YvZCb<%*Qa-v3JoM+AMQVq z#h06H(*FiEopf004IUN#keUhVkhOYGzR%Z57G={ndC)VQ!@Ep;zyl|PL$427Z%=mx zZytVh-gnZTa$DruM7AqsD)9XxO|~d=XRr@De*L*BC>>a$ezQ|Fu46;BiBc6S#f@6Z zs71lt3w=%)(y7gO_!|3}|3@w$H#-P6ikc7wI^C=c@H4&}060{uw3K^4i6e8&f)vf4 z1>j(wuAqh29I0dV`Xz_Ndeb-dhF|8yO29c|6GT!-W^f`oj?3yhL+zEl+LA2pTf>K~ z8MB^|nNmA=&sci&l{~-9<`jm|!_to(Hj7~jfqQVi1UqZb&eVFIg*@{N1;{th{Wz32 zyY(m#@Qv5ny*XuZhr$$4oa@w__X;bt)Ha)oUM}8z@%+U(mP3RvqP(3z!0VHMTWTp5 zFR#)4FQ4c@J&9NRzBs4=EL1$qxXcJ*E^qm`U-)b0K_liDFr46?zxHKJ^En`08q>O6 zrI>GesM)eQ!!Sp%GjBx#_aGR-Pc3bzjGLAH2ob(uDNq!28qgb3B`5i#PXY zx^Rx-b${Oh0UXeuwcsB_HCGj?zs+<1O|85vaBNLxG822EQ?+E{=kk&-;RMVOsD6Rj zTL( zn*^D}kkG{^UKOkGFJ4u{l?w=Km6_a_f77h|i2pA%D;{OD z)BkrgYnA`FfNeO+wYJO*a9nGg)^*xvwB=Vg*#G23%{b;wPIimT@czbDT`7ak%mNn> ze8#|YZf3T;J}F0E9!8$MT4?=nP@(r0HYB^*gY{TdSk=gsQSQ6&X@hbtKnf{GbtVB> zTsP+-;8!2|6^Fi238b~%Eh#<@W-;6)!Sxi3zUIE3zhN<;f2CX_q&4tahL5&sqIw$0 z`(AB1Z=ofF=Jkof*}nSn_a$$Q+MgX10Et!Tw_znF1;Y9PP)I>$WuU+I1bdqtIXNqR zjHaT4maI0-#Y__cc4At}N7b_IEZcq%{!8_g6!>Y`wpq&%I)AZ_^j{D)W>9>9c8rXO zR&je7j)W<|n+m`JC@akS_}Z@`9uj^4_&}Ev3w-d^w#PSd9bLn67{jaYiEMr%mhN|+ z^~z^?q`R~vrvaYfj&62c`0Cg1JOf3-q)=}sw3&OLxBKQIGasp-7ijLzjMx1_y`hu9@e7N_+j*~5LVWg_ z1-M8Y8MYyQy5bL-`}-nsJ>Vkon+tD%neaxg8zwy-RT}qMWr(;w{@yUIe))xpeN3BV zTf>cYPK^H)?Cl!@ru0!mqo;Wp6YX=@-ApHuuYSBouhi|>M)ERlPG&_32Yn^(1mnsi zeY~x|ptbbw$2YF8+wh5Wt+gPUDO%c(7G?NWo)2vAam6Kj19~p6{lO_Reun0)v(y@x z-i+R>TXSO>y|e~SGl1@;qv~^69v`4yu_k0q-I}J2X=MLO7I-Qc1AZj0v zwhofyRg#z$S!QHMeuu=!h4rS6|M0G7$nn0v3a}K7*m+Hm=c;^zsv?NKg2x7!%=m2V z;|?z1W^wKURwWUJeYL#qr@Gq(P6$>J3-B4x`6u>c82g8v3?0<~E|L)8LV6tFA~C5s z$RNKtv?Qi1GVbYB38iA+K}V8GIvio~2X2EqVOpE(7PW9`No!OKP`@bQa=&Ru_>V?(w#-Or0rWPfbrs;Nd{oK z1NKq<(@LJ+OcAe()S8n|{ROFfcv zuZr`m`KNW3XGVWw;9GBethTw5IpFtB#rS|<{$Q@+Fc`W7F3T;(12ApxUDG&m@4jn%0yF*gn@Z5FrOaoPRH8rPKG)=jiV=&EI#3}PKVZ(5X_c`TfY`lHuDedSF@R$ zBJ;VUe>o)i1=kWTnhgqZ)vyRtsRAIP@s+-)$xmxcr^IVWcrSP42rW>DC#fk(4QbQ! zQ*x(3Y;+)VZy%s}U3TAPc_7f$s4v*nek?9DpdJDoeBcl^Hhg=NA1|)Ldd&?neSg;? z#8A37ad@|Mw{P7WKzI&2UY0TDdS)sX{6wSy1KsU71iTuOGV3mCl7lG2jaJu3IUTz9 zVza=$tx@~}gRrsok~A5`Pd#?L?I&(e^B8$4a#OsHmI6oKd+}t?jyJ=_g*g$Ry!)Cr zc=cb@SHB27|AZ(a{09MEyeG{=v72EJ*_@gdme*^Ko;5Ku$?=w%S?&mTYD8->iZO1;bv}av zxd15x{cjV1Z44v}SX>RaG?n}DcMGyh`xgKF4iN*AoT%Q{#I6OAtDexP090@lOZOo%JPy#*3cK5Zy!WD3Ye4U+T=D-8 zqKo%=pguw%*U+R4V&aqed1Pfo9@QT%1H7rBH$k!m>?2`55=v}oXRy$neSeeRsg1X| zx$SSTu;M`R_o>JzAF5`WBkPMs`BG*?d9|L$3I0jjep(O}tYc$k&B8TIJ>@X&nVJqo zvz)hi**PsxwXJGy$K$_mF@F&17HrPz_!2<0JJ;lLukxoi1a3Ydx`$r?L>HJhcA0i% zfTimWbFRnbzQcpK;hU2~j4lAK6GG`Eboaq@4)*T(JW}4gL>0MxzLsRcQR?CVVT?NG z?n~))br$ud$?RxvPwbgO$7tcIeHVF$0DuX@WKA72trw_bYNtnRBdB%2{pN}{w~?LN zF&uG=ytO}L#_O_Ips(%r4phn+3Z^v~#hh@3SHCRs%Fd|`?J3pc;b?ybp(9UQZ9U8x zm7Q>eNImm?#wvtzxPV*V3)wO#ijttdA_{dR$E$VsI|hFSyxUD4KXB_m3Vh)7N!!E` z&1%HGL7Q+F0LG}r-VoE8f@*ct(|f`nO-XgfQn@VD5Z*;QleQP1LalkZ>0D2 z#0aju;M{BZYIRK2;~DbJvdmi9pNGS}$MuWmnkAwnWfqNazCDG7mYqv1(M5h#o$ais zpJ^bqCUG>i$_iY{>bHr5>jzGWp4GMDN*pWli8lp zb&ay=mJ77&s%}nUVKarB@i)qYTmbL{d#Zn7=j9TAr($Z8$CehVtQb_a&cKBJ2kpIF z>jvWQXL5xJU;xCx)WCu+{PsY05vg05QyU)k?<$k zgs~K7^#}#lPZHR=n#GIhzvU4bS@U#iZya{>zo7w>Gjel;1Op~H#!qBDYSx++Hes=S z;w$QY2Q zXvxLS_L?H^pq-!F1}HD~qfOray(33!0B8Nq=_ z7Qcg(wY^SIJatJl(7Sd<%U&UnZoDxLlWnh@Pme=ub-RK$w9E=*9F|2Jh`g4cFENPNaj{&WzB4# zFTSmKx?25a4y!{Nsds|Az^)vNuCS%}3pssWCf$E%PyOSu${oB0*MT#_r?KP>FqZID zmDUX5uSRT>0^DwluP=T#F1jWt|CT4_<(s@)u#iv_Ek3AS35ZC%gI$e&cbUqUHjd~K zBD(N_Z#6FbeQ@$eWCh7vvJq>+#DVxqz|!`<6-fB93F%g|CXP}Z1&^$pHu%?G$JC|J zY0Y@OWbpcF8Ab@Kk-g0l2M2x;re+!8scI}s+2XBC5DnTyM=f=6rj^@mGow=)h8GSykg{aV>@K)o z{CwNDlUPyevRj`$x{cG_fbx|cgiEbfR-dp)2D_DVp}A20Z%U?*M=E|=yUFhcpERfX z>KYlJdhH)pH?m_LMO7v^oC?#zztqYbmuZwsDgox%Kd`O{oHTdg!ZEP{8@9;z>gmxA z%ci+`wnC=Ivovp6L!NKfVvWUuzzy8E02o`VvGE+pKqEya0PbL~OvH%vb?Bwx2l8p7 z0ZjHEL(T5uH({_PLd>Y@yzTb~ATuEjs19IkA%+0wRM{j5{`K3I7{FkEg1&Qk5Fj=8 zG3Z@LyE-=L*7%LPu<1XDUAtF*5xX)G|1_WS{>yxN`!Dk;@&CE`)bqFb)EHS|@xvLH z@x+|(jFQ|Hv|a6T2as8MC-F=zuL7`Ye!ZypaEyr;zpI&KCvGRuZw2H)&PFAGR~f3Y zYZddtt+z!C8=3!21QZwtIIQm_=wsx$enYl-)Y(reQ%9^3ZHma+VQUXG=_Q827#(sx zZvECkj77m7v(;i^bVdlW-A@=$kJKPe8R8xFZ_>5R=gA|A)cQA->BZkwrf7TO5>Kct zL2f>Q`$;Uy-r=G(*EVJF;UdtT%}>i^;a&`f%GI7Em#vH)EXDQr$CQo@ zW5I)P_?B$#8ZqH8Ci)*Pw35TtfE4AKY*c|GB(6T#r3X`ELo1il4dEh?omNcz0KzG~ zXz}bm9w$rb&5t~rJr{68MXM9AJty_--6~{W@*&h4 zKX!Kcw|<@O{GcV4MbHQWXR}{SiETood2KHXMj%F^zt0-ac{@G#eU`t`m7EN};1|{t z4{YrwNq=tpH~l3IsE0UV<~1n6zdq^n zHf46Njupkk-8jle_vmpZP6JL!zLH{;5C3beWmF1i$%U0UDU72+oV&kZM3>U}Jk6j~f#4$IIf0KBtH(+z%BnFlQl z7e)>%xf;iU{l#EicAnt|A+2O+@?vTL(x8^}NctJYLeqVtk|H;Ivj~_~%mv{l^Dd5%|JxnuIkNNZwr}_`Hdx3xpH`Q&73_qSd2-GdrI!iyJ)d1jmBppyE&o!hHPonyh`%9e4YiQzXiGUy7iJXbuSk%D?2ISddTBVuRoQ z=@wnpIsD5l$^rV{xJ6-`W9H`o*DsG*(hq_US{Zgm*Mj4S$kgOCLLUpB<0EM!__|v0 z_4)t{WF8r51u5O+YD~;GpVM>4t550Tpp+@ycYnx}*yzB0?<}BOU6a@17d^7cde6Kr^*Lt+UEeZ|lOP=-~GRBkjtLL8)@I$wW7 zuLSZaHysr11XLDzOU8ugb&QqTGQLf5D!6$d$Hzh2W&6XpOTzQU(_J5%s{)CL^?w-b z$g|zudumGbU$U_IY0m>QCs3V&oT1iK-*O*mKuO51iHa>UDL3zrfq`BA zKvxD%HFRgWQ`Y;Fumbq1)4^9lzPARFaAKdh*>Sf%JLi=Y5H6QY5}V}T07s4@4)wmm z+enF+GO0Qa5j#V{HK%XK+5%hZQHt+Uvvo;@|7{wCD|ng)M`-^8ibYlivQIYWufxo5 zT3)ztyjr<{1Q!uFAj?2(B_5B~6wL{PK_q|*rBp~_Qgx~xaA4SZ&tnK}?#4>oS?TGx z0dcIoxDEOeb?&^AsZ`Mq$zT+O{}hb!=0Q2`7=viN373v4UsF%qv`)R~tvTP}>7tF! zPu`rF-uF>Kd?BVYMweM7x?{+(I1K6dD!_2~+UVRU4Rfbs?P0vwlXtAlL!(10jvSX@ z#uDkM*u=Ct&khq=qi8<&LYpp0Dlgqc9%QF@$&#cYY%!c(7Ct#i>vbPzow#lg zeciTbAMtrtEdlBMluIA=x7ME`D%)%Vn)wJHu0njPdw{nlwzef>)M+X#;057P) zM38W5JiW6EgRyxt1F|;WXrKEi81L_4)nwvi+S*j=@_63{MDbksH#$Xtk&vq&w_RTq z@DWc(4)*qve@$QOLj5oH$&dERep(<+_5u5J)xgT>`Rveb@JnJ9&{*ZE5>Jc0`zvl- zK(5A;fqv{{Z3K9CO??JPQf(U>U!w3l`v$XFd8B_nyy-+dktzikB%pT_`BBW_iM4R{ z0{?bi^n?k27J-KY-QR5tv9#T>w^foWyjcl}x z^kh>ps-*Hj`HzY?Bp;DFikl`(r=Q#(ge!di;~W2`g(84=rZ5e}d3usiE_NO;_qS`9zJR`a^Mvqg zzPBW>fDHyLl`}yyATB%N7`DA^VJsUAkctZ&e8R1@Q90${pDCZdTWw5wGYe>dnbQHi zB=!)ZGvcRw$Vj1u&#=Xbd`hza24CxrwLasJI7&}2FSQ23G<)}Prm$#+*qC<;-Jza* z+|cYZSa_>B69?Zu*J|3Yjortl)PiW6xCtu13>5Hy@IUS1EVA(c3t*-rJ#TsTgpT&O z#AGxDo}E6CqyG|SP7A=~E#?QrkHgwk zQ1?ym0R`s^+L&zL>tw)JN&iRlJRfGCt@zYXu%N9ZySq%w7d z5lF!n12dq1yX9KoVOM`9oUAli?fkf^t5+=1gP#J`pP>LM6~nAmZkLgK zJ|3!uog~>TL=o;e+PL37PYsWVn?Z)Ar)L{$8^wKgyafv)^{)fOyu4f!tX?$x{i_gP z@qM)LpMR}c(z5=vp8t(Kb60iLW8)?QP%;(~%=&@{LjYg)hW2DpS?4#gX@f`JZ~ES5 zT$^5Y`=naF4;&q(ypwtRiIY0q`DRu zdjZoXCvv9>P!tb@Ul(7nK7C#A_9&>Mp4b~ZXOmbdRkl5v~Earh@dLadG1)bD5B@6I$JKY9( zR+W9&lN{l!%rb*360L=Z0IBDoM1t^o7%mXrQraw4co_|5P)6KLR^>b2p7ebTzTrgs z$1)#N73G&$KN9`fZvfQNs7P1&ooX>ramvPEd2@H29|`5CiD@1ES;)W58X6!-Dz9G#-gQf z7#T)&vk%>55=MKtUVSxx7>@ng?Dmqa+L(2RrRKZ$E~(accu=R9{@r2d#&s!-=;cB4ys= zz~wggPY?aEdF4Y0K&4L~`E>r}5)Lf+(?0-jYL3aO@#=GD`uBCaaA^Po>V%2{uL7Uj z;(XpTp9W(F1U7{AnwbnjF@OHu56U_i-(ih;%0JDg!W@=+%i2DXX%g&Ag$na5xP zk(J%23T=AQ=mV%CjLNmf4A}m>~X4>A)-k9#vk>X^qmHOoaw4$Ww)WD-9 zkod$bOq7)Vw%p7Sedk2{mQUv!|MU&G(T8oa?QyN`sJfG}q?wIxcH{hqqk0LE!y}!S zJ$4020I7>v?X~>Q4X4)ot>jt(12gO1Lt^23qT=w6|5yx)$^DtVfL6}6{57#$H-}tJ zqFnjyx-l~oJsxlx#$Ke0FHItXY`CE=3uYApcWK{dys~Y+k`!V7KOjOEN1V$3MTBZ# zmSz0#kPuLY7_U2o&^PHZJ{|Yq(u-`wbg4#tB-UCv_;EX+MJa2+@aaAH)>c!%LEQnoKAyd%8lH6h^BKhcw6t$v-b%545~rR%LBFcM9K33avMtSq-w*&~2_tXQ=JItp zC3Aa?!^>i7ss7Sabuu1OLC_O*leT08()m7nm3=5jllAuGZrW0|6lBL@X9BEeirs^P*nG{{14ESKq7a z!e0jj=liXM&=U`v6B_k)ZW>eDJ7HV$3p;)@l_M8hVE;?zM4l^~CQ1MVbAL~OtMRlL<`=XJOg5b4mm}CXilPa55IIOV+jLuB9Muia- zmv##+Z-&3W*G1?y9VX?hT^kps;pOgK^0OFqz9W(~h>gaSMxugy4nsx!8q87}okSN%DGh!Aesm1-|xc$fS+H_;N=EbCYRE`^UHQn1*MI!5dlZu=7Wk|=|0g2PXk_3dJ z?W<}q;zha$eYthKyQHh715EZcNU@7h-4t62nV&WJ|yL0#(wW5 zZEo_u?YTdE9C?G4rn_5ir%*-o{MuIV@p`;$3qtnz*(7}CS<}>(*2);j^5I>rAA5`I zCq>B-hNtfK1Yb_Tw=87MHxVW)Vic)~6$>z-QDm`l54t7q2a4;X z1$%y)RK_NE-Sx~YH;9G6V!GY2;$W#-z!uzga1+&B_fa8~>$XL7_))(};PL6i zZ#fT-%fw<+r(2tQxBt5teC`g_Wvu&Y0^W_;3w;3-iwe0JDPok4Ie%RKl+?_Czy2VI zWl)E={kl(YnbO`G`=@CA9CVHc={BEh{*1UUu4+#HfDV6D#%^-elwO zr05&ToYdn-5hl0N?&l|@nzHGLNDU%nF`ntq%GDBJJ!+1gp$6ip^x}%C!m3GJL`865 z@bLn-$Uc<)u-u#?5$@KZcnv4yf9_C@bH39$KR@5S^AN^@fU^=X1+Fm>;zRt z%IAJ^?LM8^6~cVzPk-anI-`;XLa+3=PGIm4Sh+ezbqMO)yZVWIK^pPdW=jXEgRKZs z->nM@Z>uJO$>42#1m?)$$=HANe&WA+e<zTN7H&I9q`UUn3<#g5Lxo00ciYSjDqOD!|d_AHd$r#{y&? z#ZV+SOuM%znC$i9m+sAeU){Eo*Cp^=+XZ{hS^w(E^Od(UZAT8kofLTNKMtEWBd4{I z>|o|yW#TsN3DqsxgJ}p!A)S$BC)#P+6LLndzMc~}(M~oQd&`&CzCaRQiPrsXeED)c z0Ag}(&9A^Ib2}RMwyC87%~h2L`g=y9m1ZHwiJoYtjAA7;ADg4$TlmJlPmct3a+QPv z7<728%#qp^!&Ket(!oroP;*X2qC49xlj+SIF?q=5_@*&NGdD4O_MW4$sBD@9+Wvaa zNDI-A`efh4TpFg4S5^A_z;kq)PJ4V~>K}dQsqx@Z_8lo9-T^*uS-1!74xFg?O=fn& z;7Ez8yPaYU*E$(`*XLq|3K6x18+ENwLJRdi4q~O;l-c+;mmDrrhr6A%1Go zV<>0LEByJ$RST4yFf}A)%A7LbYuvkUB99lfm%1D|&xj&K8O7w~cg~sFAcT7?3&4P} z{$NKJAtNS6PVqDicsC$@K`z^It})f!ZN64Je>-wT9s~yGcny4Kr>UrOm4AT=Wfn?B zy;xja1eQ^q#uRyt;|AvUo@a*E-Ax8v231rNC`C)Y&S<7iHF7HBE5eP33%7aJ6i?0g zL0RYCgL#C^AJXX|qRsV-guVTBM(b2PIjH=61*46Kn$sbX`%utv6o0q*Y(3u*vdh_P z8{RRPfzF)q(AyoKbs4a%eXeh>evpS}_>$if5uGf6#i?7%`bgJ+u(VNqFMW#0?c_yJ zBQA|euTF9xFwk+#@|tbcQP*7GlbET7ok0fN-C^f`0mv17CV6T2w-gZkCqE}0XT*2* z-WINV=lq8~ySCA5*W$juS&;(mjh}o~$<7~H5E#YfDu+5?Ek$TbO}}gkwl}@3imp*g zQn^MpLt+z%A`MIQjZjW#7)qLVY0}htSJk&<5bK__bY%v;j$cg{B<<{2#@mZ9{A^9T z=AgsdA$WFP>p>UPexocmeUFM7mMbJ%yUgN~Ts@@HZJ1|bGNvWf?WnSr6=9!8jcIJG z{8Lx9oU7O1hV*E7W#*sbxtJdkk4!(iB@+Q;ZhTGwAs%K2oV}3wM)d2m+w_Pa4+e&k zh!wJXGbo7X;RZ`*K4o5~m(*(OJ07-YpE^o5j?VG5moy15AjFzom6&s_wr zgAFq}K$fGY-M^TQ&f;)JQs&X;nEY{QA=y8&i1lM;bOfwk>M} zi2{rUbwN1k(JfGV7Xt60m#%(&3h7#NMrf$1@imF$2^%lfv@D<6qQ&)v>}7a8`zpz5 zdZVh!=T7Uj`Mmxpj6wNzvhT0jmAjl*M$t$5%nFZHx68O*+kP6l5kN)t+*bDL>Bfq; zqal3CzkIIoF$3snp-(+c8kXINI)?{t_r1zNnilF){dhSy zzg1*pUuy3CD0-on6V$8Nh5jRPhoy!&r(YVu6~AuvcsV65ZSomOu+|6NlH%7Ix#wbsv8Y4E*sCvFBe^6 zM&xCLgi=CzgJ%`wdtSAFYnN5qB|8sYG8(cAKc)q%Gg<&e7W+2;ZAPGV%fFV_aY--_CoVdyL3+)7dy z1PX>C+?^AU@IGv9b<;64$ZC?3wOC%^yC$=Y8V8KsSb?}8bHdHqOS0A2(8P#i{(*hA zJcSH`p-W-MR%NyY;V1B7Fv}H6VIodtI#GPZa7E z<9$p04eV#FkGdceIij@@srF>8;3%WS>(#xs);Vp*vck=*pK-cK(Bls71^Bd#xH~la z0+(Ce%&%?0$08*3G~OPN8lcl%>+`;sk91KNiJyIMuc^=Y>GldWV)s(f6YjLVQqOA& zQJGNE49~Sy1woV_l^220gkdgiYGe7MG1Mn7W9poKFT*b@(pf|J8B$R$>ZWD`Z@=3H zp+&p;`tk>FvZ}u(bG@_oL%zmF$mco(>MMyi95m8|2#n0%k(U2aTlrBhECM2WddyhW z(aYj+1Yt`*XZoIohRfJE4Kz&S`*Q6;j}Z$B3_Pt=e^(AB++d)(OOqCRxmrz7G{}mG zh5%y5$XM1?s@DRGP&PP};WJ5kwf2tLyETPlEa^I@ZFmq|KHHo+86ERDKnoRjj8=|o zrld88Q?OFm_QJjyXKShUG_%!g$VeA77Q1nw`X2i9WiiIr#(e(wqu5(0+d7nqexMP_ z3Y*4%4s%^#VBRS^yWi5*u$%ATxx^}8yiMIV7m;h_34<2eEqO}~^wxnU!2t!hO^DB6 zZT7_FBJYpIgQ(bIx;A=)JBPWL1!!h+@qV=BdAbEDr!u{v%^jUD9iwlDeP;!QJ!3J4 zWCe_^X?!hllhpX~;4?p_4RpAi&c=yRsi#Q=8YTuHirL|-<4Z5X_7xID;xvV?co_Er zcgtRf!EWOt2RT+Id%w3YyY5BMMfyFtG~3f`LhFL?5%DX1ji^QyR-Yg8O-a;z2sA8J zw=Kz6CWG#C`H_*7)J@gFJpsk+-fJ4(rmgqi`vgZ1*dHsMzs+jk$T3Skrqaqj_p8rE zPj;PxX-lj+ZkC&2p{Hnie}J3@e$Lg_bj1g+6N~{2{8{Ag; z;?DJ0QU7ob)N;&&x-7Iil=o4j{QM?|%=y;2zul&%sqBHwV)e!ue^Z!N-#5YTjAyC9 zZi?Ts@PeukyWb@EnXaQ#mN*fE43@RP)hOyh3P=(qK~PxxO5ZhRJcaGfh{hKj#6!IW zY!8(?&agV~mx9|C!6I(K#3Au)I#2VxkqE2$x}g1C)1SsqYhGGM8`qWs@h{!V8m?JH zE9Y98@#9m}M-?==Fx2P4{u=90_C3kv(OCL+*>vY5GVbmB<3|>JMiLDqqEe0HsPZ3K zedZA|4KJ)~uwRTNO>$eDYwV%}TecM$h2Q&cpz-_tYnRp$mR;;unGYKvm(EEa*=l8_ zKR@?lVRj-)rFK(;MWlW%I6p*5d?+nP=%BY$7j)Uq?r5xYLF{_qN&3sgzU)^xc7-X= z)r&37^=ik5l_{dM4>S0^;HNzl$Gbx9?;E6Imxy# zZ3&V%!{deNS@g!zbqYj?#*R?RmR4(??9%hb;s;+|SBn*iYQqQCc7rbIN>j!=;ppJ0vJ$TqeG? z(cAA|A@HexjG$Zx#+vVO5V5f&!6ejC!cWSMW!bMM9(pkE!YNCvD>7ZX69THn2Wr2K zG1XeU8h=g@r!uE&iZQ3|@w?eu{x6z6Ae3aX#hb>tE_r~n^xA?G4G_n6Ff5ZjQ~si)I@JUNmY5XzD#q zqFdT~DuY_oq9W4TU6tHup)Blhr1(Of!SA!3AgF!)`5&!b7_BC!A5EF$<8m|%Q6MyfM< zGipY9Jy)LNBI6Trt0V%O3%m|`)!Bf}p1ltGMjkSS)C3VaK}a!JvC;R(^^F11d?cS> zJaeC+T3~iP?sh_dQuu7Z99qo13Hyve8Rv3x&L>*l-y4aiu!Fxy^7mqe*A@vNo7*rh zNj7N3S7%`?7CFv-thHf3`2?jt$S+homLWN2*tuMfqIy9N@p%=-$2YYdx})ZMDE8>+ z4Ke!C##eyd!B4k{get^}+|%un%W$1uAc{2h;r4q&*bpr3s~>xd9k@;6yiaS>@dl-f zbs*Lq+9Z)AT0SLAV<&JgBX-amS08*o;hFq$)VN^lX1oY!DusFB2Bz#E2vcaIBJxK# zsF%|~Spn~yCnpfb>7|FoW0M3?SmuV(+GrrU?Hu;(U1vxDu#E6a5h?j*zZ!v`f)Bj;`-Hc`v>dlj8y$@qd%Cv3 z_j}wkz4=9I@%DyD!ed5m-Uygfz$4ksjg)6V@1>Z| zB~)1c4KafA!OhELbUPFx6@tVR(1)`%-*;^7zeHAd4~7hczMhZ%nA#WyoA(TgsJ#+w zY)AsuEu-X+9kMuS?+yK#ro+M&|JK?rwWSe;tTC(NwvceKZT^|P$?`<+F zqor4mDXjty@rV?dMP-GM}iD#hS{%bfJpFARj@CA_t zsdH$@XYtg}Yp9T4CQeT~v&v2uJY(eNW$3@chmr*GyT}O5`!~X!%6cwY!smUw*-y!o z_RnAeiZjuDzA*1AiWuv1TwH{Mwi~qrN+*}8;C*k~+FG}`BMC%{ymziIpo3lwsFQqS zeHbV>Y{n?z2bJTBmoIv0De-z}_72F^^N#h0C<$UZ{_Z5~nQ?N8uU^rVqqT&KOMGz{ zIvt-d_u_s`lF)yx7~^$ybD54f@|6^Gj4mXc47i5T<_mw_9F0_IZufUi8L$2>-p}@494-<+C{OA*k3FhvoF+Db70IeBpzB=f zGAVQ2HuNW`K{UA%C(MK|=dqsN4<;@Gb`GHg8f}L!>wu`Q2?TUHhhYBl5|_dGGO}50*2A*L3gAvU|UpG~)y=3))XqTAo&7ZDzF}!m0ubsFErhRO* z**H&`F6gaI%ex%9zPWnS!jgJ2Q+l3pTU{m{nQ0tvCadR!^7K|!*|5K6Mh}F-cMgc4 zIEE;q9jp`Ba^+vaHmG-B$A4RjNrN+R-kcLWqUG5Rj-Z^;W$g8F+Cljy=yU2@$YAe% zwt2l9b!EkI;E#IlkujXX|9G(U&@&qy=&?JyWcP5*R%}M7%NV>2HdZC|n+0OD{RH;T zcr%9LE>w^vOGpRmn&R1~AikK$lAjcAXU~+qa`i{!)IN8hH2L56%p^68+nR4^n^;eK zQrz}Y@rkpTV8DbTr;&I;t%*Lsq8uRI2^hT(X%6q;B`xU&-!Lr=?sHLL9#z+jVO|gr zq|4j%q%OD<-SnSN0}=M?8wPWPsKEyuv<$=rdmofV&tAwnDD=CC3k<6gu^Fng$5fE= z4>Ea3kHx}?!;6fFV?3EUe_VYIskxcg8ZQn7A>Pi-=n{x*0tT6$HmvHwko5z0thJ=v zZq1Ncqh>?}@s}Bwz9a|*C^satKbXQ8+?=+wO^jhFq*6Pg((gEu2Ifx^M!+zI&Bm}Q zAE6`!(Tib~f;Ho#lsdS~F?x?au{w51lT0i4N@tny@4EKe$eGJAt=Q#0Mc?;lwtuFE)nEkak-#_V^ux5t5s{KB(&nV50CZ!XTSc}kqq*M0v% z?whdoJFyoFXLKBZwRK?fZx= zZRNsXY;vFrLK{5>x_5k*vc&SG1Ez%J;vN)E=nmcoBSuL%R`p)%4qe7gdTMoJSNuch zQcfdjcM3lTvkiGxJp`T@U0r$WWgM3y!#UN#*FJ=X>pYFq&Bo&2MUYWjp%jS7m!&Y# zHHJyUyOY^UM;NJg4df}$9dS{%8LEBZ`MzlW5H6iLT^|LDu`tx~;HiS#9Lr+Z2oaD(A9FwS3OB7lUU*~etC+bp zt`}#%med^y$U_Xa!GQ6@W!+c5`T9!_Tp(W4@fkBa$F)LLjBYU*R%TMPge#>71i9Nf3+{COs1)~DEI95^h(GGx zT%R_X`X6P^@@u&SX(Nlp4o0?JK1NJLrDw`L)&xew*_5DuZb9(l8*m}$D z@wZtN_Bwf43ZEPDgvmuy7)Ixp5)k>tuQ16G&Xq>~nDEeIFO}dQ621D`NCPX1O(2sc zZ7;dXxuZ;}4d)vm(A}Pad)mP1UhwlxX8%n8jNhwu!>?YGc`9T^!!x@+rsgLR)-M}X zb@0iE($EQG+&yxfDL0D9*#+ElpFD|IaB*<4P3WxKnFAnHO)e=*9>Z@SDy zB7TWkahBmyjGHIJ{eCtu0_I1 zTW8ox4PtI>2XiJ~j|rvmOdwh+H5psAulE})xL5idUYoyr0}EmZE$Fw(kh+qwf8XEK zZU2Q#q7ecG>m9tl%JM5L{%3IvC+ZFY7hD~&tq8ovt#u0Jg=uGTzrchiBVMU7lLWKu zIvZu$MH=W$JI?C9iR!N^kLxgLsN*1mSC|ygF}l?=NkxK%Y+tc>eI7$+fvCl*QIOP& zoft|x4#TpVrYMph-y9b=yU+`b(etFM_jS}(IArYQ6WlQfTj6d&!*Q>$f1f!5ksM4v zzqmlAxZZL5G6bEZS0)&kz8RNjy$&=Xu&QTn~CNxo30TCnAt zulLB<{%$7uly|Z)a`f zY=+|O=mLC8+|Jt0QN`ZK#EhKh&yQ7kYvcqX=j8qKFELAJr*~$K;&wLncD81=&cL@= z-dI`#KL+^o>E}q8nb?_{p$G|~{8b=#XcI$nG$<%4DCqxL%-_%cyO?bMC?fmQ?|u5{ zZ$)JP>$m@(6%i=d|5X{ep58g|BLDBohz~DL4+jNZ`QM89`}uztlk*?NfZ!H1xVu|$mtesi zLU4Bo{vJrWd%AB=_srbe^L;-$KY*RH&swXVRl2HnZ6MLV>>V7Pu#uCQfsvSf;BP7T zTMGV`g1@EUZz=d&3jUUYzop=BDfn9o{+5EjrQmNV_*)A8mV&>f;BP7TTMGWamI5zm zsJGfLJpN}Y<}vy3kfUIw=V1LQOYxWz`Ass0h4Fiuh!FhP zte&HY5Sy~;2kqPvmTPD^(jqNgg7!!xYAYRXoa{t?Rp_~HBiYU715eYb9OqTW_^beY z-BBe=nXk|yGAe_poW(l|lK3`PV&F5wu*qA?!`8BBFpK{avPUEy(D?;0<_BliG7FeF`~M$H z7zIhe`OqUkewc(vXMP7`{QUOw>75v}SF2xTqHCZ%;Ub8SI}vfGu36Ur#SnH0N~EVI zG2{tAiq|l7vn}7;Mk96xspGSg~0v}iY8v=i)MuF*nQKRtJ z3bj%1qgmt!ay*J~B+P8+w>MXJ;nM{ToN9#7U(kthWJ=r77*1rQh61oNXDj4-D2RRh zZ)iV`PdYTb$PVWq`?4NBzcC6zkz_RgQkPxQ^L*$00y5g?_t^arg-3R0{Fxj6(IkG? zcp@$iVoDCcY7rtj13e;oS_WWw39#1Sr`neXd-;=m{N5KB{^4epf4uoeMGVUW{{e+b zmWYXnPT-+9#rkos??n-cVuHV0v+}EIhvZmiWu+O+fZ-;#rq5`^L?ov;Zm>k7T;doI z3V{+n@?idNNu7itMW3Kjs2VD_f{FzY8;CrCfAd;@3icR9zPn99SY&M>!~*u9nB`rP zgu%vdns^ct^2EC<&-ZD43>)UCTjWXYC-1!@iP?l+08O^Y1S_`}Db^a;d!@>*r;PUy zpH5>a82uR-G1Fl)iilS_OD)fH*Qpx%;^RnH({VJ0o;EV4gfz%6CXy^?hCrmhj3@bG zlIa0$)TAtt2(orTJ~s9(Ng9OyNrKM625GuwAfw2Wr~84&V8E)gPL4)9q~J>q(-aVw z9Fx@`h(HPb9Mq|Jv$f4$Pf=6l$;fu;?9 z+`j4&w1#kRy(gBvv02HWpL!)*JTZ0D0rB+Xkgt<~?zQ0o5qb{(!kW(w%yUN*g#oU& zxzV&ZveZ7D(d};IiV*%~koB^(0I|~gIYPLh9Z3fD)6Au}mTh2Y5zr24aLm56^kCGw zh8MutE^1ZEa;g5V?yE&#?=NI;(b33!)KU<67O|K(@W zJZ#E01?fWaJg3hcaOz>S-dv?YeHQNLVmJ{RJrMGmA*#EUfE+k*`+4p#swN;;}KipRNRGge>c;>P_lP(oe)2#H&hUt+N`VF!^4-&eoqO&rv5U z;&Xy!Mg0&;)2XT(uB%xVR}Ma{x66fx)YJO(E1s=a9rjB`YnBp-TI8va2A`b{#AVVW zmqyCPCkL;Q+w2eZJ)SkgdEO8>v8ph`1)jkT$8Y;q$izN zG*jG3+)7_@bYY}L%ubv;iQN0CIOjMW+Eg-~XWo)`IX87+3T^ytO1_z8s_CN3RqZ@} z5+aEiG5YWH6ZHf2-+X21QXpxJ@{OuRlgsDdDjOxqeug7BmhC!PS0z)$Icqdak1xiS zbv+in)pbg^*>nnPL}H}5M!tr(hO<`rNfD}@UiTZ5;?wMq=#Ykx@Dt|skHag&bF8Ii z+V_nVF_)CqlpZm}F~%{VG2sQ$DmD{OCXx%R3gpJ=%}LGm%xBE6Cuj>phjWLWQ-+gu znD19_0=CQQ1KsMb%+ETudA6f*5^`Por++gmhG6ld_LjcwQ z8`awMF$wzv%Ely>!5rzuC40FvBfMD)g*?Rud6nuWnX;8b=vq;t94mF_$g$AU$Rn>OcRmW^+qJ@@$XOF-6ib3U2&cb?i zaX!tGQ_G$wSE5$g0%n73!|*xMIq@~=H3KwHfJ;CV(#5oc6M4g1&k59>&E>kGqmUJ7 zGIz4_F^SxzWQ`$;A;h6fcm%9NtZ^D_yOCGRb$E7>w1%`HIxe>9<0^B&W`H|a12R*g z8%~{8oeR@EGq-iuRLD}ukz&C;?{lL!aE9Tv-^G=xMXHt7MqTJ1m>=Zwa`Fc9hVr5} zJ2e-2(tXpr1-o^+S~!_H+_;*$mV;o2D1zOD%YqPvP=)4&XM{zAbcE<=18r+~O-MJY z^Wx<=CYa!jkdq%bEDM4b#+5xbEj4oy>8@*?ii?+vGX#ykh_Fy+gLruFXs=jsD5eyq zvuK}Km&k;ejEK8PoQPDMI<-dSs%QuxgeZpxlMolf{e_cV{>A8mRKUvaSGbjZTBet= zmAkWc)oc0v!zB?qZ5ppA32+JVmr(8l(cAOaq)Se9K02>KOhq>YTt{tmQ*DGEX?-)g ziG+xITk&>8IIdGG&_JR@0z+a2OirNsi(`e(vF~dB`-u0M6ioRe`GaO}tG-o@36kgq zl~HLi@sKsqbueer3)O!zo7h)D&)^@4lqR>=Fm8wsyOTZ>4|(x{X*-&cfzfR zti)#y`$hXhjc}K&gBHgX(iF;?bxkV|*1H8i3Mwo&(F|!KS$0`sSne$tR~DPk#Sv?d zoay&}4f-laG9R_Gq~WP{xiE14cCRV>Jp0=DixZ6V{K8iKiF$&*hW5Bn0V`W3-5^^MGlMkiDmb+hdWMijZd-V zvAt()4~^*(8jc*oNczB$>g;ta*zI=| z9EcUbspi02U$r`#TvArD*zX%xs(tj)E`P4+`bH(L(4>Szi_0_FvuV%&HguApAthF4 z*UN0H`oOAb%eL}1cBMG18Pap(O6!_#@x0!1E$luS7d#m1YM-{*{a$;XXUlW(b_6;f zS(|Ubjs4d2)?jxfgA7iNGNpj8_;%vklyQF3>2BJT?r`NhlW_k!A-emR%Tl=x$l?59 zD$y#@ZLwmp;gHLa1-J6kuEYM2xUm(#7K*!?i&O?ilodiC~#k^*Y ztI2b#4a3dVF=w-boZO(r!e+j6-`n(SgVUbKtzEfqaxtl)e9j)QH^(ldR>tc~7jI9l zAH3IJAD%qAp$~p6BP+wt-tbS6gC?*pS4c=u&)&$8=+TE&B+~lH0cLrKm40%7fAVP` zj$QtDK5em@iMmoBPKr`gM8sz>)Ce>;bdd;Jl2dtz{MVro5NMJJumYVW)Rc8jsGzv@ zlo^`<5Dj>JHO#g}EO<()9sM@{Ps!0BZqE1fgw81@L$1SY2Wlom984glsz+I)mdfxx zcoHeptjs<<6dZmsu5j``pqz(LtlZ5nKLrK`LX;scz}_@9wZNi$-Aa6}x6^RrEjSC7 zo`ej720mj3#=!^uwFBgfh(|s(GDrpVd(DA?XfzkN&U!{{qE1CHlRR5g-!bL7mwkcZ zh;NgcK=MXhxiz3ba6;vWhBi$2xo~QqIYq&AqJc0^5S_0DyMVpxAR>r~W_r^a0c|Mm1$jt$8y z&tJDXymNnHhMw_k()U{?wsBTFxl9&}ln?f?!t-Wslu5KQA}6%N*Tlfbrr3p}czok+ z1XE&Zk(LD!)L;NaLo3}v(wnpxpWdtx`Jaza&V)=0=$NY1Q%5l6Y4$`5WPMFlFeE{- zYrnIKjNmf7*Gebnh5e*AfK8*5#15*D_)g=vh@F@K8{Alf9PY)cWSBInRHqH>nzo>X zDM%+hfhC}x`fYFWmwvld6bgfm*r3jrf>_;wDyxBC135@{ch^qdAMFUulfSoxvUn|n z{)r5=oZ3j}FHk%`jsg@9Xd(7H*mz+Y)q7pla!O{5g^fUsT zn4i@{m7n@8r_dr=n-V!HG9R+*Mu&Q(9$EJt%}Q$#ak|>;Z0w5uNj&|iH+=%D^mZ^a zNNk;+uWY!{v~TsCyLoJf(@k4WB!(=J}Ih+UvQjbWw6NEve2nD+H=@Gw2r0xr}K#{0Ok71$Ia zlSCN3tXWW(7#u-Kg0#f=z4)uBqoIugyqVaNVvY(1<2d+l+|uxv!9vAyGoG5#0^VV! zQ+>9p!dVEO%*_4Vzlm6d?iqNl2cp!Y9{+UxovW^|I&L-qq4wJR;L{Xo(#l>|L+5X-q|r=$m3{qvA9}C*23L`e(d`KR1R2uU zeoF~pp1 zQ^XQ~euHMwY|*A}l~A=rWf>}&uyy7^!9m7Bp+q%i_o_U=B)}KIUF-^o0f<#wo1K|4 zRri>SS!|fnO)X}6DqQ;^iSz)-)Wy=2vMRMUs)o=>yQ_4e7%vC92JiH0eyym`pwfWWVADty!w&xvf)b|Anql2J zfl7EPwtc+=e#_Tn*+$4C3H( zxQ%T3kCb++heGae*V77el52Hk3M&-`^&=@u23#R z7u@TYef@(}!#inHON(KP8jJ3YIw7@B>F`<%ZqNzs2~&nyQ)P8(Yzhukmt^^v_)>JL zmkhk&y?yUx?>F$cWJyivHdK6+e5k*Ow+8T+wtn;-6i}Bpm6wp$ER(erv=uq*KskvB ziI|O8l%M)MlRl`IRraoILa(;2`z3F!{~Yc<0WaH)Gz1SZZ3gkzS6QA})D;wPp>W{< zTu1NM6A*oG{Y_B@L$z^liMmNi+16_VR|6T)smOSP@o~Ax&OPv$tlj2Q>Mo8zkEm7- zNTx~2(Ew;H=(#T@UwDbbNdltBIm`yKK1S1$;gSYN<0_F;1X6G*U6De|Y4d%w2-?N#D()vj4ju7&J|4BvZ6Ng2t7=qo?0!NT>k>{nw2qi&bJH;6ZJVBT$< z0=1vT>O{YBFOYf4Tm>VfhEzXYAm&p@$P>dOFF|P;Lg=55x_m~(X`wdB@1%HshgY36 zo|K7(FA=*sfDru@l8G}5X;k0Lq(p81YF=C8tz|A=q+AT!`c^xldttlkQS?`ZW*aY~ zyPkwVCS(j#EO9u1$;>c-b;ngte3`EDXy#^y#(rR(I_boe)?~lNwYRa#mF9$I9c%Ca zfKsn}AvoJ!kLkeP5Jf~kw zJ2+O}uh-LbYf9F7EK)AIxod9LT$%4|SM06sUCyQMiO;{f!wT0icf#4HUbu9OGgRBy zpI$IApSLhS$KKP>2%e-gZ@7@1nQNvq)iFpmSg5%zy3K}e1?}?X^Om_iTW@>!h9-a; zX#&57^QGfk*Q_jp{8W{tIU@5-OKIy}b{wy&?7oJrUYktsImdxr^|e<`D^zlyWuGKt zvun6ilxob*+f%NLN5^i_T*%6CdfpCnGFh2CAJS&WvfDp(9lDIh@}R|Q*mOMFBng*7 zj9Z|cs;#dfxLi29cy48Fb*467%3R}K!{!)r!2W9DU_0MJU}{KfL%YSppglC~11y`( zQleeqNzIq@V6GWw?WMt%Id7Xb&(Yf`XaMpj0vw*AyN>wYJ}1`E#?_Pk~9DtEF^jlAWKbY{I(nkqaUzuDZkXu6Y~ zIW@&L5x7*|6t~T39dtKYYz1$*f8*Woybp*9hazyrDC0a=6*&KgBF+tpr|-5c;TRaFVUPIAwyDJt zT6rL**O2+!!9a*VfKDScV01dTkkxD--GMm1db%$!$n07lT!}{H@N_=W0}tE90s$l& z)~B?>*Y{2a99+k_wXz)_B+*`FL+Z86B$#d?zV9r+taGnr)*HmC!^`P+qN1Pj$4m?y z^gl&B%s>6-ufn45DWhM)qMu@?e|R#Kd6`|b~56@z;=+@iyN?b2~+2>fPB&(Q>pYCO+3Ky2vn*Q)-%@~t<~qI zL*0s(0ULxYACwCb^Pyk@Xrcxw++KHTGk*~5MN}lOQ5@i2LhcV9jpM_Psfmg{Q~t0k zMi%?h5+*cfuf6u`w=e`eA3L8^ON)>wT!C*v&65j2JMvB$)OnO+MQqKCv-G@yj)31y4=$X zuMf^dCljXZvoT=aGOeb#K6&;5#md5Ct--#~^L`Xz<||$a{9CCGM+WL+5A>tiqUH$M z5x>zLC5IC?<;&S8-*PG)(nn1eZX6r0PjAm#R;~jY0($1IS1x+y-ppg^za@9=A0n3E zU{#(ojw)bQ3dAtkNqNa55vX5H@@dOyS-0Y3Qbdx{Dhg?^hQvy#FzM85Hu<%29;MW4 zq(E@D)!ampC_OXsm-G`trY{kDSdBmF7t%MAOSzp_xHdW2&`nqyHcpC4NEFANOk?G% zl_9PP%Uw3I4vl6Rl#GiuuS{7y-#EZEU*T@IsE)L1N@p!$<2nQ(%wo-;mhfNeoZcoW zW_}iL-jo&CJvUIpO4iyoFR~JKZcQhCllp|*&usOzUZ4T(i);M`RLOGOeS56pxFe;_ zlxGBbb^zrI{ZBFU{yaLKUr6lhzTD?U$3%Nw+e)4Y&x_a&F0ELcLVtdNAAkuq(|P~a zD-&J!oL|$F;POS2GnX*4KKN3defqKGTT7VvUge%MU6hDD45@sE?wAdEBqq~z8U=Wn zzJVHlM-^$1SbO}+10wG2Ijv7HcBwKTetH>-R0nAWqy%$SRMbFPLi-bO?rXN8Nk!m5oc(vz@rg0RQ59%BVVR&#O=rc7cX0z*-zovBkI<&9%`cq#kRp zfS!wVgo5pwugvcTIWn}B#Y_nuCzI*Du@3m1MXMD#o1~dDJu!`MTVbvV{$IvH(IzU#RFmDNv@}_bvUa?iM}HbBqy2&sYqxn`ir%J zx>{FjfPKl%CD=}irlmE-*jzcmw;l0ieywt(V(9xY$FG{fPuwE*Z9I7CW4!RTZv>3} z>f?Y>sR`%QFnV4lr2NqSu)_xFco&;%4lJ9DmBuTrET8ToQD z_XQy7h{HKft7>=n!aA-7OmE5EkOWp~Za(GA4;#?TL$XTeWKQzMNmUMj=0n}--^xVO zJ*iMq9@0@!Y`#gEZ(GI9)%?n8Nz(8>MQdlmN~~ekJKI)u(cdY|30q^jWEReu;bjsP z)jZaoNqe$=-djoZ{b}U@F7L)&+j~O0Y!KbwF-AX?Vlc2UGyPY6e-v*B*#P?~SUcDgvHWkF%q%=d;tWX_2&ylRw9mHMqnXg`(+W#z!4s&dKeGbZ-e#cc^T=6*nhr{ zft`r$m*ue$ar`niGZFjG-(g@SV*439AR?^aX9a@(OTEK)uz%Y7pSpU`|EFb^z%Bfj z+j=nk)I8*0>H~1B(e;(!j|C+{!#s8YdM`iXK z8d-nn^}m)`ftv5Hpz(3>zozlKj{Xgi3_wl!IRC#*B~Uv5ovr+#G5@oz{G)a|-zj(` z_2+&&j|S*Bl(PNEsr}W2{;!nJ9})UUz_00iw5`9PljGlF+h0`dzqy}(jrw_9{3DG& zTarhY;kPvYcbN9EFPGw9(IblEfq-A{<#%uBH-s|K|69BH4;J&^+s+^E;a|`V>pKO% zCiT(P`wgY+|0P2IGClN2z_00i^kRQQCjRu&S^#6Wy4ef7m+^s6hxMm|VE`TIjb_1vj6 z`M(kBs3`?UVkc^8xP;2~B+N%E=Sz5aGGaGjYrfCBE7hLGwsf!7R=Smy< z20JIx$R&fAc)lJ0!L6wia@9gB?vCrmyWCVQ27R=tv$)|b$d7P@kB*yyUU zH+>VEdR3KEW2;Im|3-fS^-JC;{>F{=J3uVb_-7C>m=IW$>H)sRiYK5D`sn268dd7W z)04?ZK+L(n6WCQbx$rd5-7IgLU@iz?PES+1F$X`yny%~nHN;8ebdt6rF#Qs|hQD8! zlQ`dLeD^i@`dq#J#@S5w33?Fp41fdH)4puZXmTD?@AVCVCM}71Lglm3;2?aJpb5!~ z#p)y;6Hf}~;hC9Rnv*c|JJ1t5&1tiP8J#25BF;)Bh5lP;($&&5BzT%BBW1IV8z+Eb z?QR!l?*aL2-rDDzkDfwfm3M>iz<1ol)=w4DZIm`&2au~$4+)pANR$I!!JUrg1&QE} zfpyK6r!tQQI$&{d?B_N(D1A!cyRQtF=Gb%DsZZc=)EM^!4WnFOle>{q!ZWCpv?TiAj5Y}`|Wa(5eYG)ik*Is<8TSGUK2|ca(BDBWoCB$J(bHW@u%hCGJGvl>^dB(fJJVCQi< zmj@nXzP}sJ84ie^RIHdj%2F=|sM@}C1W?b$lTv;jYP>wzBMTMQH+9jj4^B!F*eJmd z%o&(Vs&|UK?(3O*$7rR8#sFih9H>X-z-`XK?JZ9xyl_FZGf*f(nWoS4v|2A41$!7w zn8tv;fC&l$1qSOoM;$6CpaCWJ!w-oan*2EV#O}U?kvBCM;J@G@ebiQDGq`ors+++j z!)W4Iu^-K0Z!1}^s_HR@Z|G7r-W1Dj6}CZIYh6h)CQLJ?$wZd>h)t57a+CPQgylz_ zwcX)oEj5Y8@!IfO&=xZ`^{Sd2$w2_G638LeZp1nq;P^gDGJ<)4t=ZAxo`5gutAO&u2CxckC0oqY7wc}`rkE9a|I8d59U$xrxc6)Izv@M zd33TlBb#!}H$pTAC6P=7&PjVnA;&Rrw#xbM*~*TPL}^Trq!$c~gBvEj?;IK{3mnL7 znM_O9=nBZexVvbU2 zRWf1jubO95x9zG-Bq8*z0%F_>PS=IPPlV>HpDGeO5o8$i6bcMRm|T80Pq9e#ZK2Lq zgaaYHyNLKw1?66UvS428OBbE`-O|30AZ)L@{$p9U!nu;|%U6qa{0Vv4Iqccjin3lP z0+C~TN#eJdn0e&_;l;u$ zBl2Da6yTn_U64oTV*>8Bqia=3fmzJnN5?V=oxxn-6^&r;|8; z?Rc^Mqt4Mp$Vi~b%DJH;lT_JI1n$-bre-iLyz znvdx;fgpTxVqEd;RM8G)vj~VdcGz|ZC@4fbWRNE&jVhIg@_S(r-yfTf{7x_Bm!Q$S-Hx}(7RZPPy^oP3-UrF{; zog2YzB2}FS8BK8UH{cStAAdKh4Oh@WkPd0})+wP!)i)J5?2ezbwGeQC8GEGcNd@Y<@7MkRo3C1IpwZ~-B1^7JK{nxp4Ta)^|6i(a} zs#N}GdwUFZx$f??chm|Rg;{27gAAGZmLo}JG*B;ggf~6lh7Iu@MgKD+1Q;Te-0ynv6`V3# zLHh||Fk)X5$i)7e=%BOdIzVcR-0YW)(gE*%K>;z$!K8+k1#%U+8msvlapby3E$DEd zHKY*V{QO!yBl2`GY0r&?);wQctzW_;a01z*;l353e>7j%Yl=aPMkoc8v)+kIZfi+O zE^ik%u|%VF44Sm2aip9fU$8+*lV(Bo(Qa-n_H7mM2m5HTC`et%_3lE0M3bLm8SRtx z3BL)fb))pXQvr@28@L*!4ZlekT$PfOaexRk_!A`yc3`YE2=j`IZ-2B_W8k;M6R9Q5b<{f@y9>Eo8T7%r*pjfwBqiqTOZQTH|Gh zm1oGvmb$lBP78cRuC9vO^W-pK?G5lnxrsh$>-4j*;gAH!>Wt~)bWtQ6LVN%yBVJhQqUmMw5|VW>J7(uikpPHZBTSJ2nB{k#t{;AQm@{JWL1 zqt_x&)5ODSHdA*CUvNGzxx84m)E`&W6hKyyF11`vYBKiJi2O{+YJC?$91d`DQgfiF zNT~pHk#35Ac(fLdhONv|&*i1YM^jG60+Z6$>FoFxvOnx8Bc&=H*9ykP*(q-^*3~u~ zEA#_ht$BNWFTk^YNci;~HU+v1g?00+Anc&o@mpOPa@$Wyz!!^!$+*nh`^;~9G9c_B44gh#a&3ZPU{ z8QqI`!!H4v5xdwyod6WN;_tj965QH~!k^~!J$x|NnW07IU*g=G$ZGr2vQ5V=?nza- zuvK~sUpv-;u^@pGeZ|Q=M)QhB|3IT*dNAE0m#1hpUxms^nL@Qt$$)kyjMx<4wXR?t z*^Q8E6R9*^L8@5r{;-J0ct7deE#;vT(6P<8UevjB0ip({5zRH5h#b|@Y@DB>v%Bvb zDvtHQRueE0qR183K*=+hHX#HNO%>2M_Q$Pnu*_%+YneEk7PZ+Yz~vT;!DcnR1UjRA zBO18?+=_9VjAlMc4>P{YyvV5KsG{xq>;hOwq+y*j{^W#-ywt6{h1SCx?*vA*`%^sW znDJB5!9it-E~*I^)$s~%(>q!vG4rh@ugd5VQDif$c(P5z32r23-Lspw&6Mg&sZOT-eS+ zHI)K7%m`wR-98{-%V1r+!I*^@yp)o}KCLe3VZHSN+bnQ-XqX%uX-4{P2adnKbFkO0hs(b4PbDPd`3M907|h}Wy^XJ zoAW+yVtP2Sn{sya$LLY(-qB8hoWW$>{=CSV*Ux?Z_LxQeHXPoLNa=B*cYWa80KU+U z#S~s{k5JWBNt(|-Bi7!3eI_lvz);48Idj5LZfspUwWYr>Fo04qe|e9?z_NZV4c#ND zVP(G}NGdVsN&LLYy1E3HY_vS}DB{c6yzJL8fcNBx`Mlu$-8x^nLsHAclzQ+O z8~vO%Vjekz>kw}Z7cG?kuKS}U_6y0iFnVI2#F6Lzu9_!V{(>u+hndGVDKrdVj?Z$E zHXY5MJ&slMtApA=B5YJkm{UGj=bMv_oIspNrCk@GWA>aMhe!MZ zJe~*NI~JxHT2PnE6*&!Ip9=crE4d4%MH4DU&N!xxV*UK}6BF5|G~4Yo0(O&OBCx)s z1S(Y@&Zheg2$x2$YLec$r#8&re$G}~`lWg! zcG_0i;F>ndT*me)?9PN}@kpOVr8J>Tdol#2qY4vHcjSN(vR$DTWXpNND0s-VOZ zAo`EKlaVJ-fxrA7J{~HZNlR$KBy!Q4-fdM{PwZrKrOoa;fEH};Cr&g6Gg%D@yFX3O z=)Aj)-mAeTNSUH81c+)ry1g`1@k?7bkS9532OHX7KFu((9LW!JkLv7h)Naut1_J8W z;Il50Zzqx%9)b*CR&*7I;HfQpGC#_}*wObj@6i_gS_U;<(Iif`*!1VDBMc7M2(!Ji z`f(mHVm(#?iSapGNo!-wD!@Kbl5P~yCA%2$$*4$`Q4qdd&_obsx(QT{nABXQEoXYR zuSMe0&^-QLT`B|Gf|-J@4oz?H zV*FeS@J;@4yY;w>`0}yMmgJ2=LU^Ra>9ui=YjW?eyN11X1%j~#D$-ra=ZQ0ydD&o2 zI!VkM@X=SNN|i)C?F{gkyUdCC9x^2k2F#w%75aIvZE9~fW&`*Nrc0iu4@73?d@72J zSQB5lsyzt%7;SB_RVD3|$gL`3z*PFw-qC$JQq&<=buCZqM73!S(`iYkd@7Q$QVg0K z=N8ZPI=jz)-Yf>t!;?dm$WdWRD54;ylc~@jiFZ5ZGw~Xcii+Xt^Kg0g8z<=+12mW_ z3jXm5iva6~=r0WB$u%{gNL36<`Z)~I=jiV~$k_|sJIO7%`8~&U+P5XAM>9^jZx`2* zZC=tiKJP_0-b<+?{d`o*be%hAmCjWeO0}&|^?j?$QJr!pKR1Gkc#{s#*tg8+sKra5 zFFCZCr)o6F#^0V%*gNbL<#(K1zIh9NTGqb>u9S0rz*TN@;c2o*Ni49(WPwE?lTtYq z0{Wh_tae*4M?Zgd4XD1@jfNKMcd8ej8PU29V$OK)XKB0{gW4F}&g!8OaarVfLq0>A$=b*QV`El0NilC~V<0HgcX9v=9Rf^I47vYo~dXh8+f$8hk z{bDp?`zQ_1Avnry!d*{#!0bplbNKkA1}c(fTT*Fk(9N?355;%BCMtaM3-r9I-8cnB z>{gxDWDCR}RBJb;WI(IN8=R8!?6%8v2XF!}v({Q+)_7d4w5a@bZ@TQINfB`r2do*u_6#5m0h&~MBQLEyJ z6>+ILE;hQJe^qfNG-ji$1w*G9lO=Za zv8Abv*quqaI8*;9eqbm-_J9!HzW{Y1ZlV)(A&SEcSA4-l?A>91q9<@$ur{2kpCcFH zg)(i2Rhvtq^A66H%HV$RG%GOe&V#q$R0%6C`CPzgHEzd@a5dv!wkDRc&M~{7mdp^J zeIuNv1@x^}E-l;ZpMMtq?v0vfAR7QR-9F5Br!tu(r~u1Tq#~V)`oF8M0$Jk@W+Hc?TB8+@uPW*e!T|&|H+=o92C^^8>&n}AA zJYf6}<-1R)GOEyz0`1wqxg0kGcLhG)4iscEvSuT4TMllyrHQE0qKNE*!brm{yMycG zz-YJnwUn`BZ@1~Su&scxCslJt+ql^9VwC*(x`Sc6<@Z+OqmwG_nv4n~o)$t81$j5V z>E-NOUX@Um<&k7IVMsO^8O0~-TcC04doM&ZHwMS3?YTe(o2Bn7rE*V0l?i7RcrIE;-uR5nOYWc8LqwXmb9 z;7Q35gwo{nyW5{aI+^h3+f|h`)5+~A3EO2Q&wyJ;sOfb^N;B2)oKGPdh%K6LXV2n= zc857y9annE#seeb=WEV$-Y;agJ@={%92=$jgo$1Pox~pG5$1OwU@>stRb@7n;0jPx zntpf8aS&Bp4(p8y5p^lF>XHwG-n>aYGtJTTpW5q-avO+VfEe45wl9TJe!tjaQ?OUg zh1&$p`tlJNp4Ki(;AwWxcVM=^eKA7;9^;CpU5jC`nQtL#tk2cBS9XfNXH$T>y){Xg zT5WC`TVf#{SoAbiA3nl%mFYv$D99-Ipr3q|ukD=p`N9vjQt0l?qI+f2;=DDDqI9k{ zN35d%J%@3=zK1=OKj|O=y-f?d0CGRJ#WO9cWTs=hz7gNSM6SBIDzEH`*T(YoB#xD; zN}_Uyw^4#Y=*ERgB4Ns+7ZVJ@$`nG=E9dN9$t^j(sG1E5d)?0r*1zXF1vl^tdRdEO7gH7hQdk!oGGt=lS~1 z)2uX519z|AU544Hx%oi-0G(70zkg#b~syi+FlgFHM&?PhP5%t`WdhLbW-o&Y)~E=OBXS8hTTFZFEjZ@!6336}*c@ zWT1~)E1ujg!Ss>v#GbQ9>jPIs!awRD3=YT+ArN02ATt1kuHY5R&Ys)oYTxKR<{j@P2ta_*&-!9$UE!e z5&c?l6-Thzi61GfDVkXwdeg5REiK}wVU;7ZTR)sdx63PVKmLyKtmDb#uqUb4nYlHB7KAIC$UJlN zfzi<%Yf`aknjj{AHb%CgOnc8Moi4ki!}@w(akpMTv-&)-tYOdL(>J82wbf0J(3)H5 zJ^`x!SH$uUUc)5&Gb0vM{xqKlm*Dc5)WY27PO56!`*WLnr#_N6?z?X*2Td1?_bWhC z2u%0=(GLf4BGhUJ6OHqW3o65Wm$hdBc*g@w!wz{ui2@*?5GeeA^n;h|iztm%ucF+| zxDxf?%aJiTzT1^>zPp$Bi+?=p!>7sUWeQSkhWREMi4kj0O_D}FmOSJk|9D1}Lzwuy zY+!UQ+z%8VD)YWDakK@2xB#l3z6sWW|Af8l9zoicF_a6|JN->(@Fa`+z5q-e5 z_)7aajN1(*aRdPf+&0q_9M%62&PTExWQiC??0{|zlgUsLtJQ1}1{LQNEX}h&{bm9f zsW}jj?d*tb?Xg@rwQ;+_M3#SbO4MpR0s_@u#Zu=b&szKB&uD+P^+XXAX?fXPBcsAJ zxw6TL8orQ!j-VLid|tL-tQ&=)#CSz!H?#jA@zw(vu0WE^QcNYr8RdeE{%m~tQ2wvy zWkNziiKZMzpWd6hWvcucKkyS#=w`+mXOJlLzd$)a{X=#Io=fQiohl6QG44OZrV2%V ze3W-w5uGN%Vx~Q~>(5sH*~}LhtJ~VjBUi6}j7>Jq;rX+L_dWq8nx~wP9?Kt`L zk44ls9^xIKyNU%H{!buYQSixIbD7^RV)~OC8DOV@ndwanTU*;t?wss&1WT<&1b?#H5qOm95HEBF{y*7Qw+9z( z*z(K8qWi(+MygJ=`2@?cdYlY)AoFoI!Lr&c&KtlGdK|Vf^x#EnkG(@F!*Eoi=Df>KU8L0zRNo(?;Ag8w5ng zX5OBT$+cV#+PJAXoo!EWOq4cXncL9#aoL-jMZ&pI@m{V*130}_PhtdPRUHmtw-Oou z`Tlkiq3De(jx{vCLW%xpIa9GcPYH*o{G|HjM>``V<+}{3lDZ7e?r8vlT6v|;-ErKa zj5pB6u3EqsuATyopK9?dlGN-c@j4@IWM$4l|6J+IH}G33 zNXcMc?>o2bAo|4**Z|wXkrF9FYKi#1q0EQ{s&VHq#oi3A{klxJ4wyMSwQ?q&zVU`L z@0%4u6DyosLD#}~>03oZmGLYsn=atgi^@;cjx{8GFEQ{vz}$1O|2eBx!*^m<55U>i zwMA}Q+X586q^&6Q2N1aRLQ*76A{tvPfj7|m6eSm2?4wW($b4XRy?`F{zFCcyYn-*1 zDmhbSNRW!7J)2ZqKVIu^!mz1ZhEg(m*=IrlxvmC;dpBACpTj*)i=sJxTs0r}z=T6hvq+!cU+CMiV%OXQ?tD;HXsSpR;bDLDaVk<5-Vpv6$SXZNSv)@JHN8 zakrLzf6|R>Q|0H*lHR-9ls{FVQt0OBIsE_Fd+VsGx-U>v5frfyBoq-rLcl|ZNQyMl zB_JV4m(n4nqLR{G(%o?6kRnQV#{rdY>4vxVIe70+fBJp*-8bGFLp4IzH*K|?^sFfd<`NGRvx>xD(mDV=E*~!Biu7y|ao2lEEHw+Ol`ln5QQP8GT z{PS6fCv;Cs2zBgCAUUWPAGPk1QhK6v>Grf~w`E6yux3Qu{=9*z%wBWkN{TI;Mh=en zV3ypM%H74)MRJ-LYk5aSGMBA)@Vuz)>Jt}|==B#ABN<{C?n_I$d!6Tl?I!<`7Qn8@ z79uF-nN3R^ejqNrllSpF{pqcP-KA9Zn`F1!F^kr^PH>YHcO$hbSQyK{3p)CmwzhDN z&SxhZg_qnH6Ljx*%eQkg>9IRYC%tT@iBi!w)u&%|blY>}Khc@IO0-iiE4;qoM&>ln zZ1ITGYM?-`q+IIZ(@zBEm!kmmWo8lB_#>n3em)9T(I?>gVCVd@uzde)Ny>%^@W?fV zX}JuX;@q5OcvnZ}TE1H?noPu~xz~R0LWHcYJ1|-Zymq5qc4ly>2TAhj zc&vw?)*&6HVh~f=CkF=7k{77n1(V4-jrnroO)L!U$$EzxXCQ%9`ZXv1^ONr%P95&_ zRfRrq!Cj598|k<-SS#7JHeaIfX5pJRBi~B>jjHu*3nMY(?ND`d@!-CpvPDFbvN+9n zbWTDve8f@GgI#b*zR;#Kd*G^4)&9ni49{}yi3>dK1#{^EAg{DolR=VW+%#B2etJP4 z9I%Qn&(ECaU6U=W)DP)7sPQ;sQL(Q4)T5}Yz~`EX5j;1_exbMz&&Fek@OujTpb)I0 z$F0J{x0IHQ;&8L-5Lsq}!MbX9_M8A5gKr{O!H;jpaKB*4H|=Ff;=$IVhCr&fJS%Ug zqf=Dc$EPpH%me^=a+o)v^d+mXwcyKO)?o62d=OyKplJ{IpU*Ly+V$A931Hfm-}Rqp z3UgW|*f|~Mx-np2=3hTF`1Lhe<y?)VuUR zE{}Ry{w58LJ=}5KI9sq4$2j1M;`^4NL>7V|2^AIp*5InWm8LGq!~M}i-k9{O>>)kg z-#SLqu&j))&=uk1IJC{?xADLEBw z74Vz`eoM#FK^cl0gd*zSDkoz^3w;f83PqPtYmxnuMYJdyRF9<%shp(wY%*BQwRsp^1j2a4|Xk zx)EX^lnZB@YLL?+d2NdSuvBlo8bha#jm<)Ia;v7D8bOd%9GF>5wEY}K|t zspUA6JhLI?$7%oA09&8eotfUa)M0g!>~O1Yg!wTzXRr*6vJqteP`vGKG_D!p^QhdJ zk0YT{?^{BC3Rd(Y0=M_a3cU+3m<+S9*{&^%>Cw)k3 zhk0E1^sI&gl1bGrOiG25i^Ev8<3k-1H|ftl#`I-(v4l{-n+2Sh+8CTT9`<3e)}7+p zB!Z8mEI!$?;>$@QFBTyQ%O|{rVe7%X7ABteJt~qiYE)Tq3a2k&-oh4fl84$FJA$qvdrb$C5fyMP%@WfHm?=I^XDR9aN?28q_wY+<3&bbEaY&Av*&~-P ziH!oYZ;{SEF?72n61A;%rvHQ_l{;}lC{Ch%;<1hMafm3xUIkkA0GNT*c=K!dx5lh< z*WP+Azjwc8lPYRVjYBz-G3q~{`3Y>BNT0c)oLxcBJd^MV8}+8E429DLO$?G*e!I$< zp;fk?W!;3{Y7(Cpx>Eqt@~710))5V2aJ;)a7zl7yxJ~b99@*oozJ94bOG8d?Jo>%K zQ{}#tyHY+a7T@O!+e%D2%qdK`5v;-NyuDrnwM~pw^BSVOpF8|MsDBC9^~=Y)Phqld zCZPK14nQ3CWWnqG>2nW2&+XI$n+l-VNzC7q&uz5Uk=jL<5qczs4xT_Br|i5UlmFr^ zH%I56#o$_?z*)H_b@NiHr{bgK6&r&lFdJni3TEcUP!$S;oS>AdeIQ+#JlKoBd(y<# zzdeDFYl5kl*o$V~|3rIGGI~(Xk9~cd zBc^<(CgEyN%OHiSRZQqD)dYW?gx)BN%5*~y_!&Jwy46vfEp*MPL;a3T z^a=ryLfhZrs>|_Nm3Z4{h!~#DB!_h(X>Wz!Oos=qjbyB~axdK{QuZs;ge8XZ?+&yn zXRGPGkl^@MYQL;wF}7IJzD3}^PFQ$XB&4CBAST1d)-#Lrvu~_=epX&LZECbm)6L$9 zYf%Ymj;3iaUHrkTGc}DJi^fewA?=l^m7QkSY`W!ANSElQN6OV4yQDai>1S;VQgnJ> zh<0(-p9^TyqQx?+FnRPUm7k-1A=2#34ov!jI@ipTjP!5kS|i_Qn-%JHLoqa7uLY*& z^U=kOAv|0CashviWyEh{8oyODP6^>`_kTZ>MEzb*N`$qo^1kU6tnBg@x=8<eMG``8tZNIr5fuob8AL9`tfWGTX_jilJG=1qt56dx!$XY)Hs(x z*klR;hNNApwm&S0thoG}7rkkN(6@K5uSvkOqMa5?>tGJcpNLYuNuSnUG>`D_SM6;U zLbCRnHA>O1L@f@E^gf`faGLK?(OTg6mQt}P(W5UhY}1<^7}pv(YckpFIJTs}QlR58 zqg3ch{$$W7H93r~B>bAen`fDn7B%nkOlA%Ci#fSrj21o%EL*egU>#rKeZ$g!Bn(a>a|0U!JgBS8Y0oh{-W9iow27z6t3g6eFFqK@*7;{xF;Ovz<2YCL` zs26C%?j|X2j(8njIg}v?Q8?+aifXR51TKo8;rU2H2QvMP zCm}j@DGBDwk=ODphn>!3J{Fswe?R6Y$*FhGe|v0=p=B_vfKtlE@ob~G==TcV6xKp& zmbqB?Qu(Shf>D^x?8Zo0DjyJO1yU-Ce&6~^a-?3?57n3pM%^~)W2rEqG~W(2A~i08 zPj4!|+b(S58zQl1hn*6E5iqpPvYB*+@oYV|5#{?Qb41pnyL0z3@2B(9Jj+6i7QIN- zE^;5erqBrDm%9owA67X`SlJyKVHRB+#*pLGf1fh|J80mfl;A~tc%GCUld+N0v$<@z z^@Sc_6tOlnZkXe5q`Qs`RSfEwQDN%63MMF4^l$a!q7PVSQs5P7S6ptTDrz$;YI#-L zT5iQ*Wq!egq)o!!LSrcO9V{z`D>E!~dxSQ~F*I$1cc@b8eU%;L*_%~L?R~xAy0sd2 zXgZ4UtZh+isTLgX%8ZYQg(4oqpYdlvMBoOcUDfm&3L1R*gpV@OtxDDZolp~pg+jea`zPJV^&FO)B7~^#t!@IeP#F7Q<7(G zx*l>VjBu061c7tnQQP{`UsDfRdAmb#YiM+T(Qf9X_rx?;;MaX>2@t7DpOH>RJZ=cI zu5>ccSXz3%k!;nFs z_U%q1B`Gf7na%8Kp`)o%g{}8y+%Yy1O~#h&y}{j5-7KZ^G1b6PrLM*Fa`D4r;}B9! z=^B!UtWT+yc%|7-ws1(X5#w=p`5GXXqt*yi9tIfF?PXI3rI6eT#b;q;H_;x%{ThP^ zc%LPz?l=XT)vO#j{<5Bi&T*=3P^o^lo+H73;;rkRu6m^a1(R|5$FK#qfPZzs*1`pR z<(fz=%JT*8lV?hN%Xb`!Fb6Rl!-L5tN9z<{HGSkYE8w$i!<=3t&AV6P4);=@o6Lkw z+D>fdBuB#SftGZ9{q8<~PUm=U7RJKlsO^xMzL@cVx=J^!E(-(EkP6?iYU8!rMB1ei zIiz+$Y#ekaj6vfHiuEUiYZRrK~_HS;9;yh)Wu}{aW_kh#qTvimpi}L|? zeS!<)YxaA~bu=Q=z3fIy!=2Vqk)G+7%SK8^+~u0A`#4g8iC`9o`R8-XODn6}UGRdL zMVQKZNy6@gh8@yoM%LpSi+P0g;BFp&v2J202*F8mW?q%*ybss6D$vI&At@fNbS{J2 z=@|zsFwnmnuu_G$u*rC>Ny~RpN_~nQbgtN*jM67om6Nnn)9Q5#$z^sjaMQHz5FYXG zg29Ia&81#!A|PX~Bu_;|qaS1BrT$im)|En>IgykR>7fwCZ6d8_9YapqDc7cENyRA? zm_G=;`0A<>&&wm%SAH%4P5^RJ4YmHm@WBhCu7}R*{ZaLf^I3x(mpXv2Ja-oXyx4+w zGA+qAWlS^vX?h0%Nq`P^Aq{_y!^az7&U&foV-(x-J10-j3zC` z>8Vu#hqZRMyi|_$^6RcenM>g_+0@C|*$fx8cPa(zQjg4b zcyUQ0Z8M~u;)H-jbj>}IxZwn21!?gGyM~WuDpj`NmIfwW<2-n$>sF5wfh-UDDcJmlub~;Ag0fAk! z2qD3z=a$R_d;9>3UT~ku6N9H8dg`w$+dWx7pCZuBVlpx;CrY~BAxf59W^2W!)WUsM zZ+4`RPW4uMihYoNLn$M&w`W70&p18mgZ?@0fZY1Tg+8g5fW5mGyDKSV%@xddAlKxq zfCW2MdoU#1x+FtZffa%9rR=mQUw%bz)JcwSgun3KqVV~o=huk(Vs*J75i1JMk{u8c z8CEmDBcwS|+&gl)IOWX#XQSsyL#E1Vo2N<=3p}$b1bw4{S6C7*75r9B+3tIT+S8SP z@F|bGt0jbTE#H;elAphIxIQ#`(EEyb_+`yG4(lxtWzola*qf`hgnZ)2k}?z-1p+SK zgfL5@fuhZkq*<}Myz0~eH*t&0K|l%5w;%3J#K^Hpy`B^ExFEJDEzSR_!uR@;Novb$ zOKF#%7#VmdFAz71!g_*J21~~pB!*ra4UR}rP}+tPkSN{m=a!Nwd$|E$6SLNRFjQvO zA6aI+DX-p1S+w1)fsn#ikiK-@HS=A{(EV}48;+s`D~6KG`sb*HG*`n`EeNxvf_hvi z_$%m**$ba%fXsqB{28dWwmB77yR$<0=P-&UUK;3dWZ#yQi^Uzv`VvWTX>!{qB>81*9I9j^c_R_o5hErhl@Xd5Td9;0Stb!z(<~@h{k3 z)h0;_C&q%@N0!sajBg;@qW3zxJ=KVA*{UZ)sl-Rh@Sgpcl-J=Pi|PsGN>~@myJ`g; zo)Ne@+j=e)%MtaSnqmjCOU3$CK9lVU}6wm6J6|TY; z74&m0-Dlgao6K85{H}r^x9Y$V9?lulKRRry*4RXD(57hTx;r#Fd~&sG>LV$X zc=fYY>|L7jmO!*;#tde%IuO$MNaY1=0?H>01#`aB6>jPSA=pPJmn!aMMLJKvdp%s- zW!vCImyx$&4A0{0Oa??QM;J0ip7`=9wVP5}G-8H9M zsYEbKGT~XPV<>-A7Q0{I1mUS#Y zdoD*fKP_1eiZuH9{!^fhvVA{oqPS1fB=Ll~;O3QcuX}<8;5kvY7BU&B43llP*7TD} zAok?TdRP+5B~|L}B@F9vO07x3nE;tZ-VZqfJ&#_KPWrL(zzC7NJ!W~JOSp-l?3OFB zdC?%LTPIA{prl;iv*hes{c~fpnYkq(#2PKDrmR4%_2E584wXs~NY9CTZ#$QE>Ec)1 zw2V~Xbfq(t+y-5pPzEvj-77{*S6UoVSSI%tB?qp>KM1>X?u()@n^k#>tuK)#IY!=W z^yX<|(L=3sxmAnK8wqx2RfF<6Q&2iU?6M z=d}{Ep@;NQ%CcLv_%ds~jSV~-{kr(%lI;EIX%j$l1a}pizcx7)Ctc(dQ>s+-(NNfO zXN93tmjNjYV%RJ54WM_c{zogg=%ueC#=D!6KVizGqCWfj zN)uz5l+mPOec^S2=M6ad$%&yU`_&{~BOpaF5)S4FC!dTT@W5Fo48IR@!mnrZqZk4> zok5Y$t5YHbC6cdi^_UW-uT;cLKAsecThD5U_5!;$gBUEs=+iiGysg{%P`GOg#C8k5 zt{gIzNlz1oO>kc;?#}AG;dh(DL<;%L zOlw~U!NuYHi9n@)oesZP@kS5nvw3XsKWApzNia10AOlo+^r+tr}eBokic9WY*N_l7v^{%MX$xN3{s9IWxX>i zUSUXJou0a8A8*D%^G?0$RV}?4qa*eVh|UC(Rdvmm0P|pw&huqTltY~YD6GJ@Yl>tu zJ{d&74TZ~fiWb#9I$ZbXw46>^uzX4o58l=&J7+98Q|dKB+PqO<*g=6j=*#X&FRDo? z2@?{Osc4>|n(7Gk+L=)7P6%Bg^;DxaXUINoq7At)_?bqdbnWFjpS(CnpDEl}JNRaJ zrC5UztJX|7syzjcrAD2>y;U?H`DdFJn@udfNv;W`LHJ~eCqORoVdb;ZnqbM$TcsW?14nsEzXvkiri36+3uV1+E;4Ca)(tYbmaUJ2x6obk>fw;Egbi!1jShx{v3an0qCe#d=jX) z2y{~v{iBVDzTh4RYRV-sz`-rPbz02k&!Ho9Ny$y9%YgEF&tCw4rPN;v z{-NmTFAx6d@cqBw6bmSKuSh{NasC)TMO_e30cCBr9ufaqeRo~qI=!)~B@WFG2?9zv zTzx8yXApX+jJ}m>go#Hl(ja{E^{maubDYm~*Vg-W-|A%_3>KS6MF3MrQ%{GsV%(|fCJGK>XWpp@ge zkyfTIEB(JK#%DO~F2JK|K(N9ZLA#KzKf}8@ zoS2+pGLoZNaYt^ebSjqByNHH{aGB7Z8t2KPm?^sx+vML`$r# zf&!MVDPJg7W1CtFToWmGu)KgIKk}(7Ze`YtZ#t2i3*3IUV{t}+wHP` z)YfD?4#JKc5tH{Kjd~BB0cct})h#M|^aBOrcQKfwOIP{^ANEihN95MZqa|e|70?{2 zL*5-I;}2AoecZHjf-%q-J2b)I;gXt&6>2yr($cgb=v=vSu>97Q&4XY|!96Q&0u=s@ zWt-2=s>Ov4f(B%Hb?xp>?s{pEBmP?FvjOt^>~jh6{dSeBq(Me5v)S+OZxw)q5z$E7 z?Xk0PL??*(O@nZa2@Ael)0iK3t4Z3k5cu3RJ~|C7C<}P$td!4P9Z*Fmt2JosDLey5 zT$>{eX(*DDw5aEnTLuO_$Ab5rd@&1l%EE5UVzX`*jqo%*wYSkCX8>U>_^jBz3_p6? zFt$DX4w?9ZhOe+NTOGa#3bc?jMWS0Y!b0s@8}5n{(N;Syc#Nv)bLStnmg; zGy4ows!=rDYXI!VqA9zs=X@mL99?9giq)UjDkP=OfJUb?C~~TF{Yp;l)}^O@Tn93u zPZ7*?5RKh4uyk(U^ALh9T;jpMu!CH3G01sa>N;zY4HA4jh1MY0+1Tu*#-M@dR?a_7 zlEQ{hzA&_wTmtgdz_emE$wA8VoE zxz(wu^9v*~yvwEBQ3oR;u1r>|Eu6r4>6pY!ibI@nhb3y~TV7J{o}!|%KHn2817qLb zDbj8=!QrM0H#9C@k7yKvA)RQ`0zf)B`~y@*WlPDi+~#`FbxX=MxIj!rH8uE!{p%2?b3oC6v%{En{3%EN=*E7`u&GnFZx`;8BCKyaL` z

  • kX4xb~s#)!;DLgqz)fwX?m_8mt&EzaqZgjQWOl6i}ucW&e{BCxnuc}Glo5fcK zSNd#J;NjvXiH0c)_%b{JnvbmjArJPKTse^rOEjiqV>`U^ySAcK#zw{BYZ=F;{Axod zhTq+znmS*mV!ctj3B=XhxM7OSAVctbkREHfEV0QA-$sySv8>ziWhA`hU5dgFZEc$L z!!d}l3C->V>}Ubvo)H%)MM-Uf&3dVu#n_0cOsP*}A{cQ$M$o%vd#X+D=7Awz5Zh4$ z+tKu!vTieqgzGr9-}iZGjMLK+VvP(L8fQ{VMbm0crzJkT7kbtr;%BYV+OH2h?0AQ( z?~s_r5=6uI?Qr~Q`9YKUK><^057jjC;5llHn z)Q*%K`=FA@V5g;uVNkhSKFDE#)DC}n0C?vSi|2X13$W^bT6E%)YubC0QP!Ju@r|mR z)$1~w+jikE$d71fu%vX*t8OvC5S(YH$?kgNF<~j0e~|_oUXaDXj+pD@V?G`pke(}{ zg-d*^HEuK6U{wODGgv{Ixy*g>rM9}Vpj}0~YUiy+e(thju^X};0%r|!_SQkO+|tJ% z5)#yq7kH}sGNZCI-Sm)z1c}%Madr)1Mk)FB`v^&f(pu?)uJxQV4DYLp3pm-hzrDG^ zMlRjAxRyrk6eByafRqBw{&Xdv<|{;TeSNMkN|c*+6;uOrI)B0~?zYkXm=Kq#oZabW zRx5k2=f?E~b`kEdvvZ^(XQ?~gT|`jlT}c7(-&Rap)5*`46m#aO^(k<&s^9772D}IisPB$JV^!{ z3OVkiix)@krdKB-)vZ{abI*} z)*R%Pa!=2tR624*#Fy*dz;{oxq|UZEXKoX5;LwtNx@mS(>eDbm(O0pFIn6A(Jz&X{O3_B9G((vPeMAp3(=SpY zJrE3Z3C$-6k*$nTB<7>T@!O;@{*!fEo-qH>cTH#+9Va9`kN_-~Ogjb5ZL3 z-O8mW3JX4u_rhLm^&nY5pl%`u)_#4)ghSbn)}jbF#;!BJMX+ zoh1u-a`V#VgsI)YP0{j!<$qytUxHCx)D)QeSOEt?{g2evF>cAiR*J zMMw^=kW3&BKcTGPnOrU$uZj`PJ^w-B_4QWLXztFcDxk1tZe^`jRwTP|Eze+P6DMH! zd5-iH(>?(ewWZ$8b$GCq0|%Ke26U?iJUfP8OH8{QF;hu0l(lvgQ$BiJz91)gMmT{G zrJ3lZ|4B0)EG?1q;h4+{Mm!H$jD8;NH8~r>a4f?oWBO?XXk>5zHL#W$Y$gHEU&-)o zCsDYWV=BL6qt)|pLlSh?3P?G1KW-NKF85AG)geF&)Rcs@sTp6n04S1ioM&e`QO%=P zw|w|1T{iZ-f(&o6Q+qPw5VeHucs7p>yaLlyR1Hi>ilnM8ePJc)_;zAAfMaIfQ^qNV zFLPS8=T6kzY~5=zm+-~>XVPVH>7Bf@57So*_*kc9syZbb?#45?Z?!!Xg3S$bPJ^Cg zNQcY<|3TS6vI(U^D(G+zH#h0Vk!gAa3tiM8P}DvI$n5B}TVM-r5Lh~0>pDqcJmBO+ zYVs28FUU!59C=xT1aA&^oytru$;YPYy}{FiR=)*~g(}N9O0L$@;vPnLr|CsIolivf z0LA-&UYA%lhFhE^7y<$@+Iv9u5c3a3vP0J$?_ED&C6a3z>F;`Q z25s0r1VRxAOQiCJ(8GNJR6219hAt}jgH812uvxg9<(Y{* zm~sfAmx2uxaAT-WG>Ur5*tLiIyw~VOo5z8lz z?-$IeXjW}4Up;$STpei6&mVG)CNldc#*W3Y(TyhbShr8?CuQFt_lTmxZHS;h<(sB= z{=7Gk^N?Uq1A_gP`R!LI)gzlIurN9b)#zX`W|X!IX98_h=H{|*gPXQYEY|l4XqVc9 z%$~GmCFiSOf6QBbZmhiLn#X$5_cAGk0BfH^BdR5q)*3?(klUa#;eB>-=}|#KG(b z&U45CnL*yO@|90f@9_G^cPwq%A#SEe==zbg;{_WBV`cNWIB(z)|8{=u>UQp znhNffz5ftQCA@9t^8$>TP|4@6JI-gMSV>O57|*>cUREd2$SD^;!m9Fy7z}`+YD7mw z+iw8{+A^q#)~BF_)3K^5YUTnWA&p)Wgj>3JhxxvMAp>Iq0#)-6S_hyoG~E31$I^Ht zGq5U7Ri%XF>e%_6c-EQr?4I)QUAv7S+qdq)76}3s206IQ|Dl|Lp(zB5Dy|@gq15VG zjCB`R*YN<+OL^^D9TuaN`yi&W1JZ$f6aV?XN@%Cr-(7z-My`uZ^kta&yo(gQ)~Cg* zr?!&hwqIqvf3Y}E^Hr2zfvZF3p zrX&B%IjUX?EokL~{$}}qp%`Z$-nsxKwc6)ARuBPz?7GuVUjN?7e;(DUA&E^n(j>6y{1}b#y zLcy`qH)9m>Mq7LW6{Dl2p~N5gs)@mjJ-5tun})EZFe_1qdumJ=N@QUFI!K(vN)>TyukcI)7-g40Y|B(1c)b63!mVJ;ubkbO zugiuVQ+YQR$T-S4<+t*xNEEw`xt!?gc35H1Vk zQx4VVybFLMXYsz!S!=9J>`n~JjrbG#{9wk|{|_>wle>v0=zetAo(SUy4kvB}n2hp( zK8=ptLuqn)z<>TnUi;dIN080~GB1^z6-%V=$87o^z19Go)LyJ9 zmu-yZK-bf4(7b9F%=R_KD?!>Ll+X6`;_PT_r8NPO<#zj*m5fWzEMKBvlB?6y!QwRb<+thaf_0DxB^Bd z$ONlITZ0;IbLJ>M(Ec567xCpA41)_0ul0lHGr1eBqyZ!>JkTMy{ypeq)D25_K)ZUC zxUV)sC^1ck`_Hn(slK=t>fL3%EcICStC$C=MdkJ+lpXtwBf~XuNyYM$7lD3&u?EN_ zY=Rt3fnL)knW71#KoZ0A0oVqiBnjMssqSP>^w_U&xe3)JQ-zx)?Ha4&~y_F5he79CE6 z-$|$~fLeJA)FNHJ-|~!&>u_D`FyxC8Zr80&9;vcvP&C985m{x#DkT`i){>yL_l?N5 z10*=_=@~OYRn-yH?`Y#v%9ZKz-+=DrF9cucio%jFMgSH)$R*P=9LpO+7W}L`oi7;8K)M09{vr^F z=cUKJDfsP#a_YW6gwz0x-{}1da?)%8aE6NLTdVKfgar3v;Wz(mSX4|D3ke?s+1#4d zHhwL*rPLGJ48xR1bb*FTxS8LRappo|!yN!mMum?hhiMu|%8yY`y}98h#%4ZpFiykI zj0KWdS+1wHS=59x3%%LuhoFQBn)cxUNLRcu|U-;%kE>+4q>&xGv_&;+kunoC1 zt_LJa_9C(~X{#Q6JXxqXXKo3)eQJdqR(oxi!EN+)e2)|=OvW8$Cqc&7xvS=_%UU8R zcD%1rTCv~{LOq&5qB*Hv2^e@|_1YOP|9toBtCnwgwgT}G1OdMpSW*rwfw`$cMAL87`UWvOV+jdXC;XP zgU4`NJdwg7TjML(EGk>Pifl)I@l7%5rPILl!C9vK_Y;IK3-wVq_jFzQgK?bMyC2azVz|6slt zsRSUfnq;mJG=vQ(5zFh)bPWA(Y)^-APO^E+&PTF?c1_M+MG0gJqg-cs#cW3ACHXy& zO+Khta&8NS6U8k&lq{afcKLus6s`ucA)0l)1j4)zx_TYK$b^Npw%w(w(a9hXh`b6I zaO#Ut<;?@Dy4jx0g-4^5))aXeL>ks|I_a&qY}1|7@?ln>m^m_e!@b@%0B`4-ap;a*@in<H|&3uG+JWJ;=)PJjdZbmm&a3@=K|#r}(pKeD2xb3Q;{Vz5pk)_G%q zi>g=*zEp@$W8<+6l3^;Ks|Z1qiL{-6d`iPu`2Y+D(q&MXi?x^$eK43BqKCBTHG@V1 zhyhvAf;R`hJQ#O@=ewJRBS>}+J)>F^rH$gWD3M%1)<|96 zC-B{PT*y|{wU<0*Ag0_&iIxv?0p-0p-*amG@wh!lfbX5;M47sY2@FpU`+somJSI0ln9Yoxqsc95W6U<36F%Z#)BeQ|U@bZca*;iEDyQH~Wf~!EYK*dChyhhyy0~P*p zOgtbNq}|x2&BmwvPKvX1qt#H}=_};2z)!@+JgF*=vOK>0i#Gx(p%z=_cLJ-A8;bft z&5)Ek$U@I}g^JS;hBKr`gIPM^7#q|%UQCf!k?$GhK9!_k=u=FA(`|qQl_h)DIGX6j zN0tX^N`Q|-C1JhCiQzDkkOoktFtJFVMoXa=+3nbn{ua5;R3C`4F;2?-i-8G9b+Df6 z%I;nRT0WCYM?k~}-?m#uqgDPmGNvck71B%-Ho6ZA*FZfko1~2ue~=ZhIw>1R!S?BQ zxbD@th6@8V4%%xZF7>LbYqsS}Fj9d}Sam{#i(;dqS+NpTAikUmIY~gPeNZ-h3F+?} zxWtdeQD^VP|AL8OW1@_Iv%Ae^uD4VN;i4DEolYYiDGU`Y(7HC#rh=w28uM+C`uo74 zM<#Cz)kxj{#0xh^8aJ8DLbDrn2xmeIIYvkPH^6AFf(VJ}lO>3jEZSVL%(6G&U_J(J z@f}I(gT4AT(&ZB@g7m-~ik|>R8y+6>8DO;4mM{8bE1(G4`}|*&FEH9PpM1dL7))yj z1T$-tw&d|i7Vgm$uMHT)v4?|G%~8l0vK!j8i(GRb4Z2r{k^5V;0bAlC8B6Dt~;a6?h%WxtiJ9(}#O)hyAz)y)8y&XMiPA zjq`+HP@aw9;$dmx)1)!mYT$GT@)`N4@14xH2Lyj{li2tXir^oq|3xGDn|LD5%|lX5 z*JGwO?qH2ZWhK^WQ9LZ43rEU)sH6p!>kwvZbm!egghz<7-HZ;ksBaZWwtKFC<~5LX z%MmnasEs2#EWP1!Ps9+~cwnw;tJgAuBZF6U?8YsW!~J;gUxe;8dzjp&;VJK_M0aSb>rOEkR8JShODbn zIYpZ<*o!L<_m>zo5Q@i@EgXQZucoh7GGvsz!9wzM(eNeZi>#g7xLe{c&Rr~p_;Nh6 z?fV@HlwC>iZ+0cp#M5zY9GdP~f8*^uij>G9cZmVo<2DLZ@;usJmd zrqxZi<}W5#tEsfHWv>7;P^(a1N^ER~5NgmtJBz)*S!^A91UZX?`$^2#nxo$P0VO5H z_NaLQ2eKqC!~Vu6{sznb2Fv~*36_07EVQFJ2)eYsUw7!g$5thztHXo8%KG_+r)c*C zE*3dns^K_nj`V@6POO zAiI#PJ;+>_W?j2VR^A?^w%Lf;92pgM(JLSOyAUHXuigpKdSMX`SrSaOR7iFgc;U?* zr5YGPs$oBa(GndXeqOE9Haa7D_7U0j3)r_L+%%_!8#qT9%tWp3$v?r+I6VI(iP?uY zdv5p3iK2+a_n0hdoh)@xEse;+^Q2W zr#&=-pRF!*+Oly{V$rm(FCF{e0ui$HY-sp^p*QP;bN2<$b+>9BthG;leOmk&JG9|b zwgbvm#|Ql;UnPCPL>!QQ5z{4hQtR7oU<$q|oL{Y@p7X7|)Xq~)Sx$!(3bh}{+=1$d zwbq5rQFRD{xDV#HccHGzJbN&P4j5l~tp2AkCvU|wR#qPDTx^&55Y5>#c?GB4mJM@! zj2exH2y%`_1H`32ldlp$v{q`Mlxw1s$T|w{SApREMrLI_I@N-W;mzw>1x86dto{ac z_qqRxgxzth{Y%YOWu}>>$Nn!7es2%?>DJm3@Mq#ct2yrddhk&#YgjyES>?*c`jVqn z(kKGFU@qy#{U_#CCpHv528fW^6_zg0IMPScXi7rs>gMq(9;nZon%QnJg?m zP%gG1K<;PZ7f6X=h;`#cly)fx-@|Tn!<1P7tvWLO)!;q-ex_(0b((C5A2t& z2bH*l9A40iWUmVh$+>-0N3D7s=jEv7#p1J1;N+61KVY+gImH9G{%=tUO&{6od#jrl zm2re2@G*$u7qH8p*P~|Q3J6}`3k79I35{F8n}A+(98~RG6`+#GME&AA176B!fXne3 z^-?!dV09C)1Mrp=Qh{%0fK7Ub18s|<18Su;6+qi68_`yX zrJkZ$;p$YZcsgp`S1>RFx_@1A21OtQF5tfkZ}}joe9cV&AILar$7iUrEA0a`qEl5W zdBV_Jz}H&|0iwlBGbk=PD+3|=H9~;`LUaj2H23bxTL@7pG_?I2;*BV1nN9*kb<^}3 zQD>q(2SRiqG^`u7Xyh3n^nmb2A>x1rH+t%zx&(Ce&1(<@gQdSNQhlEL1D}_&+FfbVh$e&*xPe4 z@~CZLy9hlfKC^WTwWCA;QH``f^!D1JceE_iSso?01hnM<7ZFjl*`g@6Lk~`ildD++ z7n(T-{oeK7nt-YgU(umtk!9NMHPmlh;zF=1%zb=%sMgMKu;Htl1C5_j(QE=(ce*tc z&BuVlAaDnSE6_Z448xn?x^d|P*AmnjXcvV}+HRUeEX1vk056;^bD<@L0N`=|uFjAU z`f+IP=0ymXV*u8HQ)sS8LxGCK%SX+M66k}lJQIZeQHI8HDEldz)B@Kr0cFX%F&t3a zFdYW5-~I|EWKe(|H01s)+Y}6mSmHpmuqI!y2*!fW`my7n5G0TneBl-1Fa*ZexT-$b z#W@t6tziN~9Y4-O5j~1N1;{Hj?5O831%XVsEC>Ti9(vRTV%rf$8I)Dza~H7f+2Z?` zA&db-L!$a>=Ya@Nc@SqjfzZT9|Nk$V0)KZCB`*Fl?ccNcR|J7i&i$3Ne-+AKz4_NX z_-pL_wQ2vF?teWL5P|&bh#mjyZ({#-tN(u*JwQcG{^Or&@HMy@lkZp25gg1(C=STi zYI+pWL5FA1_7kupD?xaBUQ#MR2K{#s-Rrg#D;7VB#r(&r5OEC|KtQ{coEDwF04)?C zuJj`=3|;@v+d(V~1kP>u<)Jt*xL5-s@R#)~{ONzb?dT~&kS?;6TQNQAGiHXKwOUZH z{q4=2AVSndU5;v;1yL~#C@r8CzJ`k7{R9J8d^!vS>Rft=N{_AgQ1M8Xs)vdw)9%PWvoas_rOs1}NDMo_(c!V;mII{0aoP`U!;P zA7=5ZqesDMM?=r5E(QPg(>v}wfUXWK<){E5KoAMKZ{~l*6MtQMIuRhq*L%eQ9nl1V zXd&3H<_lEZ^gs6lji4Z*06_w(-rtTL)#ibKROkyn8X)onLQsKd9P_sb z;zCzt&x{`h3-bZYSJizLG=D>kJo5!d~U!Y^35|BOC}w*v%a zG!%}eg8Z^A8Q^MGLhTPmYR>^+*z$^>`(;t|rc^=<>~t>B6^+pjG+^Xyu?Q3q{_p#z zrNNfjl_a5ZhJZO701m5A@+>KQR6M&Fw!iusbt^;60dv z$ZzSK0btHD{}g$|NbOKxR0*Z{@0tG*w3G<1JkAk~#EbqPzs1f~^ zRs9D>T5f5{xgUPd&LE23zz2wcqQokIfy>pZ1eHbl@yDYLa1*EpFTIyTx^k3el?6xa ze6c<)Xf-_&*GH?Meii~kMgPdIp%PzUV8|LeXY!x-p>F;NC3VX0BnzQeddUc8QSaTM zc>TMDe@-2NrEUYF&!7J7t=GQ*CR$^+{~_mA!E6OCGJ}#Ig8iq51hfEqZX~DF)1VQ$ z2g)M2j`C3d1@W)j1=8noO$eQ=A3XKb%GG%gsgANPfUJ+ep*dTqSx@j=%6WtLmfmzY zN<&tMgJ=CpH)y$kd-Kn{>z)H{wuRl_`RVmPfM0{U?5bwHhtV@!n9c&ieM}RTaQ|_` ze{Lg4;0eKo0x6HGEpXEy&5R&0K|lKoZ~UBobQ3UyI3@Gq3c8Nsm*uN<0FVs5aoMOW z&vhng0LECBodl}S_?Ly<1Y!YQc`j=n4LEgQ1bbv_AldkP1fii_WG22m=#|^;z{)>T zrvHf>3Lud800^#A9;K6T)1X}X|ERM01rDZb5RmC65~xxDH-X#W*^lXGEPu~P*bw?@ zSS#p7s`~-Ve%YUC8GpNjpTN!D&h}^x0W!ea@%!5*-(GF(UMNR1gn$4osi z>G#af1eQ9Vc#Redz0}XOS3f}vhz>mO?gEoXoj8D04R zRc?V(CK0>v+wI0p0V^@hc|1hrx34pqf@eSGXi1=oPX3D;zgXdsK&YCUi=m5Beg*lg zILMXz8wVAjfovxND?6vtuKf1HOQ0|SR0Y&UWFKMV1_Xa+-{S3)3CW|{0R?S;Agf8X5 zZ|$99?x31vAa2HT!jWAI`ts|dO|H{|iPml;QJ6~AP7kB=jHK2|VA#uOt0SWUZB0%& zB-I@ipZsEgPJ0VzzSY6b#V#$Y9rqmbT8~LsECvY9e0T#2GquQA@$Y~rk@!$I`icCLA@?SLbFM_<43SQT2UzZX=e`?h;md#TsOK7|E*IR~+~H$844Omnrf3@_->f1So(J!zCemRUib zS`@FW8MVPu)+Aj(=&iMeiC7$FPUrE?xgchC+IWe~wss5$niB6-5+j|Rdb%rjO2y6B z`@dx?@fY~7UvptX>=1Qas3eWC-0~UXpKa=J(jp}NJ}~{ZuUG8r7i0<_1!A@L3fW_& z{{l!rn;H;=HGi2Led#$UKT}O~yi_;vMX94Q&L`F-qcfGw|NC>+S>9OJO8Lf;z?1Ly z$7>yUOubv;cS5T$DUH9qbV#H$O2Xc|qT$NBAJ6jfHzt+!EuaQ< zwFA*2OZzxD!kG#5?@LbB%LAu=P;U&%c*jkh)}W zeF;gWZnpRO^NGr_!~0g1169H%*DRuSCNM8qm#|QaV2$jgPG6|2dRzQkV!)96`*}`_|(w9hF0(gZr1%t^W_U-aDMl{tX{~I<-2qRg@N8MiJUt zt(CS`D@JWXXl=1-#4M=}YS)UrLToWZ#Yh!ZVv8-bc7&)s6TiFP=Xu}b_`S#bhrjIm z{(P?Myw2-9uj_+;i3mktTqNx3r(&F)K#fNv`0J3WK;PRP<4-w~neqQWEZCwBa7GjG zIsYQuZx+DaFhCGidiqc7$xgtx{7u-g*NW_+Mjr z;6(L3WKB{dZp$ym1$PZ6w<6n@puo1W6=D~46k;Hf5gt8ZA8Xfk0a(uzrBJR}H8`Ff zF?B!b+YmZfT&}OXw#PHp$AT9rVffn!7 zeZZ*-V|dJ}C5zVTWx1X({e82`vq-69($@P?jU3z+2c}|?$)K9I%*RebFK5oJb0Q2Q zd3mM%^)jBeYQ6ra2XS^W18H7*V~5Nc;=KMfcakX>Up*h?JDb&R`4K{q{3bK<#xE?F zN65^>g$ovHx^q=ltSf_l6TgyMFI{Oq9Q6*{Yqp`ZT_=wOJk34AUzYz3abE!X=HUsx zRX)TxZTWofg78rm#ExMu_n6eH5%5ykl@@Aw7BGSMCv) z!`;csH)b0L&_brc_1?n}71Mx4jlKn`h>PSqGSuBg{FKrXHpxl*^MGe-Lybmm7VCpF zIGY>|59-gUz$*DU=~m@l;asqH>5n5^UQ~H<;)fv}qtef1MPCuqIPF2Lqlh8#3XF36 zTogmL5?o^*BCnv}cl+&?K%wJs){~;Pr}{wk4mUPIbU$#AZva0St$=ns z?|KXVxCM$)|6f+M1b~jQ3rjC4fhi!s_gg`>!YNS~{4MEI$l^J%9wKaIId%?8*WV>-txF7@?ZulEk1;~e zz*O_`+lPL8sl4zYW_FF6p1+ga{RQ7n?7J;Lb)ADBmj*EFnU^>F+~}Ke>vH{W@z)!^ zs^n=C|1rTzgdxw;T&JEBGU#f+Q`ZpGdz0TwGf9>&Xq}5pnMo|o=y6#YoK_Ly!{z5= zzK3Cdt@;?xM(S*yz|hXg7as|5<478pCS>saBj!Mm%r%%Z>^^$6U)z;{$)<mL>8$?DGwzRsytj&PSYc=U}MIC;BoChq%cmS7qO%8A=uPE~NDOxZp2FLSC zk7&aTA})jar=PY|F>as_#xgFXyhHEj73MbXZUL2N1&k3z3L{Z}ph?e9Xl{>4TctW*ZOfR8czCtWT*xG}t%BtjUJdC(}yg z0JA5MLgi`%xRQ+UcDp%q3l6fj#m)7sCj`A$J0Filb^gB^GqXo7# zF*CDsbJdM)N9w~9yXG-dbYO(Oq^Dx6kMmyjqa!Xytf3n35Llwl84Uy0#gI26|;y8gV8^jGSb9-C(H6|EieL6w0AAI5KF&cI@Oj z@P^n6%PCFmUavt_up0J8?)AiV| z>{m6gjYbRc_#mEu{QU*ra(->1nI3D>zh!ggUrrrr+8)n#q?6!cHjLutgr`EA`qfDN zn!+-^I$w|tYyl*s% z(l@G~#?|ac$|ZWuC;3w$q^>T7lQ^b{X!Sa-L?v2IHi z0ke6aF}P^@4bz}-N8RQ_# z3^+MQMRE8)C-AU;)nn48piQ2+?5z&O)tD1Jw)lj1EkUKVj_oGmhgrrx5e>d19YTgJ zH<&oH`ytz_(F(5&%ggb*uCJw9a|vYhp4*{7rQ@Tle@x>2(Hb2dk#L2DHU0o8n*au3 zyk+l*-hX*WJg=SUvYlwJ8qsY{FH{lQ^WPwKGR(qhNd4>M$A}@D(yD`;UqS9z+@MPk!?Txi-^7|a)dcjN4QO5l zEzSVmf|ETq{}OzQF9DQ!$Fx-#!D%~>4tu=9)yhT`H%w|A0g_)?v~!=)+Uq+PJv!@~ znF<3s_{{X){mMoGm_5VG3NYd`F0V0r20a{S)!*+7qdD)<2ubrwc^ONP9+RpW?ka~= z+q8F!mEvOT9*o_(SdOlx9F>dXSlL<*TZ=yBRxakHIEh5z@_gZDf6vdly>~l0GR}My z%#=MfE-$(zPw&sWeL)8z;z>v!4WLcrJzV{mGV+CP`~Y6CC3!9tV(zNHHO3~!JZXGc z^uTxx-|j^(M}>t#`8V`dT*!~?Q>+KxQ-7!on;WlK#t-X=Jlc4zf5%A@+gHo&w7DsZ_-V!9uqEmIuN4NmK7JJdSNlRa`acp84v<~N zf%*>m1l=O<$x7hXLX9gN4n0nLF(^0%Xhj9BJM$zf3-a!-KBl`<8pWwQdDw8AUzI)k zK<6Oy{H%5(;%q+bU13iD2@tMZV2UGyr@MFSH`1y)A0y+ub936+JfYPoa zQpBHE(x-s;TK}P*r^$_j>h3=|*^N^I!%9++J)mZ}>?Wy&IB9>Wfw)&q1>MUo(fyN? zrwu`T3vFGZF2^#5j;xy`KOL?%k$W{DRs5#RUukR5>K1%dsmC9%FaqvXpbv>EQXcOG zr~${?!^&%qzbZd{l6D@YP~6lX+aFJ6raZPS=Am<1C<3w9aZ?!BGvE*p+fnH;mAKx_ zKY3Wrd$<79z&6;Cv9MNc$PxQW2@uck1g@MQ)V_If&xXgrKJN#wuWnAQtvD~2kQrX^ z{>pkTcMpt|mz9YcAIc~;GE2^WVgN9F&y~Mi__soa{w1Vu+oZKtJTKK6r|&w0Qib&5zd`3h4o`ooo+`!CjKMp1%pf zb{|!pn0VUQ{TVdzS3aHIqmepa8)p{RUlq!DZ_E+b*0s=_;)^d8$l$Z9Ch{vLn*TsC{1#+X!XVXxkHvS(k-A* zxmu-L#wBlO5J{X#YpsuKOf!6ss+PHow}1I>tsjz0)xkyIUT(Bz+>#o}$EdtaPP}L- zu~!epzgITbIXZrg`<-Ocw(9kc`shIGZ~Wo?8l3}>I!h4!dZu7?q7m@A3Q{CT)?QDt zN_##mS3c5Ofq2?7kug63{ATU}2vSz>{l}iZx&{yg%myMvR^C*3RLdMaH;KLj z>&yEYCoiG=!ng03`#hU(tJwaDO#m74ox+$?*=;=g^UUVnKR3m{+YWWc&EEyWsIRZ0 z31~Hh)^67%iHYUJ$-TQsQXj)|9ukQh}N_Ebh1%S>)RoQA1&APVbH8^6g zJ#=p$T?@J@yUn3|ccsPPW>Q`Lkm2gb3-SHSGEv~Ayubb1#MpNMI-BF@FBot}?GM%Q zx1=R6-#?S#!2zSiw+23mCQ%oa$+LnkYuC9ULaQ(4#X`P+cJ|-8v2@WoWNtn_I{7r@ zPp~TAWa(xeFRm&F>eH`H(rLTKm>_g41!kD*=xCHbQgpAq8}c3eJp{TGU{)d~C6|%dbh^L~y-lgbCvs<+vrTo-5R-tP0?LSgA^(JWC$3F1h?^ z8)GZq5qr-2*fjAdTvhhDaq$5yH6RmuG$~l?f=cT7`#WLW^84Zua@xfAVwYV0ZC~{2 z0xT9En-{I1&wi@iUr=;LY)L?FX2d%V6Dc$x{o&8BR}Z2b&CN3&m@nUc-WTdpSt3;< z&{|36S6kRQZSHL|-L(!LAF0~&UHt!!sH(AIxu@A!d01IhqDOeerv{FLC z!OG_Vvmrn0LzA2r)wU*wo*eA;-0`o8>3=f&wA#Xx1qIbXNNO}WIhAwQsF z-QQSG;#o82)JC?Y;g63{RPpO=su#u8)x^{DF`6Igqc0I2<-e(Sx;XV+`_A)tMlMIH zOZi*s36GRuuk1X9JvA(=x`HF6zY5mEmpyJcTsqu^+dj5~PiFDdX6N98>V z(?tAjEJEO}L}XNy%gU4C^nBf6c8$?;@;=0X(Vw^66(ev;d}~VvUSL5@R%yc&J)nx( z@^JXU)iua-i6>^F>g0e@WaOYokYztm-WB>%$8`zo}F8+mStJ zap%k}_`ORS(7@-D%E#mmm za&SXpsEhbnBa)7I?mh>w$qV}7wTby>=oOr!f8OnYF3edefD$yNi%HDwU1i~S;Xb3aeLwB?C9zzu@P@BZ7V z3cLlt50Qe_C14q{)!GS(WRv3k)O@a-f>_rs#R}R~TW~gQQCb`TZ?@h>FZDk#+wOin zcphT7=>Sit@@d>8BsQ(ucR|U;X(ypSN{g zwa%#uqE_P6WD6l5=38&@K0pAm9v~t!pWZ z7V}@6b9uBOXjGEyX|dbA1>cUFU@w?)D-o@=As6lzf3>4&I6M+cN5NCm{BvHX$AwuO#$6%ZtGqZ_C?eGmCHkCb5}J)-XC--_L!<;c)7B?oEF#`$+x4w6J3u4!R-~*5XU|ceh&B>7?yE}2_+EX#Ut;T`F~u! zg1hk=qigjS`vf)tb-YM{MTEk@rU#=L%(f1YNw?Aj{WsHM``}BOsXl$C!i?;aZk5&> zpl62GlVeF68N0t?9j1wo5Wy8oVVcP0U#1(ILr@Rm^OVOg^LV7TG;YHSo@6;a^Z$Md z!eI$IC$%uC1FjER*{x&}z8*60VoRGUrPFK8eWE`;*w{JDGibcF5#CvcawpZMG19p0cf?v~T%wTVja_Xl$Q2JIrDVGk`Y zYaM**bI~IY-D=P#E00}q98oN4F^PgMnZ1%4NF(OSG@0_+%fKmWdG{K(4je!AvtlwG zG{>O-yUz~hx4BQk&u*5_Rea8)^B2v@tOnyc*t0b9{k598`z6Zs@9AKy9WHeluiV}tT43#vmm}D~8;pH4sH9lkINqDHI(?hy zvdt>%Fu6QcVc7erpgIi7FcgUb(hn{q%A>t#U-)5y0UG7WS;n>2sSHqv?_0=kBnai* zNVY6b4ysUheu$WmLL`0PLpFJ&C27%d%-o}-!Vi{rcq+`O+Q+5daPf|hHiVluzg7j+shAPE1C6bP?!>-x1O#(hH zm+6yBA3w}E8UF~)IAe+^A6LEI#p387l`$y=V+X`~ft)ATeS=MPU`imTGEab24LFkA zHa*clzyV#n7{#=F)xK9LmT#mSwNo5Q+T7*9F2-DuoEo}0uMEOuJ&;VkL(cs&*i#x1 zp(S6DgDLvyJ#=<$wbohA&{})IMdL|p6NgVbJT91TYoyu7p^JepfS-4nbldN9<&oHX zy@tlVYOPqw(pF9iyZFrGX5JnmEJTT^_pH+Kw0t$JG7Tpu#UzoV>?jkiRGRuw++yf-5d5BTfYb)Z*eUm}w>;n(R+6a!eBb8N?LWJY(HM ziDI-Nn9Lw`@dP2!)dd4d^oOMPmkW|Nn}@U}5?b5_<=_-^G>YT-;Cby~Gw1PCg;ASP z2|t-%sZQE;E_E1isBVFAi$NOOsW2oiv9co!x^zXdIdv~Yg}nU+CBs#| zNXOI&up2n_QgDVLP+zl0Y7a2g2sv&jH=kIUR^i^`Xk#MJlUA*g`T~7R<~}M;_Y^j3 zgo#FZ6zpk3(lTp})u;CQ^M5IDIS*`vLkp4XInAilHA?aeSZ+#fev~F_CzL|cGV<5{ z2=l5@7g?HiFDMA(?|YEn&QqduMT?Yxef1H~icl*u%EPJ+8s{Mc3jLq_*50}IDU0=v z3aihrzeE=F37DJfjQQb9)l~lhNo&)!4oo1MAsXsW2qi4;v?Q|-)%QbkIR;7_XsX5& zDQN{usvsQbR7tYjA!ol0BWr+#@wDzo^CZk?P{Zlz4Z}-erK9J|Lyj*!yBhI%r&eO- z>4rU?K3UnS0%AKgkZQl8;OM&n#DvhR2aFJhz~V-aKJ6y$RNCvjCuU9Mu@-67Y4u&FaL*PPXA)s}T?-s{&OEy5T>Y>(ltvu@~{hsGlJ*HOZM z@-Y_fPB>G>VBY?=zGE$Eet&1p!+d-ixSbM>hZLYt*<3cssD0lhSvcr>8ArN%{;d9- zDJv`GVNNInV1glPd4mDMGj|aev1eeDXo?jX?gLs41m+I+Y8&_#xS2!hCIrPlc_h=~ zr~FjEQ^E!}6ym+f1P{3fzXxyCX%Y2%$yonxVO0KW_zrXKCVX6In~!?Cvt+>K3-HRD%9e&zHiduNzin` zV|Pu9Mls9o+fHz8=U2wOZ&aHzm3JH}62l555a;8Ec`wHD z)E0#HzR3d`k>QL>1+?&j2RD~0y01Tu840DCENmY`K$oC)K<~rO@X0WnokdlBaAO$* zou##%GSR_G=hrg^svfS;ITZ+|Za7Zs6ORWh^t2hp@TbbBgFV9$Wy@@dpAfH;2H~YQ zA6e9HWupRgx-xn$;q)>_0%S!v?a_|;!65fye|wUSb=*P?p~@<;PzPeVEsc$+7d#r?eM1hi#+8&A8Y`izI8Kljny^JJ5de9RBeJyxNy zwgD{?>|^d#FA=g}J;2Tuhfp!?!2SR;kU@9qWcf`8&qV!W+)u;goeF2!?mlM#k-?(vXe z*+ChA4Z_7G{@|iAncBY1%@HWftXJkpua%Vcx7-#2T3=&TV42vGM!TDZ#6$&!4K((a z`*U^w%x>nLAwF=srdeCLv)f%rlYDy^RAFtzO3A5Nl_Hv!HhhYpMsjOn`!@d{&-w)5 zS)EAz`Ex_7AuJlKs)>U~>MZYtyYqqFiPj}A6jJ0!wN87E05o}*pfXZxg>a@#fgZrH z8R=I5HA12s{hpt49-dlnaYeG4;lE_Tbb6I7r`36ic-Aa9JS_tVSoPQCUWN3T{vETwSg5p(zQVH1Q z8g6sphvDSa7g}PW_Z&n&DRoP~h2?OrBfLsBBTR9@R%UH;K%c~uzQ+ceJ_R7)h)plK zhzBGuwVCj%WKAn|BpljWuUr?~HTa5GF9FAlo?gtg-$wgQ^XBZ??`soTS-N-_6jjg` zU1ND%&Acs%YqcP_@oC7)()Y8l?Ju=_vYarSI<2d~Sjao3^9?7V%jaw0p#@!FnH5)bBQGo#c!Zw5#>dC|&`FDv$L%JEILj z1{rC%nc?j^xP^&|I|*0?<0h;HkeJ}yb*k0;{|xJ$M(jSvR7i*_oI@;>)L3xLOgl`@UdBkAb ztwMv1_#ZA$p_PXPdde0pV1}zihG_rKPt(fuSVOai{yAFtgARp@A9&h$s_rJ$m$N|( zrda^K=|v_mTJ3$KIhzbOWbl_7EY*-P(0Mgx-- z21fDdKKE_k{TGz;jtqYgB3lp&58DT01O- ze52pkx8G;F1=US}QE0FB-A2)(#w%_C@T7whfDfx!j6@ZB&8ZI8+czr6J|z7bXvuV3 znPqrNPO1`E@X5p}n$&9A#I9t+BF{v1^80Fhn)CF1_2zFN2vm1KOu*oIUb`^nxPKJ1 z@SVFuEx@34SI$-3a-&g`M&0bybn0eb4D;Asw5>zm6NmeB{2mZHIBbK8-iH_PzUKY_ z+nwnbv)*PFR|3JF!Go%(OG0tet&!6tG$tWP8xlXZ0>&;$C(cqIM+EJo=uyC zLbUgl1c-38D@7FVPWE%9!ry`J>LrI755r<(rwl$|x?L;qq5S>azc^~s42WGTLLw1R zA*<|eQG$t4yM_M8;Fpz|Q%jE!b|E@FGsIW#kSv%b6h zIMa>+n)ikDSFbYQH9=U{3>O9Adn3(#<>pb@RUJ8}l@^~_AmyToLDT+R+&8}vkv#Sy zg_`t__k_2kmqdC_L&s{v`HvUk1g*DJ@KHyLesjF~s|`+6X%7Bj@T)vZoiIvZYT5n8`Wz1O}SQvFR! z=*t3yCc^#C;zC15;wEsd;|*bN;I7MgCR!fR$YjAj<>gXv@Pf=GAv3)L)CFADIb)!( zX4G}SC$*DoUR6#$FUidXyK9#i#}*Efbf!FY-IT%b0>stzp#c3dBduZsPZsXyi&7Oy zVU07+IpHSa;(B=^du_Myp-)D;O_%ot6QKL=PyYQDy~+3E%)6@b(ArEx(?H7_oiS#D(E0hvHZ3_uujF$)%_+Es7b+JOuwx1`*i{~j~OdOe@HpnnN4!ICmL zw0+9!4rRIikvQDm(6|#Arv1|h;;+9F!*b+MBy2*`tW~vH?ktgW2lnvF`Ir4(c-ph_ zkY~`w*WTM4LN&`*=qs9v%RszM&Bkj(WOMt)QpO=GEyf$5ug(8R3MYkKh$1g$u5te*;iGvFIHvE&Vgp#^#yplY`eUI`yr) z21W&J1FDw7LK@BK^@8>2zMU~_m>>KP%xqI>{14Jc&81Fv&27KXc-2upY{u?d!ZWGC zN+-g2lI76)dXb;Ny+Xb>7yf7+`BHV^{*Ka#N3mvre#j}R{jSX|&GSAAj9>M>pAAH! zSo?02grJ6x66alNo(hi9p4RQVXr_;YMoh8mSmw$oJ~WDGrofr3S0>11H#f+7f#-!t z^3&f2;2;F-tOsie5lfDI4l?u! zJfDD#)O70d*cA60uZQZsdId^oG_UTFu>X5*3KeEhR8W{p?Xt=jDV0f0n-C}}VDHeJ z@LsqV$nucNtj9Gg4l(H)Z2_#LzGyR&(5USIM|1v*ipNm4u;Pk|dbfuczG5g-CiMoS z{)=Zxi;K)w!#1OVfw|RhQhqvkOvKwPG4-Ch;#@8=rwn5IFl4}`uBtk8-Aw(7AY{|Dne0e>=9 z4>vNFOM!@bP8wO$2GDKErf zGt7QB=|++~IktW(*nC|XYTG{8JYkz|y2dfiytO=LHrlo`IcZkBK068N9<*Jp`?K5` zU+2K-v?}DiPLiV9%y*dSkw@DM$!?Vm3OxcvQIWyL3guL{@y#Vm7HxOOQNLZ2gAtPu z0XW>;NVa$Vlh;v1B=U;zT}J(pK%P-W575aha;Aq7zTaV5VJFnlrb_Xc*>@XBbCX~1 z;#O4yTD3s*!*1}@=EQz!l}7R2VK5y*JDge7Ke>ZtMyz5M_EvKSwXw ziC?9a`*loXK^}71Ep#vX0t;_rMoEF@A*{>>Cv<+s$IB*y?yhV@6Wu5&CGxb9l9MW2(c?x$T5Z9??W#b?cyq9&KIv>-lmB zSz+E_Amp`~PJ|hO?s*Uru=rJx^~I>mclYb{GHWQ{Nu~}IpP^V@Cmd@y7%S@M)v8B# zY(;6tuYZ^V3zciOf_dd%*(Q{?Z6Adzpk1(xw{C3_6K%Y5d*d{+QV~3)Zop|}ayt~4 zUKF(#)UXI%i;9I!iyuCe-)KH*cX)+?g?V{{@!k0ExGK@N2dVJ%IDf9KJ~4 z-Lr9Ch_5b^ggHKELTNeGnYE_w7LAUMvYPMyJdwvcJiGSuwcqHP6IB~B=dV6`XBk?KSL*vDhF0dHqUO&_&Lu^^1+jO0p-xV$Sb;>K2SluaGf&s-!UWyq+|>Z zWuvxFSZFctcZdi8BDnoZWsvzB@m}97KFFJkTBOTxYkNkE?>(vtNFb$~kgLpe7ao0% zhkfX+O;`Vm_gpuV3>p~XgLuA`Ov-U z)8tMt?}3{fzeE#uiWj26AA3g_)q3`P$sGJ`Lqt(%85&hd$=j4Qw#g%tv_z{=y z=e3Cfo}>MICK!KHL>o7I22LpbF`ObgIPer#Cq;?=u=6rm%*?XrpzCF4L`EC*CqPe`s>q z{`-@z6Ej*h=(DhKSit>;IM;{fp#%?UmdPXM85e<>T4w~F${w5a3K4tG-j1VR-T>(N zcrYl@&MyjlUP(ws81W!I9y+A{wll-^&lO0n>A4BHG@0e)bjwRNmJa1j@=|_i;k%6S zVDpVG__`UgPbAtI=S4^SSSd=pHB>YG`1rZ2WwOWglWuL3?&M85M0{2kxnTQwuq2LD zvDoeJ?A+fH6&^4x@t_!lZ%PfIKjaPA$QfU;#%6Qy2J~KmC>YBMip&IyMi+@PNb2bz zQ7IkV(E0K%@v>HrUrw8!Kt=qy`;uDZAJ}#@MGK~b?;1d(HqV#ard{HAIV$|M2~Dgd zTS9~v$h!}kMH>GR%q7~ut&fhv5|Xz_vC z(ogzy&SvMulr4j%i@BUf9v>}l4x88BjjjhDhFp@Q^%M4TBXN^Eg^!++8+F}x%0<+^ z*?cJOcccHww{J!Cxl!K9eMAxUm0pUsbl1Chklgs@-&Su-cXC>2C3Oc$jJivxYV5xz zfP92UoZjG`QIh-~?ch3#C69v3JINQIQuZFudtY77&wRf;XIK#>NTl7juNJZi`Z zZZYWRJ~6C$m3>8@eWgN+FMVXxxRqAg(nbfft2G}o5=weoflPh~%(mR(OwITE$>Ha!7Cc~5<1qv^x^8G<7M)s5d zJN3s4WX`m(`JvJ6_FXM9-f-BabGr=X+dSNr_&tfuw{yB4C11B}8|~Y_Y2u{@22$z3^!hrdBEK@#wc~#<9O(AGviI|Fp96(eQH7$?lM0-fWmS0$}p;d zmGyGFq>r$u4~#s#eNVpjh{m_d*o@;_6%xvtx=oH>t_F)4M}X&VJ|V`3ID(Id_Jrnl zDDDs|{TU9jhWnqB9S=SupK9BnY`nFHce=CYDu5#d$dIn$9!eo~0M?ACu%)PX_=rDU zP!>Nk9}^GL@m#e$_eH-n8Lv$YwFqS!f;rQ5O~saY);JC{@~`9MsJr&b$#6w4MNzZE zlt-$3)-n9iNpGQ z>n7Wj24MVr5^u)8Lr(q)GzLsMLk1h!D8eHF1$=e$&y^wbn0YpiR_faW2xPLKmy0#zgoGz<3t%j* zw=UQ#^7#f?0?11Ewn;dYMwpQ}!Te+#?fe2(F-V)KT@cxuWd*})e^V_lwJ$#D<_%6f zoE;-bz9e0<^^Vz<8Am9o6*`l2?mQI|O*w?5E&y&F#uwfbX;~smoZAVZ+Qj(bmL$L~ z7I2)ehkiM^=I~Z+1^4(}X7o8WkFDgThF3Rr9)cLoH(wUbRX})|rreB456h9BJGKC^ zboxL!B5KUX)rQp1>o96sDgmc>aZ5~@x-tWD>caOWuD~CGET8%a6r$WhkIKn_3Mygk z@OSXEjWQE@DZ0$HiIR_*=Muu6;YL}BF7fv>B^;@Gtc=;WOI8u~Ev^F~;hANzb?8Tq zf6al54S?QxOn))M_i(l)T+slQm|zt7d6h{b={O11#uHL3Y^0i{<}}~h-MwsA9WOvu z2I=dtO=UzuI29HS#7|uHzYn~?K3i0ybTDvW7pDBi!?hzioeg05`B#{-?pDB@c;8;0vM0iBiFNIq%(z=dzp<6?EqIdDZ?=={w*mYTa$i+O zJh^578nNjSHl$~t_@q_&@-axSeBWnCT}h6L`z+{O2vh;@`%n_&Yqw!h&{3j{kd1(1 zkD_7=#AJOP37dt5^+JN)bi;+VLEpWea|WjC24UJna&@xJw+scnVFy0X>Um_Y0B70l zJ#FG<(57{tcjz?cw|cAJVNc9#iV~<4@9{$C6w;2%N4U6Aq+IQof7) zs;D9w8YRT&<_(pet_AVun|8P|E^n2=J7Uc zHEqw}dYHNR?%5JY>VwC8rHz8Y)xlOlV`14S*!jGYbe|!mS9*UVJr}OZ2e1KPToXd( z?i@X)59MW}e3jjw6(&JMlFv?b-OWTw>f^B4FTn(CHSBx0v0DR;!f-s|tN;(@AieI(G5`j~`dP_)J8STM zv6Fr{bl;EkjSAH0^)Jq`GB&x-+lvNv_-Ez z=nvKMFgd20(-f}dyK!pN>6edZBY%M4zQg8*0$Z;I#h8{xR+ z0?@z>wZxQpb!_7Z0}x`yC9Y3&G-qi4Ze4z$=(q*_zlSe4y#T}a@~8qcGt2+<%hw(r zhD!|A+Sa7@o=*kK39ZYHR{N@M?q+jl6f$Dc(zd1>QErvz&vmIxJ$mw~p9fFy?IRAK zmxC8vSH-88=I`Rq{#-3pLkWt(YuA&1^3G(6S8Sdsu!2@HRx+rRAOmdfri<@;NvEfl zQeZ=@m;7}c`;KxAL!kV+YYelkgx$704^636_c<*mxCcxorsfhZE}YA5X9Mf4yVV-? zz+P{)Gi%venT7#c+|X^gs-V5TyrSMPs)XBO^|iOFR_d4f3_qM zx^`yy0UuS_OV>4lw%%6F>XFR5@yZ~-W{d@CJ@>lBlM4~Wz~jxHfa7PTm4j*EBki2P z`%;9DgURWF*euRd#`sIujEETF6HYz$>;5%B2-GY7dZX67p7MT50*t$k3}j%Wy9(epQy%Q5R#VXYC`L8b_?PkMC`)3P}GF46wV+oeel% zbSl@k<>3W>b4H`3^>?dlral7lGX(!))fgt%`A@G2fc1O*ZtIGf#OG&Xa zukZ{WX|7-}tTm;O2>d2%zv^er0*r8GF&kj!kvYr%HBX^!%ayF{MsXS$Roexwp|FQnhS!;7zlFv+v?WV&rw3eQ$(!^mt(rZB zSwH)_jR`*G*7j!h%zI(aq|uoK4xFB`#O-hUUod&QE=yxSN7Zg1)nM%SmALaxqqcSS z8WfAw&cO0kOOSpo7R_7V6Qi!6eaT!Pk=dCChs@D^4)pO7HJ;E{XjFqsI8x8&9~Wu| zH-+OKuj7GPKyw5wun8X*8xUAwCZ3e%h1vW;8SW&vg|6FMzPRzoqWvOV%njJ$(YNtt^B#2Mk#BO zE74INlnZ@hPUHmsr#^exMCV*nhjJF6zN`f<|Nl3;Cw_(&Qf(Rxe=ubPFPp_>%BUa4 zBW4R2SU&?c|D0(*&yS_G)gCv-XR~BIM{l_Tb>-52I~#GP1;=)wqp~hgP@U2Xs34y7_`uN6Ug~K{)A*r_&8#Q_fl7G zeC@0ER3a};wbK*8XyxhUa+`rI5b0`R0!B6IP+;OL!g52K-(Gbgh{B0a+ zQ0(M$T?M3n!@;3j4y!^N3xzI?{IGs(<40J9Wx-_I$E!bP|I$b5_cfL@>9F48kr;lWionA88C7cE81T}IsbV4w30o}!Mvd?n$& z*ZO#kf1(B81{%qVqED!Ml!glUEa5V|9CO%t}G$jD&0fGL< z^AoqWnv`BW(5zZEHY4ercvZsIs|RJo0Ed4^wB&8O)^t>YrXEcIyHHK;&$lLf=Z}AZ zK~tA70Cvu2X8bq#C-CuKt5n3YDrluwTPqU$NctWk6fPJ{GyPS%3Lf^5K}11WuUA%z zZ#6D?3TW_fg}wH10Cd0iJ^*3IPt>|gB~dydbALyI_ASa~=?;Ekm{6mPUdAA$GHwm| zshYgN7~##r1IwZ_B_v;CSnYORR=dTDIxM$hFZH?_X) z14ax;E0>44)Iq2{&pZ8^Sw2={;Gy|5+`iu$xay(!O$a~?It@<*ZkVqB$vfUVTZ5re z_rLVF2CarXvlL~vD7_N}Wp<|{Qw_Xay``P7^X=PJ^+RBK!KSbjcR4m(-2cPYTZYBeEnA~Of(9qJ1PB&r2yVeWcnAcV;7%jK z-95N9L4y-q8+Q%v?(PuW;al0~?EUWN-200kG(4;4oK<7gs8Mqvl~jfU19`GWt*F2V zbkg3W8ygTOICbjz7I2TWz?65_-V;?EztjVr##SM0)+rR6xA=pK`~Nn*KffA|BwE6A z6x9AE<;0J6uq)*674ngC3^AqQ$5&hV))jGWmgGOC+o&oa=2?$IHp#qV9{>)Mn-w_z zSOOEkT(m`{9~{eGJD>|lcYk~BP-R(1pWHjxn((Q{Wx8lB9QHX?F{xrgKS8eIP+~^2 z0TC#j?lKg!9l{9gCl-01IITnDUyDrUsge6#RvA%#2~mFxZb(Vc;$)V~|9)jHS_3$+ z-Z8I#Ql>Xu&Jq%79hfP34qn+V89qRH6U`Lusw?-{R>(TsGT@fA60z?3L-?_M@2-JU zOohnt_MO4613c!lEC@v*Gf+l&_cIIID5Ni7AfKP*IrUQ#=UXwpPp$<<6lk`*Jwfl~BWxEnkG@Fe*DOpk4JB=5Zw$A5hz=Uwu!cZ|N>?YQtK-+XF0z3y!RFn8lYo$u8p(;PJq>mp+K9^(yi)E; z$}RPgWOz8RnzSp?5SW7%^^etjoe~TM6G(tAH)>|MPiHhuw+NqH&J4|j@sr=x+^%C4 zGc}8JTSfjdPG0iD6AJ;)Q$<`Qb;X8&pZ@&nJI*O`D0UskmRV@Okf2}P6DXn^zj&?& zE?l|L)bUXKzxIgccb}DFdLHL+veSQcs?JR169R)|r#UOx>q4+jWa_1*anSx9P}|CV zQqBL>AP*sLX~^;W4z{4izs#hy{#3ADbO<}f%JL}_#{d1|poay47=Z8>p|>_u!Rp50 zB|+;Al@PS-aml4nulTQ&eWX^KD|r36HqV7&XFP-~NjnOC^RM&4B{u|A1h9(}nRPpB zV4}sLNKb{bjB;Xz;Mu{meZL8Kp&c>&KBX#WmuHz{Zf)aN3GGK<+%DEhdgFl0*Yz}u zvE}HR>O539=8Nd3OrZDb4qKn?#&zY%uKr~#YE+eR_<98mnBXW zMp^GRUE_Y`vLvZi?yIWRDKC6vwwsobuhqUTlcLWh#2{BIA6H+vh^SYn$?I{{a$w3w zKm+2qUn=YVOR$FXa`Nz!5x(oXo)*t1C1x)pQP+V+0+nmnq!FF^_~71wBPE-bD6{!g zSKciL)u_QhE^FA7_9yV-HxN-BDnQ`^!BGi_5RbA*JyX9;(Mx4IxW{V;Dh&tj9EW ziNB1|B7XfJ)yPflSJ>Lo8&&D>25tdQsq8=B6=NQT8Jq(OmcfB*xJ!f$NLq<&w|MRr z#^j5~wc=8gi33GaBL)5pr{yT|r7lWMYS&7wgfbf}6O?sR84gMbl;$wkQxWi-j9CW!w9`K9{@e)F{CnPxnPOSy19GFu?;H=`61=pH?_3Z5i5x-8s-t;Aprl)Vwtr))`NjC*DsjT) zJNTBg^y8-WS$^P=8zHPZ6ZTzsq@3Xsja88|7oKC{HX#2nYns|M+?s(bZa3US{F0Am9gfm1f2=H2d;M6WbI+#m%VLddcGDzZ>OBxRT zRLT`YGK3ibI*YnhbvFT?sfY(WBMldspwO@2z9_)*~eJnTLYS0 zsN96he!^|-*OxO3yFcoyX)S62X!;45*dxLG^4hu4bzGrNRlxX{9KXA9TGhgd(&a7S zzR2V92WY(8>uZ{yE#bXcQ0}>Srx%UgIi8e`7G>hf3p|CoLt8aP-oPiFaQ}ZaFagVH zk@uHvH(_o`laECLIkTrZf@8nLFVPFonQT;yloPiOz|Qv}?peo-RW@z#OdnFSKCu)9A3O)2m3iRc{`)av!DLw1s^|QDa;qjut-m(D7;1sHJx~*R% ze>e_)C$G;NKR5{$q#J4mG#PAcL|MOB6%ls7H?laW)H1lJXx}&AFl^7Yj^uHl=;Svv zHt-)dyZ@}(-`h!@Us9V=vvK~SVN(Lr;Vhbsgh{wCWhmpxjJo0&+@YH1;-2N&_bVd4 z)JX6V?_>zU6xfd&OE3{G!1%> z`(#W(WeZ`c*xOwe_~c?hNJ!199a4IBxyDjOlw1kjWf8qBoSjEODy3qOL%2|V)14)T z%_?pnMSttsqmtWQtlU$ae3-EF_1x5RbN}%*wp<5k%I-wN%9jVZy;2{it+Ipnc6~@H zm=M{^KjN9532TO?HuPH3{73>Cn_B8N?O+R`su5(8FLTyRA8&X}VMNjToqZ4f59Hr zSDfmV)qQ1=R|RDi%X%ILqoGCd!bkehtfKJ37a_r!f=Gowb~U+-gfgd54@+h%aL7v5 z-1SztXOkuH@3%WSd}gLOjwSDo(Q`E`-U7`eaFQWnN6~+!c$PQ$S5sx1<2sXS{db(rWz^u5cfqXG#V4!9g201{(82GY-dwz|92%L3O6}Lc z?n)rZ8f0!Vtn@91UZ_k*q~#=E1fS8!J7a(#$b@|bZ>&rS^D0-AB9hj1BBvXmFN|Fg zx9QaoK9XQF;al~BkIy3eesnzj%!+B&uY~G#ynzfc>tSRQl}of=Wv+`5r->41Z{1hY zK>AdBen~ns?$RT%WVzZ@P^F4yzqP|BO~CUy6U*ZHLIRwH%xG0iU44|?i@ZboA~n3H zIRqw77P*17w^Fes*a8+PUlJ5=h-$-N<(R#K@ zlymspnjx(d_!`6z1&q%==;h<>jRXHXJe0(YJuQ3zZ5GZ=R#OfU)6JsF1T-S#hDX6~?}PpRF;kmu_cEAgzaltqZ{jK5k@U6B09eOs*k#3iLVSbJ@U;KW z1Yt+Nedpa)@m6_y@JtWRH$*Zx={B5Q1mFlPWN3W6c&{3$|0+(FldVE;Ic;q`t(tlLOy`Xe>4q_sEaz zv!gz!-mwJCV4l_7VnH`w{NpzSU)e~+waW0tpYq*h!vp3*wna?$*k93l_~>wS685~@ z-g9R8LsBw?G?#1RBpsKA70_*(VbBx;XlRR`+?Ok|4daVqZHpcf!AT!gMiPT2ViS=E z+1c$aazQE~sB8Ok9-+2efaL!8p8lvZiqsqNm%FFMnKE~jP6{H``XK$`;#~e`3YrK9 zkjfzHyW2vMyhx8z6>z2*`t;d+nUb_eo=EqbY21Q0?j~SL_C2XMJU3Z&&!7mwJv}^C z%K}Ql9^N0)nljj{66IPUTGgNAavE%;r`hBEa|qorCF=z}XoD=y^C`oZ<(AWHqPAiMJ7#$M#B|--6f85Ow2%H~QS?tB zr>c3^cCbl>=bIVzl~kVM0A9LO{%y9=xhd%K>6*x#)9xkWndZQPHQ7lO3npyQ4EO4>wtzL$0KR@Ow`PLaX8Rw(RqzY~rrsEES%r}O`uC*WB5@=>oA8~@a+ z>K9Q!u$6vt>MvXnKA#M6WghqypgmXEbS@mlj3QSqV`o>heyt{0oSv1OyupEju8o4g zr|}Jj{!2`R)_(Py>9p_5H+}}>oY)wugVPnp&MI(>UyZO_r@4GB#_p_#YttPknmnZS zoKW7#&D0?O>T9&owq@F+pB4q#y{b-M`xvkWTyRE5SKtSy43iyua4LDfS0W_ z1CDVW7meP*Xdo;=Fn$yJ8&TAE(PCp(hCnggV&e*U&oq5jI1}O93>xU*vuE*zAWZlq4|)8Ou?Lo%HqgXd}8a zUbY9J`~+ol$1Cpf?Y{n_ynf?*F6gh<_Y;TF@nogNytGjn4?^u*Y{ zw+ytFnTgH6a~!j=oO~dDG>X+E$&%d*Kh{xWexK*RP4TLqN+UK#lnRY`3Ze0P55R*^ zB!Iiws=JU(NQk!4xB^C*mu8G6ia^6)u!c@ZW2rG5uJ9TFMgKx1WkTGqVizUpf>P&IqP_Rrw%PdjYpA*Iy({y5qpUbUaNm(Jt#Kvr1UXRmz`L(5yy!rnP-H~l%*x#n@FyGvRfIOZ72Y*OrQV?j)#aZ>?e^4`8 zG*;zdM?f_xIydqAVSKZ8&YN{2t@d{yv|D)r^3up++KdbhlEQ#pR8YA?8{}70nmOpPdmf%eT!+ zhS#7jgHw^?FPv?GC_8MB^YeU5xHw)Dyg#thB<(!ztL&z`+nCyt-;v2q-^i-wdLkab zzSWSEcRh$yLFA}`wV1yTgToSd_Lr43ZP91`mQ(zxi|&?bQ4>v{1_L2nE&+eCI5?u|AIw5 zArxK+A7$zz3veu>!(N!2{`nTK1KWab?UDJy0$v2=M{`@A(bG7*gCIbgZ$}k{?U*C| zi*I)qq)*(BG#GIiG1i3^YuDllj5~R(-}`|&=<~=X)g_Db)4w@)d8Rz$8TEyx!g#V)B-q z*&M5>()3^IPaKrKA+Wa_p}~>5RPrL5$jo{hsPVFf^HO(g_mNLWBl}0u2MI;Sps_nk zoF7NQTD*`sc)^kNe`f+-cuyJtL{>uFT>qD=Mz;B3i$wbN=B^>;cpO$>A-5bIH@P0d zS30%{Vo&Pxew?DV`xx{xJ3tfhEA}}5VU;cV=4Yi3>~0ThILnWJ%tED`27@RY7P=zN zrUQg6+k>T|(o&jeRGw7oSm6{X|LwITXB6#^aNuQPVE6i);HW^vyT zw|#x8pn@6UoR!*opbdixr%~6^IVv9`-ez(Kr&{YTcGXz*+EnTP^%rEm(iPO_aRKf?1rn3&WPYQz)ql#ehB+l&G4`7aFA%z9JqQ> z`R8_fMry%^o%`AEqhy#*5#uXcvZezR*4Gp!5ig#|-~0Zyz3DaaEaKjJ-aW<{-UbJx zEq~y4xk)gT!djl8c9tbrwipyk_n6htW}wfI@ZW(cop9c5-rL1qr*224^wbbntW^yN zNb!7~vR~6l{pSIA;rUSXVps<`oENgjy)_m2(*MI~iEsfB!Q3-<(f${~{Yyw-e!$`! z{k-TlmB0Rc^0?fywRo3i?)1SM#_cq7(*xn^e4yYwectVyNr z{(hqN7puKdbd;#(i;sfRd${*9pIGXyBc-NJsMs(aI+k?npJs8hk^h}J{x^aA=f}V3 z^}Wd0!2iXk#5YrU}krqWVxbMp%eYztN8EW@}Vi;#sTnTs1+Xb ze`-VEH{~x7Q(=dQ{K>XvE-zbC}XfmP|K4_K0Q81 zoH%aM(>affgv}Mri!`30AI9RMRhs|S2UW0tZ(d>P^`$fim)YKV3cc?J{fvI>u8JV; zneW$jZLg-A^&EP2@A)nEVCvcaVv!>$X+LgF|NJWDQ*UUnJR~e)(ZRs|a`dJZaKbI%(m^mC?N;Nw3% zdlQz@g&o2*D6JmWIeE4#2P^~cHj3zh=i}(t>FBLZi4X0eMXw%w;mKOZ!b{7c>RFjj z@sptF%suqsfO6*l^X$w7a8LwZrJcIHB6v zU)52Odd}Mo&EANUs}i~1s@>EL0JIq2Nza^_V#(u9G>pr#S6z?(;fQ=^!kVr#dw$#a? zq_8sR>1?S^aia6<6+!qt+A#PkFeh>2pQn#(W2gSp@GO|5U#Z1PUrx?BC$kc_#?{tufV8VlptOlHhymA!Ep*#bzr=AMRqNr74~5!1gpF-rnHsBP6|tsX zytqI}^pMOKkc_4X&ECK&-@6QZQk`Nw8Dg#gSu)A zL7BGVL++*^d*-h1utmdzF)l(L-WvCzocSXM5W;pZ<$Fd%#M%2~%;FAs<%c2sjU4pN z{m@{=fi5K`B=E=w?rW5yDv++w6C%64=*K*N0AMA%BP@YY8D5|52B`VwRi6G zu(yTBDjqj|#y9eD!zedUFu?lno-S$d`{wpL-|w{JFi^LJ**;_S^cs5J{9u8${^B6= zZZmd`S_`g?C~Yz88ZMIZ#qh#foM4a^4mH28*Col+zmB`_iKE6s7u9=Hyk+CXUs}k;kT0+(~@t#>`xhKdvq(f-k|+Pwhk*-e!lLp$TS7-k+u}XJcbU z&%K++(kBx$X=$)SbXnHke~|GE$ZPDBMg8%=*=FrYO{y&0&Fh#Gx2|r=hNZ$RLo-aZ>=2Qt z@3r~y=eje6%7t-8$J zDA8X{?|m1OUGnUj8hE6Cjz#b3ba#|^oR{LAFuh8S%x z^EmY{%R43N*@2v#rS$wzJ-A>~s30U;zL7##&+To_G0o+P0NnwcTVxL%Z%9RMdg=?} zRj5$=I?XnVjqfUM)c%3MA1?s}zVd&^@_G4bCi`5e(3I-=JZ-}D#JP2DWGT20ync#D z-nQ#9Qba%0J1615_z*v%=+bEbZz&ikdsAoxawRQo^tf|-OO~GXx#;!W>z&JU0&SQ5 zYI!4OL{rec=(L9Gq-`!XK7*%ev&4U&p3#Tobl8ZDIL51rVr*hj z)DAAh{KY)PXL;Ul0!y>MRPQ~)PFT5Jn)FQvyC9c~$0`z+_VZI~$xiI;<(+lvoFsil z5TtuWXoO@6;_%&%wG7pebBZdrr{SMu0_4Vshh? z3s@?{klxJ#SY1ScuxFp&otEqg!(lAUp5t(UKO->!4rO7erZN4F+m z%5#s;fLnAVEX|y6t=_Z;`dXZYGjp5R(u!5sG~HQ8>pJ$*tC`e3Y!!=|LywvK*&)+l zG?6O9UutUt*=uW(>~raOXr~%8F1;V1@?jb2ir(GaI88KC-ozw#Win`bYEo+t=n8!q z=RD#v(0`qL1rTYo^1inn5rI^`xjas*_LvmLF2xFtAi1NXs$}%s%n_|uPFw+?MT6Nj zjJQJ%l6p?;T?K0rK+qc>x6$P`Cvo;zfgErV{CwV$(Iwk+N4;L*D(OT;=4QYxs?o5+ z)PR*cDm&7ff|#OFKMh6jEW2b!aL<41#Lztxo7MBjnAY`vyqwl5qmKLO^v@=W{;hhZ zH!3X56zKO$CU#OXQy3xjE6DWcM|oZ#EuIhbhRHC?yC$7W>9`p^%;GKQ7B(?MPFu^z z?~`im{bXz+`t;l?lO-#CZq2d*TRiRQHUadt7d3l_{Nz^XKNcgAts!C@OJbG_rt7y* zQeOac_jR`M^ig&N+g6({B^wtTAKXv6v2sOBL7hwu{YI_2LT>zBG%Jb%?4yll^hmc? z?-QCi?M9{Pxm>sIvh#m3D15T}w4W7Wmw;b=HmP(%xmM3=E3-PNyBl{RNB*f^%>G>(``7O4uQ!!MLgRax zlDL^;J9C1zbdbA&<)eTIjvMXII8qNESuK|K z^rTs?mL0HeE>KHkBEMQw)d-C1`2ADLa7qHSmAPwF)6m;xs{O6H>hw}=v3%a4rHYH)~COEej-?r|^4sYN?J995GpW<+p5#{JHzj)u#EY-iw+6u3N z((e9Hm7KOiX@HuNaUFMMiO2C2-$*6i{Jj}vCMsA*{!MlZXm(cW-in|Dd?r(0Ce+qd zO1gIz_v5%ZvF%5Z*2+VPZ43bG@Q1I^Y|$$?UQC5K{5x+Eg#Q&EKD;Paw-8G)jEma} z8W(=hOF?@T0h2Jh?Od2Ai*ABLt6`~ST6?10gb!hx`c`(`T*^K`rZGUd$lfun`Q&@~ zt{FtcFfbzX@q_JT&}q*kp`es)HX_9Jgq&t@!RZ6Z9+@U8hXJK}BsrGV`)}F<^Msvv z%rx-09t9IWwBA~omVJz(IMFU#t7RGbR4d)As2NR>2n}b=OnomfEXMUJhSmTiXFIV? z%cdY6{Rp*qmNhg6Vo%Jd)pB<{rQW(|Tv||&5&7Gz#6dhaR6CB~9@>7$k9jmt;SVQ^lIjhkZykWJdTYR-_@cRA5+DyE#jG-|!c zHv(w9GWF1VO}XY*RUtW=as>0?@MBbtGU~l!xi;GDqF@38LUub9R6563$0!)bPhMM{ zrb9(a{@N_fhzGQIE7?>Jn=Lr%rdA}ri!-6YkcXL-}7Yc=^Zuyl)OIE|&xv;s{o9+fZ&Potn9nrzx` zrxto>3_7-+B7XZh^TAUnS~U&H+`7&Vd`?;mxt4FzuiTv9MT3ZpDr-5YP+HMkGb7>_fiA}_s zyI&)>^NIF$pYtkWDHHg@epgmOCFf5593ApOH1C!TIhAa9t(l5CYFSh0?mM2;3X;=W z_ma(%=*f788`Zhglh?KjMD>70Mu=!3`7ptSF%0IGEH1m~j)9=uTV1dWQOBBqO6K*3 zmuheB`?7x^KwQVn4RzT67ZnR*IWY=kvVK*wRCzuY(~y4c!hAbh&m_Ttw-<>VRM~kS zy=b#RV1Rt~&`k25g-sUw*TU8&GG%)X*{ ztJ^f(x-oMHAyac#?+Yb_4{`2f;n{>aDID|kEX4|6mwF2j2>CY?OrL+oA)3$GoGNS& zewv-@{2=)QLrizsOVwPUN9g_!2a%3;9%3$k$rfLn=^P_Yf0rk>RxHXem=5kdOcn7v z&CGQv`6*`Am-9NMD6iW^PU8@L%ePII4jtqOfmPuPF6jry{vS*Xwydz#whUSJ5yDQZa@lCqs2Z4!kwzL2G#o zXW4RSNRY;`!-v0zvcIu3SF&>>Qp6eYt*`#MyyC9Pm6n@;fEx=GCPCSH{YL8;Sden7 zxA&mq>am+yo3}&Y^z-g9cO7!dO4#rh0&U}d9KW7L;1UM+i%gQUw7u!j;GFSqYI^HN zp!RLUEiMkdMkQm4rh~T=RZ3BB988=awr99tRh27z^K(T73A!xQK6cLKFWP9(;dtuS zB4YV}3cWwX^KGE zm%^#}#EB6hRhk#*ZE!Cu-Y{`|HT~Yd?D6~tTbaoCd#UwvS=^#MQ9aUcaA6!=mKSA! zCj)5RX>)wzfOfsuIIMR~^Pe@=gZ@<*5gLAjxxNgpC75JFh`D>Jeqm)wYY#jM@tTvv z&wBYMe4!^(b7*DE)4TA&*Ju0m$^gW(z-z1()Gg=GbnLBQJqE1nYv+fX;O=*|!d2Eq zjLhm4S)QSuJJrRsLLVdVjAq2&`e}t6-D|qK4)}`1z$9Y6A18vga@#crK54Kk6?SU! zPoJgeYOSqU^ztYmg^4?*rRIxc`ZFq2rI)h^2;ildS4vgtHYHOuxcehqy`RL(bjS{$ z(H1JvXWJ%Wf}4_$c>Q8=Lpu)n^!qb+2ji>=z)QS#LD@?+hi*;hCv-#d{a(o9ceJHQ zhs$}SiWxzNKeT-w@g;u?==OsR!udTgn;(WMV5RRWqGKqd3qzIyzCFoa?lB@qbdN%@ zn!KDwhjTVFE1&=qF%bV8CQd_$)(xtRqpeEud?KVFW@wW)Qv;Gos4zYqz%xIvo|mq- zzZIrowEw`hVt*ep_L0)j57xAihvS#QBCRU-(P@+4p%w=(HWFFa&CJ|x(+<5tZ6)}> zJL7YIYc#Cc%AUz;sHU~mkCzZ)0{rL%Dtt}3xxSdd*@;M^k#h)Ecwyl5HU=kcI5AU=LwDj@qt6^i;4LEEdg$7Q!yp2k0tmDnT8Wha^)aoTqcWN^I@tVOMo@ z4j{RKq&?XI@I3O2;BoPmJY_1OkOt!SVp_1_n!iDm=h!esX&j)kZ5S@8bC}01HLf>6 zIehEXGR5N_Yb?5WCqyaJnwE3XG)ux9r`a7yfEE+AX2iVAMlfFS;TNt$+SvF{Vbty)+BpUJ|5LI{ZifYyR7-zXFRfV;E#%oxlNf0OBLScbNcZa6sK^@o zV5&32rho~#yv(+4}ELgnglyHNF|nW|gXYfUH(JYC5h$ z{0dkxNb2M3%#G?b>5b$UctDw0jgVc&h1=~HOHcewvq*irnL9DJ+mgVh82W72K zZU-f`Atbkg|5QgAOn|oIBpLl+mThJ?fcA^~0yw8&hR>^7iHg=bldu5dSLQioJgU=Y zP#YyS9Ns0GP&M(_hd>{!6!j{8a{0R1TH?}gjqtaCd?N#p+yRXfHP+@CqE-djT>B~C z@A3Fd{CMbVYaVM4oZt#)_2&e zcyT3sLVs?NO+hA7H}!UjDM-*zq@&{sJkdjA(3SWdZnPt>n`^466X=hAZ#6)X=uANvL==wyX6@>ze`UHq zZ?K+c9Xw+U_a<5NaPF&7WE?vr?nrkc?ICw}*^Tsp6(&@^Q!pb&l1d;;5zz1%P#*?w zA`gJ6Ut4Y}75*S z9Hy@e8h;O}w)y?A(e29EGhV@1lH9Z==??a(!$k5UEoSz zsL+RJlp9r-6PlS*WO^Xrj0Z5A5E&DcfMV})dC>Xrf;v{W)Xzh4Wxnls^&SH zUi3Y3TT?krV@(hImt%?=*a-qQa}SfBTm^LGnVOrJkT|jE zuE1*in|#!6R75qi45< z)J=$}-0zLqh%+nV(iIPY03UdtOy=jR!b!u85~^PGd8v6#$edK_y4K{R zHBJC2rHVI9Kc4A)Cl(ltZmX_Q0wVmO>uTbffCr;8TL^#pO!89J6qKz*Pg1Xpgr$uf z){bqbA%~IyCilP%F%!BG&IUU?GHMS{SRKYs?`4+hmH<_6$l(Ns)=686Ndpes4UBNm8h*P z;z{4UK5~g~8Bud~&>nba^w313Ea5Y@9}>mG+tG%@Pa9AXaR#N!1LS!0X2yUhWqV0aK>{(i zBB^4~3(Hn_71syNqI#sk^9T4WRT98X;2d5 zST-{fkY-rW-Qwv`s|la#vvy=<=f5l>M{7p44eM2K_o>62Tkb ze#7i@CBvEAeW{jIXjcsA;bfYgz2Y)-#;s~onvFXZQnB`W?wTm)N1H}{vUqF5)+65t zEpf_c;sH{FIHu<{bEiY0l>!jWiPPIHS_wU$;w}H{i1Ohy-r{sZLv`w07OLXu9ghp40_NH!jdT2x z&3wY)tnX)SmX0WcBID`AOSzT^WUd2#i?& z6O$pdL_C?~mY*`?@H!((QQ4q!@{7n3*gZf8Ffq7WkHkJ+IbJ{5Sz=}A7md0S=zWL7O2iUiRAz z+}vdv^%VcjfpcMIf=8sII@MH9*HA?7YAU z)MJS$t(J8BG{fNS4>(F2D6fd&YD`5$Hp%vPiSU=t_lueNx(SYiDJu_*7R^C}d$BRg z?BvX*IN<^L6J=>dF^7bK*?-;~*O~i?(#+Xxi(BSE%mJdhweuG8?jD#Vi5KUiUGx*n zDeVq6>;xifQep9Ux~!b*2tTm{(+9Drgs*Z%vDPwS?&?LzZte3-X|D;X(lX%9&=QR9 zm-jFRcd!2T@${)eZ%4U(y`yk43Y3G3k8lc8$S5bz# zIVYNlEydC@p?61WO1w5E@|jU)2KVG$Gjhb&rjaH8<-t6fFB#iw_LO(FgJ1*d7b!(> zP{Eq?7hH`oVdOLPWRo{OmQy_4W(!$}l^S?zZUoA}bVXol%z&m`J8V}b+QPc&w1?EH zdMt?4`X-jJ1BukK{ORtXtajNgtTb(9RVB-#0PG;Loc?Jz54SI8WOJtqxc2%3Os#R=mfg zUVGIJeAU(&_ny^NoI#7?H=*5UIxiyDwDg@Sb(gZt5$%L0@!%YF*ZxxU9Ap$KQ-a<> z0uB`I$RTcvY4`y!pk$cL4zY19u&&&!LtiYbO)FGke$lO4$6OB@_X6oY3QM^pGEfM# zI-c<<-;R_r$%+u@-2Wlo9{iIA`+Dj!7-hEEAc5bMvac;pBK0O>Xv5}Z%Yt?3|UKm z$Yb~K>Vgao&^mH6%!tQuL(RQSb`xQPO?!Y$%}IH3r;`uJ4sm-924&gbd|*bmeHQ3z zO&Gv@hVY*|6C~3Ml8f%Y5}j|1IsNo!WUI%4%uP^57!$YlAW(^+|uuq|RA5ik)bEJs^KyXd@AP5oZGzWue7R zElUysEfbn2c_Hj)drb`te`KRMJS2^PvrwXtXlobNPv{q-FM<$Tp>WJj5)WLhxDligM%^F3>V~&&kBV} z?%y4;ZB{{LqQN)7Jz+V66EF#mtve>E7}rgI=QK+O*b2!u3rsp*5^^ybhAnt~t}^=E zkzR!?t*1bR?eQTn9c@Hm=k3;-~YaDtJFDR++e*W7qudzaH9VJ({4>7J`JP{2&lL@yH9Db$%W|gu5qmWVdS#ti&oa7CDR{4bP1X z1@8P#&5@8&k+Y@iKb4DY!=>`+Hn&<>mGd?1P@To@5O{Jr6NKKOGnG&QO#(qDBamcs z1#zdpB#;g@m|yuUV8(?8s7*}ZX@(pU@@e&oBS7*FdGszbYr|uFxiNEdXla(1wcN@zgT;b&2?Dfr6_W@Qj~b7{UHCQFshJLkO_^HYu?;Xs<_u8-%HszbrTu?`Nbs+;>;A_3@?p}O znojx5pNK4Pj{FgdKCnJMn2LBthC?gmq@s*xV?0|Oa3+94Bb;^Hg>Gbq%B@bHwV zzf^cS&vJkuaisA%;E%anBL!Z5mYKbAs%OeE@QbG(173oC+jm$ZO114tt&-6cZ|jdX*EG=g+@cQbShNFyOB z%}@f;ASvB^H@?r~`|ab{`!_#;weGd9>pE-sd^kPtpv>1uXk0@jc-8>~`E4?F#Go64 zx+V5D97JrGr1Z^7fAOJ2^gL&dz_ULog$EsYZQE(ZEY(Utd2CAR-CrlT1Z*5#WM z(N@-|_w*SYZyCA+Wicq*6*2UGDHflbi16Q8F$^tiQ)A zlVa4Tte&(**%g z`B7!r|7pY>5lJ7<$+POb0%wa++mq8#8RB=Mo~ED~J09Noqz|(EKLJniYneKymf2G4 ztuCD*linop8TA{ZXY++FS)5aaJ!_Q*0}Ri?LdMM zZEL6HU2l|3WO$haP<%MfIDLQLorm>N_)P~rF&{f!*@hj`O^Bb9`GrI3t;J`ONq}?u zn(zg)(f;H_TZyK(R5Mp=>aWv8#qvUehpOh6;By4Cj$ymX_Uc{WX~--XW50%t%$hMD z4hth>mFZ=`M%M5AofEVc!T+-vh+Eg4ZERFQR_wbi!s;SnvQM}E;4*G9E9A62Q zlLth|!B5!O8N5#S<5?AcKsM`L{Ah`blmtn8=moI^ySjgnG|gHP;H^pZ1_5&_yMV6! zF*e4pfb{cTh>y>qoFBnZquAG92|Fa`LZ@Wb7Yi6ku-ioknPQ69o27idb0JnA%rE7v z1@r2XQW-rs|9AY^#(f+O{0F{6)LuOtFU#yXk78@8PpQoulWlD-CCLW`N(jP_;fIRE zNCE({1*mGk5?E-@W?-fr+dDGKZ#RuI-EH0Lv_oYJ#A+Mwz}QL3z>%^)8AQdRTDC_* zK-C|L8DY+YJg{c#^ZADwy&j@m#5rH`0V4Cme-b91Qqd(s_+a|IUe0vhe0`_F!uiQhI24s!v{5**oBc{3jVM)&ZC zq0Xf!v@$Pepw?n!)Ph_=aA_3&%BtoTHUK#;0Ei39a?Y+vD?7R;k;*~+5QhQ}Hbuy(2vEw& zn_RsnQ;ds8c)3ZNJR0rnZPdvuGA1(4_kFouJKqG0MgSsY#A&|!-tF;V>u{@>9L;;c zGE0>o)2|e*#FsH8^lk<({Y1MnWEkm@$NLsT^#5HYg4ujza9)Mn-E$n`5Fa~&`3a25WW4tHDA9V4E zNX{#oTr)ET;m!oOpQS%=qQ4%3STl_a*D~uUlXW#}5+G&4Pz0FzKiyI-%RvC4E!pQ4cNH9xZlJBO zS>viDF=n|&4PD+|bZwSChw5Z}ec{csPa{xkcr#_2q4e%^Cw2NOz25TG#}*0XyYV8u zm2*T9V_MCeme4roy^NHdg(qo3eITpQpO3n|d(j5>Pk%$6yUe!bpfXqdIvGD7!M1S)nF?{;d|$JS-?#VhHr58vm!4@UE?C}~1*4)?&q~=_ z;tUyE71t&nTp4QAf8cgSC|ji6gR|_G2Z=fZTAnMfyML0r>iS0!!rnN*mu=@y%F(`) zgnv=$9@7RC2-?dC=ihiy3yoMj{Q=ne-<)sq5e`4rPqyUFI$X@pTO zUkd)fW)zgo!L7~b8Q6^_nI!|T53Sb7a-}g8%asJ{m*sTIMfH}e?ow~XbER!0G`AtO znRkJNQnl^q)hEP#jFjYl^e7niclZ4>Ws23zdy&C`%Ur)HzWeIOGFrg=-zZzL{s? z0gm6$q1St~1T#XOpZqpfh6TfJHgku7e3YZQVH7LhhHWH{PyRezr)Jm`IExal6m1V@ zHB9|xv3JLQxlXiVqKJz_{OQDz)MxRVC}L>o!`iyyGHgx~sChW? zOqL5f!p2%pmK+3J%nvWlKQnw%^K-izMm+)IO6=TSVVaB|n19kmmLpbIO9i%`^SfN2 zW>v^k?y_Zz;8K3F8|#PRz-#M06ka>e)r2ZlNJ)TC+iJ_Y`NZ>SxS)l0Rl)*{`U;r& zd#Ba8F6-(D8F6Kko&FxCU66qg?N|iYXmsPtN3J+9c$4A3DigoLM^y58G|tCoApLHP zYNkd)dpHo9(L;! zL<^`2;I`!sgO14{!=>*Mbkfc^=kp$Uyw1P3(kf<-fkQue#Ggiq;wFLV9*bw(2%;a* z9w9!H?u8KX&Cw4g@Q5-$QL=rOt_-uubQL}+Xxny@c^TPR4L_7cpF1(SdAho7sLQn4 zW!6EYTA!q`V9oc2x9`;wu?=by4zt*URWJVt1fkxv!$UeX7G7(3>%T*EIbMkwcNZ6a z$3TINi3t3GZ+Jw%(p9dh_xqG`4}Rf1{9HeI&HrAJ;G~Q3=XQ6*iuRSKYONs#sf};K zVzma{jTBXkd+pL_{U`C$y@c8>oAc`LQHj{ruRCU+$GFWAi8x#BBpZRAMvW(Jq-2Q{ zS*@q~$1n3YBHQl!Ji`cD4-lwM;c zTvv}FF#v*AO$s0coEJ=8pMDXDm8sFlHkOxgHRgha8vJ+~P;o2~r$+>*M z6Bkzup4$(PE&B%5ycmMOD@mbH<59i|Sb1O#^V>9-*ZEFOG}+e~M!~XYip%w5u^5=9 ziy*lc`5-Ex`_F_Kfl{<#TXjjy9r~}nK_DKuk((wlQi84aiF1`qn=iIu1;Ds}iiZoi`hMutCCx&#qH)tA z252Q@-*w=M6RdJ5qJQ!cyM3qLq( zb3*OS#5ZV4&NxZvXVnlEb4fFL2Gg1(2R*;gUIs4ZGx=vjm_&(EW{hD-RJV*Du|w$V z$uwabsCk_sBIaqEWC%7_xs8gQ7KG6iLncqeV7qp{JpM!9Da}_q5i__g`$L?ati(RV zmjQkGUdxX!v@wyLzu<>QxThSlwv~4t?zBW~H((ftNU6hB3jL-rQ!cUoP;quj^HDcN z940!@H`wI+quoRxGWAI$Rk(II0eESP*WOyyc_)IGTTX?}5a8)+#}m-6G3`)XnL?P*97A*<|yl}=E) zRXSgNi8FH;BNJLW>514pA8G6?mQU?V))%%_L+no$fNoTTC2`iy?DSB`bwz#w~a+1$g zErD}Fygn7_C_n;enD?YR$KnVMJr^%uS+sxLDOl9rA)Z3w!jd2t%-iBA$tG>O{QZYw z>;NY|9VNA&H_WmJ zb#CSC#S};FJa@!LzT zPE<4N3>8r-nAui`!6a3&gP!Ux(P&s5bd(W{P%78v<4Hl1+&mI2J&Gg%gpjdFYD_TO z2qaOHoc*qefk=&)yNxuF%J{8?(rSH4hlLX&8VZK*9i^5|ts!k-$mG;Lyz!^_xx$K9 zuO_8BOp019(tx$cnhGhF5wMaaH0OEnW)Zu#5#gl``t%~g;MxNA4|2;7>2FXd=RjT)Q;fV#wUI$o_#1D_%bC%^ff-*?fM(=!tb+ByAB5c1UDxBp zJxZ(4bgh~t?NHeu3k^(!GyO0KaZLf z=)5oSKlX$gT$R(&e}1uIPnw(ppmWAWHl5lQa+>lurU%M70g(Lfa~? z*GmPpS}uD0?as!_Rz&|x*ZYA#1C9(GY}`8@I5CHkZO%UBt1~n4B@;?zq;+T&t&N-- zU}AS!z1$|Z}eYqjW8k73){*|jA^6^B*nCwjLCwLrbt&+>=r|8Dj;z-zLJskt4 zeTueZ-L-UyQroiskOUiH)klH>mXPEW}11BkqE8W^<#l?*J?}N!$l~pV* z#+Dz8c;ihw*8SgYyIIDM{(cW~ucCpmyRE2`?ezu+XiTQTWFp*;jekhCGjkc93%*0I z8D9AlPO((qFT@J|VGNQBfUj>6QEa=Vm^jVi@bEQGA@?u#& z&;2>v(^hEBwEjrMjGF#Lfh~|<_rtYz1F_G8$BjwB*xg?9K#L^x9%DZ?pr_d)Xk6rP zkRD-<7V+adhaX0OgTN}~J?n@BHPm0KyO}(aKW>QoOSAD%bD8KUl&VZh%k;b3S}`_1cu&N(Rl2gE1N)2x79egbyr*lOrY<} z1$b#cDqVW{>ieY;>M~>Xxkb(B>ynEA`s!M?W*QK0WS~GCCf4wA!t)gWF)`Kx7z(Sb zgsW?=rz4^!5<<6OM<0j6d5eA7cy3FhkjU2^$+21ic zFlBx{0GLw8edWq^F>8>k|4Opx11M%1x??e-L=*DP`(qkx<7@s0U z&2ZMlE4kafGCXftNTRiU$1U8m^9oQS^j&|jC?ZC(DkrdQmPXaOHC686|E=`T(qyHk(n=>_&yNYlI_-y zxK4Bv3mvp|F`(6&ikZ)LT@TZcAu$OH2EgJtlD#m$tlVvpXRx>6A~)C$NhCVsHQ`BZ zH&7!|00v0oswj3MHr#0)NCUekI`b&g9^tSywXc@rj#mk7TkRbn^kT$L(-$6ZBqB@{ zbuQi9rvcLv0MVo&7WZ3BI%;ZL z{cU!C&<}z9IkeYJ`1!7!ZeeDwaQV90AQey7%oa1o_Wk`(p39Hk^XuEywtk;NJ~a*N zaES9x>ZGOx9!$a%#Ko-{HWlRkQ{a#KTvabVHA4|sdGwSjYs9FT#X>-nQ;PSnq~yni zW2*1>IVXH0AiZ-18!AuBfR@=Yoc3K49Gg(k_3aZ1lEHv^4z$EI1UDjQkErw@;N;&+y``APvByOjdG5HR4 zz(jZe5}m}j%Y%jrC~X4Hi`2;R`;;Vd);XA(N^)Y+-=*%wL4`R6@E~?N3eQ(j!sfFJ zyOX5a8v=A*8AZ!0&z27Kf4t0kU&Q|4wmU>`q#3Da`;MuBA|F|a54%`5Y>VKHb(XUW zyHF#5mDjcv%aYGz8Nb2=Lrp+dGumYrNHYV50!6Kf4Zb0y+Bf+3ZSqPf!U`-G=NP{0 z&hXk>%s+MiYCI=U<2N`(g{xZrXL##0!Fh3iE&N@ob{Qfdt?chL$1b!Myi_Z$jaXVSB&FE-#Zk(&iXu65#FGU!iw`K(* zI!beN%$9x*R3sNeBc_+*=hl3k^M-gTq#72U{+YPzIFh2*^~XtsbR~f4a*Iy%qdQ=I zXKpr2fwwb9;3nNL zQ7h?fSQfPh*9V`xGS6>J8)iwF6pxsw+1!Vd5C1mM`EdiCMMbLsh3VrR-bwR!1Er;E z4Kr?O;%z@!zV`xrz;3+iptsRGjUA!8Ne?aPHmEi)<&+bVy8$5gDROJ5nhK*(`6T-* z*>je!oU}=>Vd7ipQCkDZ>HwN>m**0yReCFNPp4qk5yDLy?BP$~jCi+dfbg4{ao$K< zB`_GJoE)324V&8VoQ!bk)FSaa8cyjUG|wjt4H|i1P>Av@GH~8yWqv$jfpQ%TNoqa*mzsD5OT$}W3u{|h-%VJ4%+NEzcHBZPXuBk+GvC% z1)T%mj4l2Vv(lRWwvggOJ~oNqN*!CA-S7WSw+ZmBs!JK{0Vby|(4N;4`7rTXPU`)u zVwQfYCi6i+?=k!sIMD!*ttMqpq|j;SgEsv_mMDevhCJ54NTPj@823%k zP8#KlRrIq`6OhE&7(zGCi~CLaXC@yON~E3Y1FA_Oq%jy(KZ|R;-pg3Ez(?e^pLkGX z-PSF1W@RkcTW&JK&Dp;j5B_>ht!+G+L^pN?@RadbPm#SMpSLV6^fv~riAE`2r_9O9 zVt0Qve+)}|cSL@=4}Kb79ivCCM4`0gy?a@<()nyJ2f>@uoZR{ut=i?eLvxW3==p2$ zXNKHQ46@39v*M4(P~0L@{e_7IE|dFN{>e|cr2!lfK-cb1vPB`pl%9Jv%eO^ex@31E z@7Bq2FYgY04KHHWc7;x%XdGsXrdEvm% zYdsH|f?Xe0uzhjHIpTPo%k1XgCf4at;}IBJIh{B(+JTRre#{4oW&*nKPyxr4pP;n#c&wv4yZg(2 zez zLYuiQYwQKodg0-simGD4I<`-zui{DZ?>-YAO}7tXamAk@R8W>w2rt;KR5-fi{MSQ9 z%6$I&Qo@H#z$ygS_A8J@M4?USEtrFTEZ7s9;%I@sFp^f6&xw{toW6a!UWM`L5<@|F%-HRT}@Gt)?2@BwS6sYEw zJP+?CTB`@>(_0_z4n@}@xJZY56dd*#wnCmv9<~Oz(hznnl|+5|>C|W6vfn^6K1hWV zezEw;Frln>bSvBUzVjKjp3ZrgVU@>p*@)1|Fo|4~t_J_0f5|I^Ftqs1rmd`ZTqB|n zw~WaTE{H*)onr7q*LvuV1)IgaIdd#Z9v2Oh5v9c+QrqpnHENI&vKabPzI$kP(#@4mA}iWkTvzmp(352 zYEBnu`JM+{xL?;hr{9Awe-+X4i$oPwF8>;%M&ielPrHO=)U|)-Qi)O)agLID760K+9HnEWZ!lM z|I~hx*^Gp{i4l4l$v#f&{p;+#N@jSi#nRXBS�D`IptRu`I>k0TFS|y462cVB^s? zJ+WoB(dk-|93o+O&Zf^=k)FMW?&n%g*JO(nC3M{1qif8b8)U`B8(H_xjR-? z$vqPZZjt~PNZPc*r4!jqd5_0r;er}Z3+=2M6%ovDNg(nBE|>SW0Y{I^$NhG7T~TKU znaWEwg>iz*W^M%A909ju9jMJ;B#l;RwLr{(Z$W$=_1_v4N<6NCjQK_X zQ9)!trxWPC0B>J)IX}tXp51-zc&|luuT+~^N4Z%Z@EMW-ieP>OE(s&x0?2If*-$>pFrmv|U(QrRa$E+qfe6k% z-im_azb2-~W!R?$M1({%D*vGjexa~Z03$<^2qz1&w0TIH?(5rM$f9CTEN6CDQCCbR z;7RfhTMWehPbM8eTht`s(O*x84}C6LQ~`|^lOV2bqxARpue$@0Ny=WLPWLON^8%|i z3gh#~*vQo25x2Uee6tdOUp~#gih89!`Nl;hIzzG|BtqR}NFf;zhJPw)3%Gj85p&i; zQ32IfxDK{-kl3Xl*sliS?kF46D@pX7?zb=2{+PKwE=XlnZt-It>_ z%ygYEg(|sOY1QHgY-N0a;yE<$6v?;ZB$EJ6EviTuX86pw3BvyjNI*tCI#rYP?oPQG zNHXhR_M`S2po3ubHWhF-GBbS#1no!yGIDACZ`=-W@0c3tw;NC*D4X=SoR@GviTeTt;h9dyOq8a}TTPb%3%71ezF`sxfMI_sA_(VP#O8w*FaTLY zey%92UV9;Wx|%;)_PjWUmmQT1=GPgr+{EjCB4R?)8mr=g|1q|w#KDcN<)GQ-4>bF4 z%+yONzrbic7#LW?|I0qtLIfNWhOG6nT<5LE!4Cq8h;M3>fX}p|fLlfM&LuAr_?m%K zWI_rcn8s-^To^1n+!OiLWmNcrQ5aWfu5fa6A6l@1Q}v;xbh#flh2rsFDV8a-$a4R# zp0r0ICgL-&PwMGir5D1pu zus=)~H2b<#LotU-^k#0fn$%-6)uQ%$-^oj|tdFJ4{$Q#p0ruu?8fSfrS^yfW5OJG%ZT1mK!U2ln_Y`5XWE}kx|+r7mi{ks9sq*!EoyOzoZ z4F2@F*-sIbni6}Kim{dRjiL!D2*+wl3+P2+9{xo8mRoa&Im!sbikg#VJJbz&5MURH z+fPze>`ds~jt3Y%64Q89G$(}o+EH(E3`v!}ntG?A4}?ouWPT*#klwTlZ#oS*OL6nJ zBswuyN7xX$J?d^Q*c&W_P=jVz3QpyVxR~GKI=}t;%Q6((?*a{zQ-5%vvWL_r~gDq8wHvNa-jIBY9ucwyJL7$huqZP9)JG!L!J@Be(Drz1ikS!+K|tO}r@ zC@iNlJEjg^>h>7{T%lWBGV0g(V3j$NU)yrqWF+Y>>Ogi-Fr`O`aatLM;uYa8r~2*z zb(|3>+*?2T&QT*$`XrdK3Mks-e$0qL9Ij&>R+c&VY?_IonzEn2(@A{BOU9ys`_g3k z)^NY6grB%>CTSUZ^PH`)w+kGN>#=5Y0e`@xxgOqFr}UkGtHSmzd(%1|`De~vvmm`U zcSXS%Yv=}>&qNNr4?THSUzyi6+@p3R6^3&r31O)05S3yqz^WTxHv(?lY`iB95F=k~j_ zIH|Je^#{_4ob-2KPCl?VtA4+_a7LdUk$dd5&uKmtSbjkoK4g*M6qH!VLh?}}F2XVY z_rczs^s?#>;oDbBU7OVGDe~LEgd{6oKsArRnj=38bXL`004?bDEv^iCE@{`blvbG! zLZGwv^<*|1y#!YD=KJw;;zNc*-9$dcxhAQgXI2V}VTW91(p7Z(V6!k>Hw&oN?hM}e zSgZ`CPAi3j+Skl^1XOj~D&Wd6_v=KEVO+?7j)w_j!%NwgQNm_<--F3fLBilaHc~1X zs1ga`xsps@KDyk`@2wOi!3u(lFUI)F_D(s1seIh#sUefGQ&>N8O#|ARdGy2Vu)tD< zC9|U(itm-FbGMev3K|Gig}y;XDyiU$q4g=EvwM_4Ir2jUj<#Sq^9C?WsFhb}Qr+w~ zQt8r8sb#xN$o2N5O-%Yq?}h`Pjj9QS+Fnd4Dg|d*Lb5)AYSQx_GJ$PG$Mur zKp#Lqq+CbpY(|zeGIG!wV-+(e5p4$c#r=MU+|)+P(w4TVP0`Q~pIg87dAPe+(5%4uob!iK!NxwyCrA+!w~-n>C7q!sGB5@w63|&@xwG(WHT1r3p|?q8sHm zHYB-Qq87oAeRN?y;SD42qRTsPTCY~ucsW9P7g9|#={dsO3`;Euz#6?T87lG01ho8c ze=#&`l)!oW(>&b{8i88>LJr7Y?|%VPE~j&T-wd0$-hSn>qW@_T98h^CtIsNLfmgn< zcfI0x0Y5fqN1)Zy}TKekmx(lXT z$Y04XhM-v4DzVxOuuc_4th#LSuhzSVas~I%S|rq7#>a}2OtQ2b00D?Skv&F4Wj!Pd zdD7)(yM%sXfD;-ZMR9?<#9o0Iw2EgI7~zv7L3uW{f6D5(3>D1X(-`EGrLNbH496P? zI+fHQu@2{u?232Hid@EBJii+CHP8up`QwSiyqOA~gQOlHl8+YHeBt?t)E^flDNHmG z<@61~o+2D=?sb{Xt|IxX73uKkz3u$ysDqNhSv>Lc)df0by*RdXg+CDBH>m&d-gNCH zW=a6slGCZ%tv#F<{nu^$kZ;#IgKbBrw2QAW;!7o4@=(yC=AEcvPkIzY zL5LqeO1oZzi40BV~NLj4Tpv54JvE#Q#dW>}e5wG5VbO_9YBjtioYnuAQB}uQ< z!^bRlB8!UIaId&#GMaJQi zuWkC1{BMGo@tu?=_j*t3N7^0wh08nCgphTPGj(MKb#QG^LH@-Wi-9u}=36?!3ZuYq z%f*hEvOPwAKsIv3=Y^%A2zq?v#$=j~c{J$>2~N=16j>QJZgI|U_a5AB3O~Q{<^1um zHpW=Dq;|&Q{jV?JpL%4h&(N4AnRAc!%eY?U>G5_|hYTkjzQz!f`o^C5YSTcXSN6-B z=pRw2Ho9AI+EOzRSmctUYqjQ3AK)__f8Q9Wg;-EP<) zFL-UAPl~lmswjc(iUlwoq`KmMkrVKnTZ+0?>+_I zY)s;qJ`U*Ww$+rDnpG~nsvt6CF6I{Rb+NHjO)vTw{z6xRhufj=Qz$c$`8b6ZVnf#7 zG6DPi_bZwDe|#m*M)0%kcanZmEJJdqX#1;affn^iqW70;Fw#I9mL+r(Syb(PW@9;Ua`rfBH0-xK z54FFgkyuv>O=T!PCQQ6M#oE#tU+8jwpJ;&FV$9DEpDL8>sZDBjpz%);W{$*rh0pk- z|M?-i@72)!EV-G=!}@s1u)0O*&R%bs>#`tuK!AWe>Qm#uBaWtK$)1J;zJe4A>SzNH zq7pB=lV!*l`OWtrZg4pr657*{XJb)=1zj)mxg|4T5igIk>m-0C?~@?pHHTCJ9TgI% zL8w>RShc@N`&i!BM7(nVZ~>gGOMv|bEnRAQBm~BKryLu$p!_|Gy+US!2Dr?-G|^VX zTFQf%YOiU42F;GN*~QF(Po2dR{3;%{*GP51?^|LW{+NQ~hw}IVC>Bo_%7~0juUSbe zX;q<8#>*|xy!JQ^UJ+`u+;wlsEah~VlIB?)AHgW?aGMBOt0Q}xIAfnxKSb4H7njK_ zUxo4|t(rNFCKg+j_mD}!9I;%(nOpc^BG2)YU6XMW>2%-fPZKGe#LY)G+-RDXBPN!` zudXf$)$um&e;{>0&{hD5Q+2L?l>-5U11^Y8Hl|qw5E&gBg#IN0+>$ssWt+f8sS6g< zzIu9fBm});muuEt;?S4S151`ik>iM|r_8FRrX$dN%V>40w44zm;>0;pRvf6q&xl_&S%ywe;Ih|TL$AACC>9WWfy29PiIZs|y<)g$C(a6T6>pk`tQEl4H3| zfC8`EZ1$0J4~e-#h^O9DcH1#w_&n^27G~#cCG_7dW3&qahAVPE8iZ#gf+Z=0i%e@V94j1?Fgwkwn0mLqWdZl~SQx43nnF=UI-az5{jhX;doa*H%`6-E!PFE;4gp zS&I5PGc{{b^b>Q~;hdZhvcblC%!&=VMPW%*1xY5JfX?p_P2%#r(`(c&YfpO7S#%o` z=Ff%ivT*Y-N|#DFc@a>+;xBk4am7@D$2O*?i*Rj_YEGY)ie0{4oSC;t^rAFlql=>9 zBrsntizz~~VJdnipsseM&?vGVBcZX<6I^uqZ-AN@M@z(11n;)IGy$~}rB~@wX_D(J zo;QtpE1Nw(@c_b_e(=l?d~lLp=A5JQI{G%jol)Sq{vNwAhM;Xo7EoJBnT*SKlrd9T z+jw`DqV7xohHmb9VcE=iivD!E&l&XW0aj|_vCihirKODTTAPp9YTmWFO2y4Wcw!x7 zTK-7r11HOKQ!z2tpGQlZNlfy2J;+7}YAMHQGuM3(xF$qzhbok&3OSvDBuOC0org;j-# z&CO1f_fiuz^;4JmqxImLGEzGqOCgvxxjo0g!dLi&5U;j7S-+l|Fwdzp;LJ8hY{TTK zM1(nAQTmi_m$x2GY()3>?_sXd`o_vIt~;XRmqFw6E^fDf`pUv;x8#DhpRXy@rWLl} z7%RA9Jyr74kS1|AI41(FYOb4nN+Avxe_3?{fcvrXFX6S8cVH`7s^jx6NLWW_Poce9 zgx<$#Fti#-pY*yNj`&YMlsgwmkEF}=i|x!x%sH3f9a7-l;PE(JvyQ^V;)hY;1aln) zw9tlwZC4WzExMb0P6!>cgw~uH&6WBDb9r*Cu^C>G_*`*QXWRn>C&+Ui&5lv?BO7Bt4LMHae*ErB&Y#qlS?zIW0M@X=4by zc1p4GJF&+OFklpbJx&$S;^$rZ?v-! z)UTaitPm**(xc&Swi?M7#m6ew;Nk4V!s=BR?nbm_q&!mXDuB^-7~$`#i+urWH-yZ7 z&vUE9&PH+8_vGo>Z#2g*Uen&9Z7~C3zuC?)Sy_qD9`e3eY-CpAkH$7!>{ zk?f8sBrPO2Wf2=xO{kqM2q5_m+d`Pw#uxX`;WMC+=f`5wGY` zn^5?$X(4)(>Y=3>qW0qtJ5FU>xPoirmqoUcoxCt;WPp1M*k|TEy#xy^3bJ9U7-e+w zhxMR@<}kuFX_1lv>bJoX*RIM0)KY~)V{}*iIMaC`KId^x%-aV-#aMkN{uLDKzXioh zQ!&jPG+@sppc%r9Qzs6&s>QRAGa#eUv1CMx_Ti6uE6Mb&DvYj+nJn>*I%LO>P6bRw zCIH~h8nQrwA^dFch2&tac8#;;^(+38fb?lxxARQNJG@Uklwx)~M|@y?jIM zHGtf4M88zY@d(x3jw3`ICwFz(Ri`nr8#aHlTfnWs6xhI>zBgih^DFFA)ii8R3!;ov zUQ8`4$f75|yVa!uZF>ZrOR`ge#d<_VSzHB;jC80_Vpq4KnS|)X66x#GF*k2mZ9=2_ znM-)w3M+);7tb}`nlC+GQ*>FaVbk%-IBN=jpu!TKfpIP%JOAO>dGpe+(QMeB$p6`% zJ4*l7 zjz5JPoxQB^w3r6u{IEPQhc#)h9%$sHx|p#f-Y?a5(_YBC#uJ6ae4EgE&B$5>{hB1O zG(zEBEv#WTM4lp3UArwh`qk{7%0eGPO6;U%| z>YY`goZw#dm$n2TS}w9+cd>2O^5oV_TYEfPbaT&x8qPNsic+!5`>et0)AO)rBpgqz$BdJLwNcK>_g51x?g*GcSrT*q1hZvki`ooS2#!!`xIaF+D@OX*|K`Re8~QYvt8%!!2*DCn0G;Sg-f?)q=K{bd z4#EzNm9OnP?#>Y-Ud*AeAa!C4>`gVsn$viC?%`bRsQ5zK32l+4Bq|XeZ<$V-H78&t zwb4;9tk;>5em>@|-WL4pai{+FxR=%&ooa05OIFiW``_S>8$2_5_+~Yta~t7Uv`-~I zWXSYK_$qc)G%i61*ntUVfN7ublpGvc%{RLwiUR{4QR!yaQ^roon|40{J4Kt;62N^juw| z2HXj8HN;KGTW5G2ItXbwN@B|h018vV_)q-b0{p1tQGe85@2m~QK3j-@6DG3`LgH;w zuFH=VP$=?YM{*3Ak~J}jZ$9)AYR!_(ebs&?uwKjSI}5VKU-+8H(C^QPx}Ffzf*e}^ zLyDYN6#;WXd!^2l7|3-30M2aEk8m9eDZ1e7pBfQltQP!Mo_-sdhQ#EMEDBv zbRsnS5aX99%|{^Zp`KZn0lMUpxH!s2N;xYd5O8JKEyj))54!Ssq{&M!NN@`B@{pE- zjtiY`_-PJ|T<;wFt|Y7^{(i^=xG(x=bHnB;41aG!i(uvN5~Oic&rr9&XgM>UDkxl|e9W0nmPF9=Y540$#c&6tYQ0#nS-g7>DE7&OX0bW^_Jj-J@-<7z%zd6*Z5TDGWm)Qa8 z-q_!v&>~6`g9R?jvpCa@Ey~Jm8~of|t69(=%aEd9oWlm@YF=IQbILpI& z3de^Nv0n_6Df~OyY$b{On9q<2HQ2&3=YW!=*=mtyQDOSITqZ_r6pMce_g@qdcwjg; zQctQg6UJ(?`iLd{C0YH<9zBQ?3@J>=noDJs&GKc+NvUt5nKXv%j8#G-dz!C%@KvXLzKH%%dbFWuyc zSU>@(tLYSh8VDiD{Y0Ca~_2~FBW@hfEwS;G# zN;wJ4W{Ql@&}sR?GHRF)f4YFa^$dbbCI%R?F+41x;_Yh>{XTk6Ni0D4>~V&_i67gu z6B(BFw4>C34h$Ms1q`a>%44(FB~}d2O~CD}1EO+8PRU=`20A*eWoUOJw(9a0*z0Y={8D25PqDhT!VVUjRxDr2%v!!Z2PwDQeUEbj z(s)~la;!)fE&1zgSf9++gf>fUI?9>L7@vEY3F|9$RH* zX(LNoEnI60PbWEgDi^0x3rOnB08ZZz>wAbwU4vq)= zCJTS7wp}f$>R4M6`krJ|^(ko-n8irVJ4Dr2ZXU@T+#}DNb`8TYgXp$EpF65yQfk)d z7knGm0DI$YDL6CeMZs7ks^1nJ)Xf9#$0S5;Bh?-fx%NUB=+5y!+{RC)yS{We>VCuVe4{OiE7A| zhx@XH)$IuQh3LV!5BTb8De)rtMCS)&FG1Hjf>|(qwSODZX+ktqo7KSGTf2@^Bqe*z zO%zHXT>CF}zRrSm8GUyO*P&ajhonV-qEtH~0_G^=HCM|*#aZ?xt$m;@!z1+<&1rYk90C7^FS{X}{f?bAzial)=Y@aXlk+z)5 zjPZ*}U=!qBDKhHr$M=%jhq0(NCTz)omNdQ(ajy@|yuY)&kk77DudswSh? z?Te3*Wxpe%(T(`imbIDMa!?q*e@5}7{SnOw#h)2$p@5;HBt08az3!xmZHEt|C8|Pi zP)At@ee>}R>9y)?CNElEmnYUuK{UKGnVFf72kC3Ln2%qiY8etZ?Cf00>JQ!Ljw_qk|+?JKD`&S(|nEnRFB`rrHqT1s)hTrBKO70EA>PuM~X8ke(#jwjt%l_b(ryuLmojQXYemD`(;&}}W%>7&>mVq5eyfP>+uyP3bNjE> zF~pYv3EGG4`qm0xwpO`H?dR(Y#EIjI{EFU3Ra8YgoD-rN)>yyNYPC@~;TNHjcNLRNT#mhNr4Nefu?C9CexEEsk`x!Q zK~V46?1LJJ1kIyI?FF=+C>V?PErP!}Cz?~s;AK`lsbDRxs*I{2=R-&qpCzeq(I&i9 z_9%SM_1;xouh3{pOm7{rTce0S_Ge2fk7B~uDraqw+GqdJMqZ?=6 zrMgGmnf#TSZ=eV5q&TT;1kDtAp;2V%OVJ}fmBeHD=F6V~#Nx9R*t^vREx_e6QE^&g z(-4JPljjI?v#@7q0u;OU&B;0Q8$J>ad!-U1xtB<`2cjP~Dja7iB4ri)ApEO1S)GCC zP&v3SdGt*Ai$8NkbM;T+F{(_LA;I;zR9TvPO&5xJD_*9n3JL9V2nL)HQBp7L= zXr`<-^H~S8%BnvJElc$HQY>ffJ%tVQs+Tu|xv+m?$4JlzC|*R3)jNAqKYr<1x@*xt zRUK<)FoF>;M(pcQ^wL~y#CN3=>3L20Ck%Dgtyg15vca84V9fcKUNU^m$ zq->49-VIP=7?v7yXN>86Y2s?5ac`QcKJ;krC5bf=+vFb0Fl+0QMT|EYe}#apqlNJF z!~lJwGzp#JCcz((hT=X-*}eN({SWb1z_IZ|&XvRtjR6l$1~@RW=2BS9za$?F;=r&BfnwRhLQ76NP+F`HACUT9hLCD?XuEQ#mH? z`%Uk*EcT9lKkj{rO7znMZ{C&J_H5-`H8#6@;l6!W_lRXVY1_vpN+9bsa}ttkY1i22 z7^E&+EGNy^C-drmED#7nHTsk%{9~Ug#OELD4yL<%xV03}C9{tuXPa%S<89cd999qq zTy*cJGhz{G(OG-y`y2YROAh7vQ1}x0Noxqz4Uv|6?JP;C&f(~<;$G5@#f@{s9WL)~ zTfE&4x}SKpaBwU|dtw`exJ^?z=hIEI0W3d^y2bc*aWW%ePx%H3lZ@u+BQcQ|5ar14 z55+yNEGYQAarq$I{pccHSl^vPQ@ zh))!s)UEy=#qekqHH>)8en9dviPu9Kahtp~8o97g5qO{u4#Ekkt;UE21nP}_Pl{VN zHR=pf`z&OktplE?ATi0M{yWAWv6M>9S2{7e`_0_R*=qErp`kZd1$RUib_x>I2**8j zPSEGYv{@j}xZ4F~O~Ze!piC^el<2XwXs3LHHMqRdBBCbi=JS~|T8e;3EF&Ltrns^^ z@3mMtqQgoh$MdI@d?XHt2Stb*ayd7Lmsdv)1K6N+no3SkQR3ANE%+}Mgny?7rAhsz z#D(zp#!evzN;g3^nCqi3N6%pMy7~6!hBtWpJ2u`(nZ|f zZ_;E<*5X%w0edD5hd4rM(y^&6c*nL46sYv(>+OZN^5cd6DY`{v{_X7_q_;=I{!Iz$ z`mNhT6;_uv4PauHhhIbCCJus|*rCz!n;QJ7YOKH%HsP`zAn3-mwfI~@Q=y?L?fbVk z`+g@i*`gqs3kie71Ej+DaeSezl^&BXb^WvaB3=QkRkk22t+d=C?fCX;GdYS}NSIj;588-g< z6#PMpP$>?l`yVR7q*Dg0WkGA<0qYxd*a9xR*tGiBr=CEP;lX0yy?^PtNHI|ntR7{2 zP~m+VTv~uyg@78A%bAqM2Yr-|C0rhV48#Y&!Y^h5CedQmJ*I`d)G;-*&c2@fKrP_< z83J%~q|-4IyktCh$>x*jkeyyVX5$B?@KIgk!YZbl()g+Gi8 zR&w_O39MvcaC231&Izn!{=1U3=?5#|7e4?czhN*xW`LEfuLO0dsg3Cwxbcsn+8iDs zfd7hxw-mlExv~;mMU@0Vo8ilLOmL4=gtk=$w*4^`c$yI}kYJgtU>Yj4-hZTWZnBgn zoqcTb1YrgX_y?S&NzB|{1?~o)kSD}_mr65;z#s!C$nGhqcR7`{2>i(e@QR0*pcF!% z4Q)RfP;;3vK34cIbf`REysv431p-A-Z|xGE!Eo@bfHxFse!~SeO&I@1MBwi51xu~Y zpaGk|75;Ef=)L^ea;DIGDFVuAjh3qFL?WNV-f`nOyyA;*+CVshl^sqGYVNU72)CIj zJ+!rU0_#Bc71OqjRk^S8jf+mbh!$KOj~*i}wFCfUDU8_R`9G&D!v? zo`C>ErWpwhwEnQ@1~r*)!E1PYqX;mERvgb0F1dh1>|(zwF~fR2HmJF^P=L8@dehjU z2Kl52ueH{ntaG>vzM%t^9x8af_@Af)n(8&nE5rYY-DrtZyuOPC_JYT7iwQ^(su)DjS2)S5qO6~C+-0-Wgcx)h22Q^ol z&=X+lU=6-#n5jeIre>Pthp7)J1Q}>^3K1rcn6@%l7)hhBUB*d z_^RblpQ*$EH&5jz4CX-%twja5mg2v`X4u94i0}MM3wWQw#6!cu3=IP{8$AawQ*yip zed95m-4*OtWgXm=8|r`D$(%+F8VQILpAo}--2gTM2rrs~dW;SdxcQG---bH0PE1(v z5kSF(g(2)>34~`7lVfS{Ho{Ixg!XT`E@to;ox%?s*%OD)7iQQQJ-8$5>#uad`wr~l z0RK{R^nx68cs<#m_Ui70*h1}9_5j@cpYUXqhzuJjeAe*cW`|u&5kQe+WBnY~M)y$$ zfaVV`%LKWaVWtF8(Wm>rQPGYS9yZHaolN85%`_n)FY$IF z3II0?B~*JtLu@68YtT;lMM+C4{+&LcbCL=az=)mPTwnSfhQIPYPG1ta{yTo|b-tdw zKi^Q{Gr1Zq-eTGvCF*rjt6Cm0T;X?Zdo<&@v(R)55F+ZfJ~KCCLP3uMBMKkDs?gJ3 z^f6aRr-0A-nrftw-F5_Hx(J$2;8=Vv!XOdwzmF+?b< zhxk>;?Z}%9r!K+}*Y!I3Y;Ugnx7!srrGE`L!S|eAa`<-dE4{iH18FPGCWP6xZY`(Z z_8&fT=u$v=>3!IV6;$#|!T9Y}#h2~UMsMB6FV+q*Z+>CEjfj9mgo+)f4qvGxUhGDz zdavM2w8^3=ONpM$m_k1>)!}z6zKMUVIUi%|m%g=i`_W1@a^xNeZXIVn->-+>#iz@a zXpvdgPp&gdDA|Pv{aVotkv%I&He`Ch-cZWH9j)M+TyPeiIhr-}?_rh-QD&#!O6hm7 zr97pJ$Ghep$~Q6MK0tjLj6u56Vk=%WNL`NpqnDb2we5T}yEVYJ+qV~NV&a#Rzj=*% z%vB_7ipok;N+N+CdcuBt#D4!@RkV+tyitd|QTd&&wLb9Fz}=G$L?8O6e*r&6DAj!C zHKsuCvyA%6h$22K!Vjcsz1mps&&*Te-q)7(*IBFoM}NYL7WxCI z1zoZ)7g8;=&U;mTxn6%c1t)~dMUnI#2q1UOlHZ!pC|As^NMw$*!7lbazL`aR@U3?5-Y(`wXxF9 z;-pL-p>SUGPd_+erbz_fXS0BrX8-mq`j+7IrwG-OcSdPa!q8M{l8X%w->WHew9+J6 z^q<%>9B{Wp$N!8E>zlzOf))rzm&Xo;SnBl+*U!J7UR^W2ugdx5cNKk$Whs6TfVpNN z41N9MjcxhKBj|Wl^=dD`wf!dB>g%(hr^zmLW{FfU>ITLJg#Q4v#|NDKIY!PKKRKvg zCw6;2TqGWKKA1YdzA<*A!vE*J@9Bw<6Lg2;WOlpMsg(PXD857v+;JY#H~34Iv7&rY zf2)e&a9nF2YOg*wvif^sD@k8VD3R+T)~Mg*wW=GRXKB2wWt9V0E^Di|%4As$OZ~!` zlZ=kk5+|lf;-@6<)%SS6K7ZXylUrntc2Tiq-JGzpV^!AbFJHH|vjsygXfJQiTto=( z2A_hSdX~+=%_98aE3|GhkhkM@kLTgk%b*a)@#p+9{_8`db&(mts-;QdXWub}RyVyY z10MQk-v4WdqgJO(cSt8P=YEm@{_9aRFn`I*g~atjzvFc7)K>$kCCgC@+*8*^!BXk& zJ^yX!W+g|K!xxiyb-8c0UHi+Klr;Ne1e&U>>34Ip<@Gi$aM>;E- zb!KNA`IQTlZpgG9W%TeS{8yH`p$F@Gw841P{HwRif~Z3JD)vbby5}C4g+~+g2@16o z{(!P6Cs1k-oYY!nP^wGU;CJhb_Xc}Icye7#VCMSCSUvf(GSe{>q9u@Bm#IRfbcS6Ie_WWQj3qld{pH! zw^kj^6%sHL1;oKISX_6e$`q-->XJ596og%BXJ3p4KjbIc{2*tP)H|a#_DR3#;ih~L z7T3w;I!oRqlo|`g_@ri9@$E#9#S;A&z<$*C&?{&u?9FIyIvKU6uB zr20PbQ@{~ZvBnF(qxfvKTa+ZotvTFEiC=-`Ew!5SYAFMTnJ&-08 zhtwfHVi(Kz0@+|j4`OjF9(aAJ*HR)%k7b zRprn@Cw`e(!Y_kI`}{sSg1dvOkO-4wsWO|?3c(nK+!>|)kaDEHwQR?8sb}nyR)6LtA7t{T`-B{b3D$(2rmhByFA%V;V(%9c+&LBVv#A63zoxU zwjm3oX$&c*odCZ@&&{fkbowVkbzRfNo-`D3R^Nz|$D>`C-V$2eZDTEt- zx)Dnhvlv|VP)C~u7UX5?V@3ih8aa3Yqn11MAxrmjIaT!hRk}H@dNTLnZyY6QiTp}_ zT?(gLMcG{(k00{mj~%}BSpG1qE|$vQ2! zv4lQRx${3dmo7(Id5yXbT*42x!(keZ`8WNfQlR?PS8#I(*Nr z^4b1jCexK95f|Sphy@dR1jN$qCLzyKJvWn4_=JOo!v+jKcUQt`U!r{@rXG})N5tAN zgEG4Z(Skw~@e6%ikM^(3`6_bdx&E%c_aUSpMLSQ$v!eMC@x1I=zmdSfGh!wwuHB?C zW5PK8TjH<><8R5Yf6MiAG&Z?yqY>m?HkQf_N2Y|9*G4wIbHlr*MZ~vo6Cly2J3f{I zz-uw*?`slgHRJ4J%I`Q+A|Y3ip=#T0B}G|jSzBj+T5tZ0d8`n|b&o~30@oWb7C4@_ z))Hy~jCV35DzlU}FFM73=u`W5O-Dh?cbNkyYnycaZ7gAS3)BNt^{5w+>ymI(hFJ6`4xv2&*6a`u_ksHsw(Z@ zd`*mx?V=ZgI9r*`n1h^sx)1V+biJX24~~+>kTBfTbekU|x7_!zRfigig60?Z(A23l z)|ftKPGVQbxI|kE@*P)8x5oZ)s|Q_+^7T+mI@M#WbFpwS(jNWMX8uKHf*A^&<2F9a z!mjd~DW^|IlInU{7-<(lBK*zuJly%He?<;m(qli^E+agD6@;O5BM&>d3rM=rc-RGp3M7>{JD_qUe zT^VfIbM*d5x6pv#^~yhfBJ)DUQ@p8$05{pn)NR!wo2IJAY8x*b5Vid!XhmOODrohS z31q?M#rIJn5wZZ@Y}@_XTgH0P-Vh5F{e0R4zTlmUt~Bjv_* zhB!vjpfN%L`me|X5uAFKB|JYJJJ7M>po3M8DLt;EK7rJN0p!$1rCW%K;0I(#A!U&7 zvU90T?*scMMlyzlJvQS9G%pcxA8iPhB!+D}ULqk-{?nz8A+BL7V>eyPs^Lk%ol1(g z$Vq36CTv{pVT3qzb(LJDzIVVY545~M)Zh8x`Q z<%02rifaZq59OE!0`rGo0B8~|6~dDrW%oQI*C}7YATdCgl#vuWu6F>;F5Dk=;C$jp zr>3H3Ka6+p5e$!r3w4&t{l3nFbt4E#C^i56)~UB!{N^yEl=N5`r`$Y4y7T3unC9t& z^VwlvK1O%@+j^T&8Z0~ws>p+!a7Y4IkwrC)`wH!&5%u00wS%n%&abf-7guG*S7Oae zgdPwz@8;tt|AJc=+9&3_E;%yol@PNgq_B1m;HqyKWs||PB51lkPqmi`p>vu%n9CC8 zc*3ej$^WuG;A|}|Z_r)HT>s~IU%}Cg&Y-j^(a%4dVy8_iTmv&mgg0ZEoE^h8iWm7R z{AzXi)AE(l*sS#9t2Cm`78ooa?0nyg_<6c9>C{L z5^Dkia9jwx3KLNOa;bR|>agH~|GC|xfZtmPI}A5dBkpL;`x?7pUSTcw&dj6R^0)@D z7#RPQ6*LOY7|@29_AvOz7)UbW_$BdTtiXMsd0pm8T@X^2@zyr*keH0;nSlRDkF@4h zL8yx2dOV-$@PTTHJ4J24Y645Qk#Gq~M>~n45%yaK`QzEd%P5ZPNLh<|S37$jVk=7m z4#@R#0Ftz8l=W2*vqV>p6vC%xIYC*Uvhz#L#mR^7M%!KTU=pluGcebg5+&M5RCM9D zK|%^iYfd5zNIg=#H8_j0tsqF>%pkB)8p`TiBj!9nutzgJPe*-lXql^h#{DRo;OKpv zAQ?SDkt83L&P>OWPJY|^I+@jPfmmBQ4{o!0PK7$NO%msB$*c^ffpz*alkiXkmR=dQ z2t&dYxiHFRj9Nc|ClQ!O>xf=!R%3eomBCxj=@d*rVos=CUIOahC=$VTGYdou4yB9- z$1@v*VyS&WTKy?8y$kDs8Uu98OIcil3Mfqa7Ln6M^V5a^nU0-xevUMr&l&oi1(L5s z6Z(*sL*!g3vsX`jssl|;C?z0z4x8~PS@X$G%C}zLAajoeC+Y;m1GLDv;=XU*E`4c) zDIqkP{qo@+F<2wd=gwR^!JO;?uU@~6tBTxF8OclgHKrj`4kFefoev2(X$F#TcQ)cEr7 zlAWfdO-yiu;5vz^r|r0F195RJ2kAn)Q+O^FIF4>&zByE5i&$r#JDo$pCs5N_mz+r> zzvWz|=%+T0i&>HM=&UJr9G`eti3_j0N(uz8ii>lg5I^cu^%hyNUf25JZ^;@Q@zwo< zG{H*T>hnmVl#}Der(Kqof%kDC9v1W=mTvcFDQc`85m(OxH8A>T$H<#luNW~zf?cQ` z=_B2$Fj`5EPy%dXgzXMSI_Ek#xRh9T*l#gf@f%BD4hC6@bTfAJb0WQE znGe>$GY@NCM-rQhlnr_Va~rAHW_Gw6KB=Y% z;KWv|vr8a~m*jMA-l^_~i~Z#)1k&`ztSruyf#{H1WEf>KB-J`2<`Kc8t_XvcbrqV( ziP;RkmE}Kr876pazFd)zAhW%YRDC>1;OFlzOsRCd(!$XbR&eo>I1~0rXk>eaS_y_9 zNqau^>Xqf7%NWN&GF{ebAb4F1(|O!X8zR^EoV8gJ?s4iCVU&Sp4=%bG&~*rn-$q79 zm!{Ij3+_@!if<2H7Gp88yArI&9A2fzFHfX4CM72R2zomsPg|tiLH?W@n^rh7##@g^ zh0}aOVDd&mw6`g?k?FQN>b=577j32sofEmSF|{K@dFY3)6x74~=tpsG1KiJR3$q zQzS(47Hqu}QRcnawyVmc>cptRYJ?OK>u?7rgCPuo|JQ*RVG&!iB?)y}{T|4a`;5A> zXe`H=6lUnz%L~X03VG^f+&?kS1J%TIW7QT>&$ILiUfn!zzCM*S82fC|!JVanRiVn$ zvbLy;r<=xIl4RC2K)UB4mRp(Su@c%;-nrO&TB88gtaFf)ul=Eup8)uoF^niD5MR(NfM(6kW8qZF3wge{L0y|*Bo`w(-$ z)NtN$En}@qv)>tk(lqL4XsRkQRHrS+3#lfndfQ|Lkf8ub-C4EHz)Q%xSj6nk&@q-| z$$U{x!39C2EMr9^FT+D89*fVCmJundug^`ylU;{Wjad1T-O-Dx%_>sw8I8_Zh{)^3E*>Llo%0$<9nE`AdmWXp>lYOnPDBn0d`{?qK2?aE;N)b5Iz<%1^8ZP9?I@FVVwz9b;+ZQk_jS*7#ElB?P7!n&8X-Q!U) z{qo_l%c$kkm!&Uv=gU4q5VG zPz(}%U#R)VB!E7&>!zXE`Uz&kxwbw_%aEukk|eR13ZY@tUks<|&E1ZU2khhIQ82weNF1CB@dckD#>QRLHef6}h@}d+B$3 z*$Vzny;@KA7lm(jTdx~#k6zvM`yI01bWZf6-|Ys(NPIy2=vOhqp{&<8j`iwKNN8C} zJUCi$R9nCVfuoRlo=e(Pe9T+8_IU8X`_u78`s*MGWrc#CF^0kvLyP{hgW*h(j++!^ zqx=Eomxeu$Dm~re-gD}Kps!J9CM9CQ(f7Wj=!yWH`vK*qlV>qg{Mci^ppHNilUrp8 zi%ba%dBY8U_d!;f87_%0!JC}yjHH6}y@NUz$$KOM4Xw7bI#t9k4h3_Kh{yj$rXCn% zsSUlfbUu%A-#h4H9HUj)Fm=&m?B`*NIA^xt;K~{UC;KW62RLkFd&PEYI?|+lLJl<^ z%Rm`tnNU6Pv>3Hg)++e!DYdTGzu;EEQvw?|3vOw4rS6?28F(p*Kj0w&6IJ7tYwzl$ z4^IaX_)WVw7UC8dxOD4Ce8bybNb%AG1Fh& zw@aK?ElN*AH*X*H>a2=r)sfr;l0+v*$~j?Yf=OOU82Y4hPkB}EGn}qqhOWzuXY6*r zLWSyifAtdHdQ*qsBT&ysSaozWawk&rak#E45TjIFZyS|dKeyqb!asLc`DdSV7ckEs z$2@VBH`+l=T6B5-kf0{5Gzs#I5V~gu!X^h5SjUKLm+THDY@TVVb*jHgt=Qs94Z3aH z>N7B!VN_qoS2ZzY>v|uN+vhq?%495<-1+$V;)MOcR2->nx0BXyd_kGnt>3Bll_s%` znFkq2o6O!*+MK!5jUEixpBB!E+468l=MRJ9hDkFGBgH#4_AzLG_s$C2hd zR-Ne8Y}1!a#STjkCs#M2A5&JJsBhf#ac@Rh9QpD;{`R}vS6hjm;i_R6W&CL0WRt2z z`;2#SwQo$>Yc8s1RC(}qk;ascZEw11EFSl*^&(BTgoA;YbX~@l8}ejzO%2%0Fd3Wh zqICWV3yJw3Rj7sgO$x^%Og@&nCaGi$zV3GmDj9Uy^65UfWSeBOb1)())~nzDU9T!a zscQ3dd@ktzA@{Is^a$kXDhfxksPp-RzG?h>y0{*s7mJh;bRop7ixlsqM7WQ`NFATb>MKSTHV3NH1RSF;ihMW-yXZB*8LttfK_ldBkqKG1TNT_g z`e!9O>8Py53VAaqy0F6g(vUAJf#fD~(9UZo8>J^7yLXtSFpdC18|nV zG>pY4i;erY=xbFGteZS00rs9uKda(m$E1@U50hAJR!^ZtP*1<^n!{m|ng~o;xS5-7 z(Y|3}|8bqU?@d3{wcz_3_l!eIreJV5Yr{^#Aj`?wmWOK1av;D}#oFSRcY)xh{gI~7 z$9xVtNa%bF`Y9J0L~2=qV?v`)nY%qkd%o1XOn#7{eKe}c@*{6JHU8xf1Efd=>bWz* z#F|&H7}zuN^|0v{*P;X2z~M>GgCaUdlx>bmCfgANXFGHG#CVytJa(Bs`?qryH{$i0 zg7|o@3Pbqn7hD>Z*&NyuJ_hmk%!<_}n_&6n4fcMumS5FQo8H=}!%3U6lPp=RLR~tT zBWv0av*}jF_mB0QR%MC`$iz70pWBwN`ib{J{PJE5DaO$|TC6gc<8x%y2V~3ERPrIa ze`<1(Zz)U716h_FwkNjea;+u?mJ3G4uN<7qY5%dFCHN^s#YMK53hQgRX!W9InGmz2*yYq;SF6AI`nW!fX}T%aYbvZ-(!%%C8onG&<*8K{06g)$PUfA$vm) z0kZv>fVqd@hEiNDpd($#qrC@BAAn4!?RiIG%Tl;9DgLBjphRTChLHEP{@)?wyiAv5 zDS_$6f}OB3)AJrGC%0cDFMU50xioyrXkPD)u@VQHW2Pt&1vD>WzDk78JaL&(9O?q|hfnHpM~AY>=tmyAOU-w`;%c^P@{Dx*}7>mgC^y4ODEJu~O8r1Oj4yr^Zw za=AusmL=rbw2`?!%*YG%e|}Xrfc4f(G5$QVhi*JnT8twao%@KM`yLi?JnNROP)bx( zFNuxh#U@9_PslhoX>N0oV#+#M+%sRXM8V!>8d7=d8Jy{j=EvbqToMA+pSx7~X2T6C zm5ixr>Zr8E3PmID9de8J^oaFl-am-&{!V#XZ?9Ur@mK!xeLm&i*E)59hHF?(mBz-3 zp9M$LURxE7WF%8i%7{#4{M~W?Tby)W+5vX)^LAis$yMkLG}Ba^T3TIC9P;lw%zzVJ z^L=bYeQaQJpr9pO%6BgM>z63Hjcfed_9qb?#B0m;7`=lNjRl@t5)pa)`I*D%oGPPr zAFivK&B55Rfz+x+vZF81aV?V2+P-?6vRzi6)VfZ;sg}~gv+%2~A!*Z}jG5?z-(pyW zqx~)lXGJy&#P2Y-3d;yN54fZPS!6z{WI7lPk{CWNQB`kDy>_$%%ioc_bIdD(Pd~Z# zK2$*4l>Mbb6ls#VL?Mia0O zx&Tz;Y3%*GY>_;W;Wx`xjT=g^$n|O>i{7|OLV5p*f0fP;4#%jdrnH5x)s&WN8BF8m zoLVUt)@(18ZC%)>z;+>)s_b{pH{%4lII3sCV(kK=zFgC3>K~{rU!jZ(B#IZ6CI#BD zOe%yvx#n5@et*p)IR(WG856p`6!$A!4=fAe;Zqn>7F}p5S>)y=FjnZiNjd30wF+4V zm{Yfl5!vxk5Vd7QD2KSdZcyk$*No>)ku`$gLpX~mN|w1K0fH$sp&rf)q^0I66daag zDak#mqSZoiwh(oDZ^ej*O}Ga6*b4a{zt0WGX6V{nmzodRbLo#YopCRJ?Dz&(so1Ta2D5IDD23Jvo*>1^JIqd`OLzqE04pz$26w%d1J6x)o55_L~TnyuS%SxLg`{RVroOCiET|eJi>b1NhYbmDLIPBnPTzrU`^&H5lgQ1FJ*KpN z$&2o2t@}ZrURLhH=HlI}a;Eb*AK?Sb0vo}~bOJFen)i6bGcHO;dU3e$L|^bN41v7@sX7wF`*EQd zC}m6m1*Zm(@1=SwGG_PSSwC&RK{}cn-Ylv60@een8!o~gTy49!4k}@7;!ci8$a>tT z02f!9erMHkh6XyJR&~I__1jNo~Il_OSPIQX;+} zl?5I@{tr$YIs&;NAu4N5l2vD(Cquk2;J`iCzwc*+vnfpo%^Qh3_w~4wgOL-?_?Z7UoIQ>>mUBx$?+(~sbKda=r%^= zQOrg(0=F{L+oMPBR@6*U^66J?-8PahzI%kj$;hOBM!&Dl{_9Ws+ z03^#`n^EoyI}A6|>=E%v6+Y&C8sVoRK<&`w)_rB}AkHrLn6focSv&PZ1fHC*g@{WO zu68q}#6u$lktM0NBe1JLZ&n$}RbZxT7Kw>;DxacGC)ArcNs*dBS;CD)@AphSt3A`@ zc}Av^8NcTh%ILUP0m48CthP~Ws zs!Ef!r0szhgH2}}UvP}M6i=PgYGND@3z1p%6MkX8UIFpYcon|%y@kg{M)sS9LyM`8 zSkqAFIDssfB%uY01M)k4OB7z9_lUqjz#J&7cwyEWY4#!Dn?o))MhEKBx5vTGp zp}=S2h=I>Eb-yChE(ycMNq-Y45#e_}2nDqGCF_^IQj97_!4_eEI@Y1q>keDnSwA*O zCV0%8O5;a7^_i=Z5iQEZh|9nCT6H^{tGag{MF8?jRhW2#JUEXDI~q=2J*rgR+*t3} z{M|@HUp6<&rJmumV)P{K?K5f)J%y$;7BSKkV-pWD@l+j*^B`n(QGz=2%CYaXxyp%; z680!dY17MPG9t!T^`B1?LZ`JjW5zR-7E>OacP1hwkSFq$<5`{f;rgd(mV z9P6~R{M5|X(6gbz6(@90PF~~i#-4Z9Ra2quROUV8I3&Apw$xh>3z{M2e3$4ht$B73 z-HYz8TB?VI2I*(1{izwmHRty6tP;7nx$??`*|KDlgf@O`BV(O{24@2F0m2qzgR?1> zthZf~MIn3x;Rvjd;WmMT*DNB8+>>sbwF*Txy`{Sm8T%x>Y}vior6wh+=C#WVmY;}9 z%9e*gj?RP2%w_(E>oenNGQ9GWac!{i1~m0T@I}Tqg~tiTj(N}8Hi|!Y*n0J~L<+yl zUbxgf=U>{YowuRdw+rVQwIfM6X3tj8s9Q6pLl6h+S@xHQ0je!;W0yvU`ummC$W8uC zXw-om6+fMzP;C>V!5@_Y1sw4U3L*%xYJGqYT1jPI>OWZCu>Xke_~%ZIAkypZ zKXjf0%c{)E(V7#wcr^UY+>t9O6RBh6-AdZ(Iwg8t#7@h)^GpAgU{5oz`l4LZqt+tT z+}xfoLRxh!`bS*3M@|i36JVXu5_5;{5k{Jj*T*uC^X8X(axV?rKj9M!kUYpmCjqmx z?kfw?K=7Hn9jrLxucK{e&2@|m0~1Np4a;srf2BvYS&_8;2TEYko9OLl-}=&|Yj2S| z`U9l@YxQSotX_Nb^;%=br{1^qYi94tS3cvXGW({F_8CGQesAF2#-DC1e=EX2ylk7G z>#ccIP9m~Gm%HFa&$j$w2AXV%9vUBmwrpxI-xcBd2IQ>p9(5jpsZ`LGE1Ej1OW)_K zMzT+16uK4xET&RT4z`Wg4+3Hy26m>HptlpW4Sk6?B&NlN&?>vtxeTHXqj?Z(>MHek z-9)xY9nCc6AIA4Cj?HnzwQ81USPUyU6AUTav7Lu1rTk!ivEZoS)rYHE-9_{*qb$l+ z-8z&5@;zWNR_Jx$GWH2=VKcVUl)Xi^hiXfiUSsOzZmK)$c>D9(-wv519pm*}v2<%3&K@dJV`T&LBYM=X@lP7oTlzrUSJTVsE`z4`Bs zueAEd_YnKOw?nK7pRgOWO7yUOJ>H8{4yKkkG4sY|Tz4_;?Kvm$+Hlz;pT5caf>E}M z>%K~hcV=YM@)p?ZAF?h}2TOsvVC&!9@Z`BXK!@?>01bL^PavHHu@#6j1lacb;@ z94nz}tejP;By?nu6eDhUvR}ms_PhvtpR+y`2&T0{H|KL|nATfSP^rPnd)P&<)h~P7 z6ILXPGx$LyH?ah5b4r2wFsaV`*Y3!;mn?pUc~!4hZcQ5cfJbv^H|YyWCHJo^D-&%) zJbl+-JDig_lUNmB{$%~7YHdKck|t}sU$p7APqNK4H1jKjp0B$Hk9z2xXn(N*uizSr z=UCnuY_|Zd!}s7lxtZ)EHyo))&*S%6U=53C`;VxB%eYh|Xl_IZG1kG1S|$8JW+jWn4qilLrZK0?Cs8(3iF*xo|= z^#5*lk!aWeLQGoKthSRaAzt%6~+Bz z85$wE+!f^+$#EXvq(~(lu3SQ^I@s}@ApW&`92K(=;jJ4TXZ*F4f6QYEwXZ2vKm3QJ zgFUzk?rE4+_-X?IXOoQMHk%A`Nzh#s6BktK8>Vy|9Q+VTlHo8G%cn{_{3V-AV|#Gl zcUAL-#y7&)(@6K0QQAhY=pMdI2d;Y;NBo0`375Fl!msN|`tl#X2tm*gyB?+RIItlq zaw#QQMwX*tGPq&nYm9Ow+!P4cwI7&uMW>MqKTZA)64^y_r8z##M>%#RsZExIX_gM1 z*Sbmm*?<5n>LnHTPh9okrb`{{v8d8r9Csf-%%?B@H3;g@G>#&UA{-=P?Wel7WW+QP z=*(OHJ=tGUC6=`C5=vm*aviJpmHSLUB$-vrp#8-tEl)OGHOiyi`Y*-k;V5!~vwo{+ zx}b0+E?@kQd|NEi->X)KbiD4|PU#7XMSi!tBc8pFM1q%4BYQTJ!Ex%cg{|D^lW`44 zTq5o~4LTh|Sl4Cc`#Li|4OR4OtUFjj%)JNtrr+NeR=Nq)T752xf(TLah*)E8@odrk z#_xkjx14Xz01*&Z;wTc6v9T5)BwPUjgd*DfX_{luWSv+3dj}yzWCAE?MoH~*I$pw* z$WoMYBI8exw!jO>@gMmR)ZIWlYX#wdPftn!*PwC6=fM5cploWHFMEsWfszpmqT-}8 z3*YIy_0s5Yk<%1e?)@Lx(3Mp@Rd}-MVO=}!9Sm~}vQ-6Nw|B7zcs_9?vLw@N;23&n z^`ZH}u+l0^N|?{2*hy9No4M9|NS#JvA!=aRcSUZaS1QGV}X-RzeQHL=e-A zp_vHI5e4-h_~xnY?#MAf^!B!?be+M1JbUaE={D;O1v93@SN_^P zNK&Rk{5SRD^~4N#Mhdh8_;#{@LU{OKwsg&IAt;8WRrmztK|A-*2F12P#~e{r6ZIn& zUV*HE)|${a~=i7nu~)&o%$?c^KXZ$2I(*qydODPslPL>=o}ZWtMq z562he^y+x@;S1y=QXuOCDXQphLY@hQJ>1jE;m{mQEMOn7m(y!u7=@ibymF7iCYA0k zPlOt#*`G%h_An+Iz(0MHv`V9a5-o#J3d!>!$0hVFWC5QE&24r~RD6FY7YKB`_umPX z#Cs?!67(I8`UVsL97yLamKA*GbKs7wDMa9{gu^jLxL_+(PFam!eU_gxdaJ42uldm^;h}%8)>LM5jpx7_bA{77bt! zf`M_ycc!C--da%i1!fJv`kavd59{LyM`Jpa#$#aRPy|!~%7jEo!5ZE!69Az*^b#$r zm4?~7#c14=~k zKEvP?E6jcVck-aA{e$fk+{`M_2C(e9JtV(96mNFVA1;dmSc#AH!7BN!#}q0$JM1yE z0a*V?3ufVxHnN91&u9B}o>iENE8BsXUNkxmromc$0C0M%rvr_p<{w8Wp#h@_QFg+e z1wR2!7D~{5DjM}fu!`2$BW^WdJZMxhTj3w0T&z?0fJ|8TgXM!`a7;S zaBT;|)ND?xCBqH7K>|S^+OeP-6D7hbP=*3MQ~??(uTPh5^FV$w08F+UTK<)T@mu4S zFYQuZ^J|Wa7TfB6nU8Cqe5YHI)U z`46QG1urK!eEkkP@u9FsP`=I$6#SADlD1^I?ucwA_zf*b)%PfV_nvnrc2hhDNV2!h z?%V`N>O>MO0xTgxnwutB9DLubpOq8_NvqKZfZ3Q(uNB-L6CgBz7+pP-mIvqNDFOyT z*jYdmtw@9g$A2`He-DK}u|Ub2K<%^9Xy1goB){`yzfcnbG3fpTBR-;Hdp^Ab2A+j; zSih>^dIGYtKQLG#xa_Lik3`JxEZ9#z5Xsk1yUs))i?449t$+r}<2l>Wn1;=tp3>f?#FyK1OIL$nO1 z_1YY^(VX8z_Z(gZ;VVxBCGyt&g(hz44pjE$%6y;wZ>QD!e~QHZm0XFK7uk4T|C{b! zZ~AE~-EPU;AJez9@`tf9M?3HsH*9j*pW zt7g$xbqlC=i)3AUC-suG{~KAZP{m=z0EfM;>|1m$wf z2fKVWL55S51jx?;q@PHB$5sDHr5(W~V6+lNgthvSp)L2yZPwiT33sbns`|fAN&E|r zQb8GOXFHbS`f0g$y~uB2mWOwR|L|`wU6VLA*)57e|}U{5iwwe3)X?(zw+?`VE0lr^jiqWcm+3Ao@^e6)fE@oo%T^mt z;DY?0?Qf+epA);nG-55mWc?xcYQ6gtrUxEI^!LmfSP>(vn@ zCJIYZ+xPcKm?Q!cKTYFhqsmg5e@PU2138?6DM!o6?tHMpN5>|F&MzCf#Wq(%=$)Bw zCbeE1%Jgeu>S7R@Pn780fR9~;!~ll!2z)C?))qe?8|1AW?7v_=K2G*-jx(RgM6#bt zdw~rTOU|>L(v}DFQRL!7dn-G~K7W4`ixs z7mUslC{=~FXhfi~%Sft~c zTC{lDa=PsK{ULrnW=W36N-q*8@s~|-7$y-XMX7}cDNG~xpzosoxORDG=Ju;eS}UG( z4(^02dqf)`haGUhc-k%k$}2=4mL`chioi*IYyw6wN@8e)e&(ny$u740N!GVGg8o}s zwj**BmcV;jQLDh+|AF1I0JBg-iR!bmnO75$r!=$whrO>1tE$@?RYV1o5Cj!PI;8}p zQIHO$Q&37b2ug~y0V0ScjRmT}q6mJUZ`EPUG~pf9c-1dFC>Q@4$`Q8q9Aty;M;v znIDKk8nVbTWSdjp{7G(F^|S{G_ER!^h*0t?eQhsHjv=YoPSr^VYM+UKf7)PqTe9BF z)b|ur(8wA4gQf7eDLl8|`02r^H2?l`osbSu4D8JwTsI~e%Muz_IZ&|q@iG2;MivB+ z#!@dloU)Jl>aAQVaSkujOLqH6qGH2d70=h&eDU~jdjC@kj;H>y2oE=`f@FdFhxKN* z0v8f07aGEfL%a?Ks7)ITUcZ1AL@Rc1cUXyPvZs6JdBKMyn$7C9hT{C&5|)mgXI z#cr-23SCfurn)zmeAO79Wd7t5C^mVA;<~Z7MI<}5|^P8P_@-2(`sxQ=pLD|{D zyB{yij&{n{JhiwVRogs&VIO;v;>j%LLe$BGOQHuB_RL_1$Qq~hNxX7H66#+QS-~|; zG5y1KV&t1U2oB>uyzr4Ao-JQ(_~5o*;%ZC#tovNH{>qd`(LlHBr1$Eawv&5L*Y=g^ z7!N4FC{n(dp8DAeHbr8)X0^L`3~H>vZJr093PU1s{Zy9YZ0%iVI3reNQ~Yn%YIqEb zW)O^dV}@J^k~QwNvT^;m-`$EkQG*bl*IANhapNbhPv6@*WXIHa+W7pAXyP1V188;j z_#(@3TVKL!6ccdZ`C_{0<~R-au5N!G)g$hkGA5rSuQabZHLteMOY9ivFIXL$>$JJd z6zXiG4_J3PfKUx2vl2U9yA9jH5lE3*s@1Lx3E^N=}O-hzb)?oivaURvM` zkknUbw0J-$^Wn_JzD%ORxi5aKy4L4j6gt+=AIV=Za=IJ3yPm6S#ps;)LRY4(82?8k z_D!$Jxj29AFZWjljOeo%9M&YIDA^RMEzy0uN~@LDN1CsmWAL=2AjTO2LB%os7t#>V zv`Gml$mHt6Lskb;ohPR(^ez+d=k4vtH6ZX;&AZ%5dBJSnX$$)?xQ#n%GrAP)eY|~j zDrDH*W;Twdnvt9J2TU)Yd%NUUB2$J4PmWWlmGv{MYy6(dpt#0w%cpY~fZUQbTw3|i zJ;^et$wBrc5>AW$0lzE)KgRk9+`%T?LnHjw{CP@F;nHNaVkK`Ebfy|q$~DYgv|K)5 zHmK|9CpMWJX0C9N$IJ3-{(>EDV^q{%4Kf}~MhzLGAiM7@m%Z4NqGl0Zr(L_~Taz^c z2F%Fg~)ma8+gW%^?YMEBh~%k>64+n69kuOlw(hmJ+xMT2Mt&4Pj5?pdlpFZ{;B57*!vc1F&PsE z(C|+aFhXS=1quRCD(vw%jZNxT(tIgThtz&H%^v<5MR3~qIre2C^TScJ>y6=C;?sFki%etzR~a3yNBGFX%c?8IvI z$ESADG3ci#?H1xLO*1UGv8DRUwsUIR{4AH*z2t)!f83?Nt@y+lGk(Ruwa|77*T37v zgrCp0+~4}7*JnHs!K_BD({@v|+HczHm{IRz|B`1bon$sMytY)b6D zd?|+>l5@f3D;BGaJH=5EX!ffJ>>z0l($;Zq4KM*$^`es4yJbj=B=e_nVY&@fp0(5R z2O^Q8xgIOfku=d`A<^Ueeh;Wn;o3XXX>&i$Y*4enY22ObdjV@^-KNjZ6Fskpyy zLHnf7LO;e8xr#!xhjbXcy3cYtiM`f;tKki0rKhx)%DX9Svs!|>cpASraqicBrDR!+ zlp^Y$iTxQ2=BP&fDW6XTo5Yrpl zRVu}%+D6ZuN56;dQkUdeN{0e z3~-E5x#S&b+jHv^@;UxQHc$f&@1Qh+#s>JCX z^D7)$i;E7(fnD5Id@=NDvFH z(1U|M0aiacv?|}#UCk%E_zVkGZMcINt6j9mxp0g?cn>Wdcqg}#JUc(&yt(~2G3w?) z0O+6cMpE}VO^Q(tgJ|a)8ACU)g}&GpvpP?I{Qy;VOs!l~?~8dr2@0B;+ zCNtKVH~uX-y>9N!_pI1u|9hSn_~>q&C%Z}HP4+-}#Ol_ONxPjk7LO0mc;zc#-(T@* zGt(s2tTuZfXt?H;E`dwDty5Q1@re{Aw+dUBO7kQXpk&z!y_bDFu1$ZCHF;Yecp2oR z(R`J*TCsGOmGn6rme7QworpwYQB!eSls}2fo^3ayKYSCiA$W?Zy!McOTH#i8(=qKg zsTiU0m!?a8N6);enYv($S*yIJiP*0%Lyj*~${OFvY@s&CVSMMPDEI}fcNdLMe11cE zwAZgL(tqHNIMsP1?N-Nrn+y2JCl0>4u~Vu4tv>qVVWd7&y-qzinH1_y|uy6nyRynEO(dpqIZ$RNM*$r)n-WS7}Wb!spM{M zCL5aAUov;vA`Y~CMF^|HCrC|+-oKK*`lEz*SN9{1H^(&;gYJB|K)ubW1QRiyAdogW z*gyqU5uSF%j7gN{o^;kJZjvJZosGyfT$_^*L33(O`2^MfE15?yI&^H)>+vXjZ$Wi|Uo- z+dCl!E8%a1q7ttE$nu9Hw{t7RGlW{$6-!d}GWAue9xOL}hS18PqrOjEOO9(+2dfCP z642hn`g-d(XV#kT6A#uc;Y#z2P~Nvy?2dI^JV#f%+x#y(HJ49 zZ6YQ7_QSDRxEJQ+G3!LSlHI6c>BHqlWthNLrHFH8(^OQt_iV0OTM<%&;i8;m*<@Cn zUSP`s^*1t5OVI6c$;jNq={k3VIFsp%F~}5x>=$m6ei3ncWJh_=P?OCf zg|)oil)!f(i~Q%QLh%nF7Mjo96&~r*V9r)p16hqG72IM0FIq~5r92N%@G`RBm{(#u zxKFS}yR5!8J#*KS@uhJmai=K{%-7KGc8@U5trCW3^@os@DZaBCQ>J8+YH@m*Z-Z`Q z;|?Q+*1dR}WGqr^gMMPmUeWhNT`SJu0iQN5wvVt-&*`ge?y{OeqV^_x$>G+bz%kjc zPD;FwG`!OU_Z9_>KV3O;^S;FDltir0ZXC837ojJDhUw+8!}yS6HQhg{NbH8*Y7%71 zTQ$Ae;wr6)!_FN#afQ}W{HU&$?~cm-svSe|6m({pnL!_Qk+L|2xH!Fm8&EkUH}>8H z6xa}3f%9+0FC0=aWu);KxwOx4LVAYU?!{E9p=LG5ei|{tXKyq@4f?=>q|+@cY9z0p zP~ThPIxMffoSb{5l9_Ao;YG&ph0Armen(anL^g@9PNnUQxitCEs79-)FU zv3;+}VMT`b*w5P*;0DJ7^LXp2OF?&lIO9)ePGMY*-#u1YDsb$)rXEvMvHAfVKX*7nM+v&;II~lYAEanC6t&1L!o9ELYbJB>xgQD} zUC~QqX&;}jJh<}afl-l0nYwQQVByH^-|*BH9OM4@kz0jxf*w6aNqdI zVq1BYad6j&3;Ac=x`^`%&19;dF5Kf@>gVCT31QfZM{S}*#tc#4zv^&%sWb;sQ@kza zTvgoqb^kG9?)&!iDhUVPn`(0IH+ZFF{adq&Y^zAHx1_%J!LPv=zC0P%8=xz9Tbw!l zaCbAWDE0RYb&}vs9F;W-HK($`@*t(1b?VoR{`L}Xyy{!$Gn&2i##D|;UG8nyVvIM_ zscQGe5O1#Io%qcEv98ybIi?2}sC721<2yO+7pVI61&<`mPFYMCavl)tEQh6suCa06f z%2aGD15mS!?UY>DiAowDT>ph{p-I0M0tOvEnWR>(MwD&#!b^M^;?QBHHqGEG<9NSt zXewr_2)Lbo%o!#OIFXY zS1BK47cz8>Xchpz+cK~Z2p$-Fh)}W1)4OXV&a@|qUrqIB^^w%iF|{$p3IDKA2{Owp zS_>-(HdWTe8ogL9DRiCws9N0OjEJJ(--x2iN;!RcG1sIvFHQJJJTzJcyN=}y=|KI; zNc}dgAPov#C+aZJ=Rx!zh$2jC+?yw#1gpHxmEYdoc<5rxtK5fhyOH|k=Z7nfX)mN} zO$jViSY>LIe6Pqt1z~3j-q%~n5w&kj%Y?6vtB%A4I8NjEJ}_`=YDMjPB%33BMCp5m zL97lAo1gKm9`Maed@Jf%9ykifp~vgmtn1WFVX6WeiUe>dleNoN-~d-+!lU*xoWw`u zNi3W{Yib)(?57L=qI!&uO4yuwR|Nmkd@?42#eSLu(mQmFs6CSA_2)RN<+q8t&3R>w zFJBq7I3`uR)U#62QNA0uZ50W-?D>uK*`18tm9F!Rs^O^L5qbgBR};RmqHN`= z(q$7vtJUTrq%mCP<`r-Pl++}p8zs*+H=T^NnhYlW_UtZDl3L0kIiD%<1MJ7! zN`O7DtNEibd^d6|U+ksi?2B4MHK8DKWUDNMv8Nt2^_U}8ze)L?+-vbT%O9!mGjqCsDB zW;!a5-VQmlx$Z)P@<@`e8l3y-b(ubg1&kHU`DO)jsTTB0thQRN)H2)5-S==9++OF_ zqdGpHn5*Hi(_N^j=@RBn}^Pci)onr z#vIUiF?yXRkrWA^57*HQoKm%)sLa5~@|IGwtr~?pID`t7LDirTV$F7aje1H@(K<+< z-;ibCi+%kyPRfT!7l%Esx~+%#vG1LGC6}8g!Dr>#5;k^6G!60Vsvlxsy*6sj?0c+F z0Oh5T7$KZ{Z46#^m_r>6V=Ib=&*kDXJln2IE}&JM4TDRYOANE0oSTc?YeUt@R=Kkp<3wE56ZH9_8y0%y=QD$nOM zmTaCWVOE8v)eUkjr{I);nQXTkUyI`56IVRhqRV<{ZG$TMu5fpHdRS8(Ow>`37wx|t zp5d{&GHDWdt8g&J<;C+|-iISCB^n2K^OMGS1;7Y{gc0N*z!CTBL4ZJ#-lW#I*rm$Y zTlRBrj;uTflek>vp;>TS*_n*9mu*FKUk^U==LQQ{B}3$lbe(vHs}e&Oa-*fpbzvyc zCsE`rQ*g(jaST>bjdOp3w#+FUHm6x^w4Dk`!Z}zSrj6g~B_z=LPl4jJDdeR1WnbEB z;0jDuStianfx}AL11`URs?i&UvPc|mXt0-6iMPbg;)!Trgm1RrNK{*L&lSi0tji)7(>Y!*w)|Mfx?H56}A6}f4pSe9TAe@*Ui{|DfNzc1tInJ9JeJ)5(e6x z$=lkZhc%Is@d&Lh{l;9-kJlrkFp5{|CzUNwm3*2yR`HlPlU(%^$@ncM;Ij0HcM)WHkgLw;&84KzJdsnC( zEABdXWe{?)c?U*!c@BDQ*=`%#rj}DUo`&hddUa&y_6mO3NJLFSoWthyK`*RCG@(*a6Y^$N{zGhB4mrWX(-5d*kkFt zgSB(bVjH4mCkEOG^c8|JWIA~=1-PY5E02-OcV-k(c>(7)seg;}%E=g)vGZ~LK*w6+ zHi)9bpCrNY@FhV5-o}S}&h?Dr%rCT7QE3~bStJ)6$WMDRe#Lo(VDYZGnq{+rcBn4p zNI~Nq=J=C=3$FkT`3qSof1VNn?09XpFoYN z*Uz}U?e7+zb6Og6vU92zgD6-8t{|5>>HN~zspaT$T$n6SYJ<3kg$POE#6n#5GdH0Wot|RvnXSg0niix!{5aF=F6<_gQJsA}jl1cz3 z$oH1d&Cd(t^sKv!mkoU<`<+f#Q7rnI%8xV z?MK|EKOiN5-^WS0@{8Xb@|4sR)3)%1i=0>L2L2Ze98 zvW-SsAf?R_c5;8&yf3(vY|vPFVPQDkXo{&~G^Tpkn95&7$Nlv&g&Bchhj77PMLbDT)ND7(Pq138Qa&@VwTTAU({rcts z=tO0X^u0n7prG0aBq$JSa+aS9F1JPkVPIOTXSzazNUcf;%+9>TEyLOS_eFz5&3mqF zUhirOBC8k(a!*6&5k#)=*fyfQMD$PfR6L;HD>nV!^xb4^{UfV*8#-O_iP9}|aIPP$ zf1#9$J_*gMH7RPqHy;UE&uxWF#kaoXBMh!{6XzqphOl;e$wn{N zQW>GmIU**1tfdf3Mk81@(BK?Si5)i4)X$NAWKb!EbPoA}8O#RkYUtK(vSZ|yS8nv+ zA=E0k&Eo{!F5tXcvmM|2BA5s{7kPTf*QLRUjTi|_3Nk3Vzmytb$mO3){;8b=8FVAS zVP6HOAEY&yY$tg8IF-@Lij3-%v9gBZIg8Udh{*~yuH{m>m1+qw;m$G13>JOr#2hkF ze4+B_X*6;*03?`Frm_bT`%hLdhJ+zj>CACx+2&$f_2hkBva4v`7`WyTt zx2$1VUj=kxAkrA<85S<-XykNF3Y*`OB7&z3$7wjSrF;EM%(@^^Y8Q@C>K$BoF6Hqj zgj{>T{5k}%Sy3LBCqSKICH@99J5tMclYR-7j6uT1*2h#gt*U0HCa24z-!CA3N1WbR zN6ti4F!2)t{w>*0s9}yk!cZc(Ex)nVlZF9Rz^Iu)H_Eg z@|A_hY+IJaUKjDMzYbut3#aNSj>kn?2=>l@eSDvcZ#-D(9z%`$7hmQC??Y5QkkGYz zSD1lqWvOD7ai}0^B(s+aLNn3cLr4-hn936&p#zQ#r4Vp83IvZ4>J5&+T z@wMtc&-;Nu$S-l^Nx$eD6%n@xToGn?z6p53dC+A$hLP9kkcrbOoadTeSOS?~p%U|e z)@AaLg!|HHUN=+gPGd_lq#@M-^O!YmQ9Bt`S?i}J<}e8-vd2U35t*>_#i$;D^H^#w z%kK_}NoPL4s=ZIL4ANs*2!ke}`tOXyL9$ur-Mdc7kz*2xB^ zy?A~Oy2d}&kN=+f>16Raw;!5g{NA^4*!H=vh#)B&w+rp_2l;|59L%(F8`=Z5Jr}#S zDg}gI#pz*>zy3%&0GXCIU}5Dfl9!OIelO0WrRrLD@1LN$Wi6j=Ng@B6mSCA@-`6n4 z8)lyE{BoLI-N23@NiQwNbbcy!^`X+}r=mr;dLo@^>K?P%goEpHm*+}IsSXk3CdaZR zQyZrdb_jP!#%gdPcHJb|Cyn+3C!0?M9H(otMo6k#OpiYq%F^6pJ|*`d2kUF-LpXqP zY#Q}(TLcrDq>y`a5r@4mIw_}_d+%L1dW#~XiE(tT!_h9tK@@rzdX$k{o8Z|s4hiWG zQms;uK`FUSH~DJd{(wM^h@(z%{RZqo54gOM1k=B$)ZvXh&P@6mG)_NFlc3%P2bdNe&%cB&Ggrla}dUbiQ)@I%RhLx!pg>V6RML5gq7lIJe~301d^+!={ww z6Fx%1LU6`MC^rvy+EL3YxUZJgrjP{Ud1;lKPRyO_gTy}-X0>g-c9q%&S5I$)yE9-Q zj`YxoB%(*%WrO_8ZmmI~JFf(I)6he@l@nxQS}c8@gNnQ}4T@YJNN?qYzdiTWz15cg zfRr5%IihwVb_pO!(ohn!=K+(Z;-T*k`Mw?E{4|FG`r9^A+aQ-Y;-;_BHuV|_zSnwB z=^bw~TJmHmt8Bl(6OLq{Km(1mpob4s8Q+r(c&|bg7&>v8oceqEQsW}vT|Bwb*XnHa zzAS#POt|8){97FM+j?op>SVZu54^sW(!Qt0dp;r^O#KC4+m7P9+azjxYz1;h{bP-< za(=@k@OFJr(LO@lgt)Db-4*nZE=$JAMBLw8d%U)*y>Nx|s$8~#AHIoy>L)eKJdtKK zA9IDegK?^!-7b(mpn}$8m}@6zh@=FLSbh4xr@xYFR5A~IvA<` zjoCw2q4I7;!Fch(lYAJCVYtK~&Ui@Nrttf=?zg7Bx(Oq6hnJA9FGQ43D@YV>UftX) zjzkL7ir)B?xQ3rNm8)4@L$sf4UTn52Sezy3YIA>>QOsp=I*udHnM{u{-P3>@qqza0 z%y6KRF&S9<2-%sSP5PQoFunz%PhE)c8Rkd(FRoqPffhh;TP?XT+p(YcNm?E*Ae8GQ z-wP%}u*m8)gZL(s2v3D}mUf5Cqygv2hxvq8__32d0!A#F>3C5M#&Q0nErZIEt}Jzo z)ohjO7uGk=oh%I&o77crd*HAdL%a-RrngM_&F0|5G|KuVE5QVV?zsx@jL-KaeOJL& z=pBTnk%rqk^X)9=rnHepe>%>H#Ae^yz6){hScoUgwQ<-Jlc&>pl{(#^PebnD^#y#R z9*f%Jt@uz=2Z(HKXc6_Xjo9C?;E;&NXjm!3Pu!w*GpbT-!`WMl-dI}(eQ@#E%i%V^ zQ2x-*Q=n1kEa~=`Z#v|zrm4^G`V=dH+$12L?)Y_*3WDH|)uP zzIheGN^)4u_){xi_Lp2t^wYW%QdyQ?2@r)wx z1p-JkrEGW4f|ptq{^s6pQb?i0`fVVVZJR&Ev6Wc+Mh(WxgwIuW8i=kwd7E^F7>n|A zT~!MHwir@+2QJGQx*otA`-+e;Fz1F#G53VTi|Q*|+!AouXV=aq!$k9O*GA8MVpA^(iIwp5ldjER`)JRu%QhT7@^>^-;0&R2zSd z8ZA$C>`NY1N3FjO{`Xz@VBB@6(Q8m|E4W5C)Ci3XvQ<*gYY<$fwtl2~*EhJ>GQ#f= z$iDqo1vIKBqzjO1H`fCFxb$S)%!m3x#1jQB7Gz>P|8aoy<2{5j1!FqOpoP*KQX~l* zJwrND#~V63atnWi6h7<%T7B&hY1f_=!|e!>50|#1E#oed_uNZYd+_j^%q!_CtMV*s zakRDM@Za>&`-h>59soi_-_qYeWOy!)P;_ESZ4z(-(sJ)7+ahkW^#!z3S@hN2e^wpk_az(`5%_`bzwBH8{WD$JB%|nuyWA3KVJY=U=jU zImA?KO>I5n+rK4@P?EZaMpx)vuaWd&%cJz%)jR$=@MzZ)I0dRfly8@Qhxg<)VBf*I z?3(Rl2^!3mCLBwJqq&`P-Wm*5;6h2qne-#^j1*B*K2&M3dBHuNqso&P-`efrWxUUF zUu=d7!=$P%dQ-W`HI)D_BiA>5YJKvkHUYmIM}OLl_mct>f|}Lx6q9iKf|XG*h(%M; z9m~U=r^V*!>dC6aCRRP4w@+p-jj(iuTFx#P-c1q@Xz2x3KpAz~loahUcu7C`Vg@?D zbJ*!^+v%ZM9dv)J8OaRE=(vTp?StL_7aFAfLwQP8IcEtg)bb>Yy{1cFMp72sL-nQNlFHSkB!RZXT>$caH_0Iln(ZvP)aARiwoSjZicRw0Wx zv?qq@d;M;c-hGO!zKMk-e8`16cctLK*)F@i`?0UyOWBQDbQmxRHXD{D=o6S6g;n!V zYOUQ(k7%Rwe;m^_iOh&7JC%XR$ZWoE2uY%h% z4i_zIdcPr=sXAh~0pzaszt*)8IwELl@s%17Pg`PQq6NLa=N{H8u0uj_EfFQo_%2Y; zDG{OmI=uvZuSCeoR)tI3E4&S-Q0L*z<))Y;XBcPU3Dfz zEege99HQ=4ONvA)9`v4w-*la9h%oa>vR(PX_loIX@16J5qar9ImzMf+YNE7=`dU*h zF0BT4!|3tX)^&~E|DB( z9*eU#@|mcRQtm~G2fx--sfJg1MC5oK<+bI-59!1jq;lajnkgf}$Wvs5lgwdFyq_Po zZ9(24p<<`aIW&BUANj0va^o7FzmDa@j4%gsi!3DJ?jY|E5cjDbOX@Ei(~m$WHDIL8 zDOC96f-Wy@BN8M(N{EfPnt9!7&&Vw4K%Oy9M?n}HO&UbLdP)|1Hu09oI?5sNTp}Qz zovn|RP*v{dSkW$x459SW?o%cSm;C>={T|mF#IuRIOjG`@u@kx`Ux6 zz;bJ0G_n8;iWaj}aSagxS1I z5&`HAm-+4Yj8@&JEeSLU$;@Rb*cTZjJvVs~dM#Bl<|v(Do`G*Le%TH=&hqPe_mIz8 zBt-coi{+fm2|-5s*D>4{-?tM$%4_2ekBVm*PbDixE?!DzL2wj)G*BY{36^icS{aYB zRL`bKJr5gZWQMy0LbJ2o9;?fi-y_FWNw1M4&$ozr-9I9swM70VhPaE+*4!rmS^Xw2 zPJ;PhMNIzF`!Pq+)#MO9!R9RBAq}J@aezL2%*U3E-l-5X9XTe<6`nO2yQ6UF$eV>k z4)T@hEFXG;vst}zN5xT^^&>(Fh)(uw1cIsOA%LhPM|La%nK#jskKINqG+|ur{naHS z<5Pq6dmjsQA7YwFgl4XL_Fr3FS@wbE(QXTU-DX04^~d}S*aHPb(wZkw9;(=nkVN&8 zx~UI-6VhSq^zb!8TUj=|k8B66a_NZN@SSCJXG~B$yLO%IW<)`w*!l!ZQD9z=ybr9m zvb?N!1dp#g0`qcxaw7x-UZ(in5to>eL3bwxB!T$gfI13LCcNd=KlTiviwFX9S#zV2 zy9jV&c+M&MG%pO_h)A9R=cROth@iWXOgK3{u3dSJ3;C&woLgk9at?jWbO(Yivw_MC zcKjYnOJba*>n%`TQubO}T#R%(%MWh_ZLJCD8*4j=3{XgcA|lUliUNJPm0wRE`sC^{7-I#uiPq$N5Pl+(bpKfaGc35RbhKxKiswjX)Yij-R-gKwzj!J2UcB7y43zE`Fz$fe4qc`E}eDIPM9-`3tL{wlsO7?8Zw6thGjl432Y||7bj~5wSlo@chw(D0yQL%Q;L7*@U z*=(X3Ac?xH9P~kZii(mFr%-JK4c~Tz(>SiClC_6u*5W zCuD*00g-+(XCgf#&w zA2oKuzqLqEvZJVrJAiTm|K_BA42?F*K8N-L7LKHU*nx+ybcAY6zX6#b)R0y0`2*EG zrSmY+RBaK8NF2fz$vZnGVDtr8-5PC8LV=UmoX1f-yhQNuGS^WBb|MYPyR{C7Kx()ZQOudc z?dmAEq;LT0@)t3vbQna)uD-b+6)@bYFp-5NI`V%Z;bEe?xEUpsw#;S6S8xF^(K5L(1nLQIK^BRs=4CgV(=q6r?I7>0&JU9#8=`}f!lBNqD3ySA+MHx|YX68A zf&|cf-QAYX9h~r4i-cQ5>uV;mCGX`?niGi@L}&z75arp_Adh_7OyGhro(uaV%Az%t z+#=>X^<$B-%1~j~-xi>cDmo$tFmK3T8t+AxcSaEi^;~tMH4OgzhG%=msjy}+8X#tl zD5v(J!lc{C5FjO$8K}4lJ8A@>~gUX!fJ3UOaIrd}--aJdsA;R&~T~SoN zW`T5Uf8By2F#}zkP;RdlJqkH75ZtX}c+;zi49K)~GFu-8PoXt-OnGm{PX_E1j54^R zunh*UZ3QUeuTk|w{FfdxqU1dq4;P#9W*+tBMe1E@`6~7Gnx?jMEhJD5y5$M_2}+f!8^In zQI;`IK^GS=`l1EMoJDsr8%Obh$?3%3a42UF<%)_W{9qf(!*m*SrRkE zfG3I~1lL6r&N0Z=nS;d2x}Az5(FElNTdVR^yoM!%T_rYrz8zWKGe+3f^!`hTrKNMS zbVMhv8UscXB9p*o8nOS>bPAy|LTdswMP`&niAE3!Jf2!Hg+Rxx4-CX3!P zKP>bg7Wxkh{r}8DcsY#9F+{}Jf>;tNdQOg)ngL)NM4w9zL4NtiBj|@xO0Z8T8S7E0 zx0%2MybKeuHPN>2$wx22+PuDf6jg^r8v6qEv_>Mu5HY-A&~RqjoQG>x0GR4AU|_`f zr~l?b9d$`ezInH%GCD#UMWEif;qlQ8>q7=_09Y>3WE{c60)pLkO;K!kw$u?|ab#Bo zb%G4}7+(9!l3V{I7=}%mvPU#eqBCPay;%OJM%4F4U}}CcVM%Bwv?MX}EpN5g;RO%= z&qmQ^0&N&EJ!tmaMr6aH5DT)VxQznch_9TqIU^s9nh^_k0~XuB2kT-VK=tW*hiAi+ z_GE#2j^7gS8cdP`?4Q59lJ($t*#SBRPq^S!D(!?Gz@qmi{Wn1W)9&(IXNRr_TQU=CJ0EMjnOAjbEq;Gm@^veBVGAr|mouG|Jl z_<%?)FfOPyiiBJKq2x9USqQsV+%kc8|0H`3!OTK+pYSSp9WamX0zD`s;^RbQXR( z8`Od~O6htp!M(~hzA2ht$NMWB%uui<5FoYMzP`97r-9QsU>+Rt9$XoCRxrH zHm?I0kEJnSkMXrA0vJ88?Lze}Rlj+V4XBFUu{H)JQb`A(?H_wO5I1Pu?g1~S6KfE%b*?QlI+4ILs7oWPo)+tKSD!2|WPnd(`Jfw!84(|*U#_W{iNZmxLO z8sb_Y`_^2o4Lt1t53ae_VELv2!C8-wJludn1l_xUOm}Hot_iXAF5pKt-faJQ;!R%P03Y9IvOQ2No;eF4 zoD56EncKKC#vuG{V^Tx$LdMGlUUT6s2KVxp2bBTnNH=XHTm<(1&xEx8Q8{4t1c$fo zcv#j^=6YB<0o|RcqG&h;U;(BG2+H2GelP^6^&IjHqCF|iT!>x0agE&iK&ES68tjrj zEO;&GwqelcAQ)}?$CAv(U%|pn zUcG?DUT>{+JA!}Zy*voeq^LFTOcQofQ`jtaBeTG>bwCj@y(4&c5rV(~i{zs@jR3B;F_L-2qu6&FDN$t8)1Crbk8T%edNW*{^4jLlBF`KzfIwH7Lljq zURdRQaU2o>LJm28);8-=ISJG_FP15VuK#b;280FS(4wUqJyY2z3f67pI;q*i20N%Z z`3ZmYvL2PCcCa~vU9M=Pmxuv2F}T3QX5Cn8`(F67idh$!UH`Q4&mpZ8u&E93ZtjN7 zhCs{Dz^ZOI|k%z-7k+)z?svaDW9s(tSO~+*e?Ak_T&zzb{Esc2SL}NdR-(w1vb(kYa9*H zZ-C|uSf;vpPDmsdepzpCt*3`|{>TYnKGC&s{`$uVi5_ynZ{1x7(VZx@0)ev@gWUiz zgfM|ITN;0jUv5~Vq;Uc0wSrJ)ymj|y&t)^gPqJeHY{$hP8mHH@2@2-VHut({_{Y|iX{TerUjRFmfYQ^A3{2NvNb5!{&z%tURPVpOe@@G%8G!n+< zhg-G35lic=)W7lFX#;`yYDWAF(Yd_nVMO}t_+!+KAR~_4Uq=~4>mBO#G`24G@N-yo z9Sgv&t@Bkhz?RV(3UT}GzWmcQ{ExudxG(>Y2^=rBq<)X$rO9-omHGS7vtwLh^+#|0 zk)<`yYaNYvc@c0)IJ(S#?ySDwbIzSaaM2>Dyfy5fuDcvL%LBdQC9_LCMx`8hy|A7E zOSgE19VvghOf|JSW4D+z;IX-J`zGuTGiy^jdt0;fmyu647ce_`c#hL@)1sgF_>T+l z{`G$Wp0$6Aiqdi^nOR%dU!)b_h5xD1a%nhUF{9}?(3DJdH(8(Z}&=S|FLPvD1D zS3YlNNy~G5ZI)A)?Cs8&*-F`5zG7o-W^E6Tamie=f^pz4GLW>HiH)h*4sr1vYam_Q zY+_8iYttr%O`CrO6kR+X;=f?z;pYDpNFHv1{|=M@+rJ4Wf_FgXzXvAORw-7(O`D(p z4pKY>D5%H(0`PX-U=;Yj1LGP6{~v}CUHp0& zh4_A@SfMpa`q$X<-zHdo?*A!*MFCh(uY$i3KZ5@)@$>J)`M)K9{*EzG0M^6F!!7jd zhVt;Nq56M%NB`$U5DLM1Pz8Piso?)5K>xR7(BEz;0>FAWPyB{-PyDw?_uo}1{Qomz z2wgl1BhO#zL{RWosuf(D{a+$lUNANO2`0B7?~b2M1lnIr1RjAkNum!llGB7c_ zV()O?3Jegu;eeVfztN~Wes4I?{%kmm6wQwtmb-N?Yft;cm+WIX^GMP&o@K|rlRHI; zI_M-xL@z#ICQ>_ol3+H2lS|D?D3e_OETz6ypR$&fZUSX>M~d51O1+>QwS$bu;^h>c zyzmWqd%7#}6DUQ$aZ|4&L1f8|Hj!VTIID=vN#c;1w(CyXy2 zY7o3fnJ_*rn{`*`&x?&D;!bXtPN`nKU^W8=~HPDegEEj$(D__ zu#688)L^g;Qti@~DEoO0k$f$y&18!c+5o1nBr~vme|N zjYg%ud;L+GbMM(~&aXHWXBE$niv3c1qUn#av(fIC_9|ZcvjiLkoLCL2i}_^=MbEq1 zqT_oNFMcge>z=s2yR03H9)xcJNQ>0~z+@jq`u+0oJzbJikV)~Hnp5B!Ac1dZByizt=-%OEB_ zmmEOeHka9Cmes-hcf=A&ct$|+(qpSSB@J2=WV`qnJu`JXO#TMCQd8nbR@h1D_J`}{ z$u4L3n+>QWlWgbc_staIJL~0-5PeO^rYiGG_vQbFMuL&0RFRw6CA=nxkOD~%PQ5v# zw&8H_-;gnrB7hviTs2T5y_MG+sFquufF60yi)_N|`HmGBj7RO6siKCI$BhW5VEaA|5o|jlZZ?Vi&laO;aYYm<+&g7LuYx#6B ztH!{M^u3LG>RY27T6BwSb$Gu=+|+A0MIdQR2Yk{Pwb%O3DyNYd5ht$aLu#SjPyt+w7{*)m%>SMPqaqZci zL0mbkV~G;WLluhrPd{EXJc+AXvHBV`S##2XCy1MbDt1#&@Zo*ETuPz3p)69FAWH; ze&Je)XLXe|Q+H`zS?*qSdw@-B4p=yBHp-OOVF?$C>wnl*Fm;wquU2Un+tvvU?qi&U z_Ay&s-ITIezle+%k|WI#f)co&>h4UT92Iyf5Lq&TtJ1@XE9XAVY%sRvYLCyHICKgn zi+@E1*Ka)|0Cg$Z!WPn3zCn%Xg_0ksHmBt{5AGUOrF-Nk0~g%u#kB8TTV0+lZ--Wq z?=6sKt5*l>q7`06R4qgKqSZcT314oc&F0hhtnqK8>3sIk*wWd`#koK)f~OoOXx?>X znE1pezUo8RDC9MiDe7x^T?SRErK=XipsLOtPm37m0W7EfzM7<=CZre%(~m0l^b<7m zmZVSjbgpJ6*+kxmmkXOt$Xrku0E}iCG)yGv-+Cj^9=nh@29?^he22%&4%#R!_m2hT z6!%iU%ZDBx5{vEi;*b?{;h4gWLn|HT0*V5A81Kk>0z=7(Q?tmu1>UCc+z8%7&DzTi6sXG5U!_=?pwH=X{JZO&T-Q@o@5p1d&=*Y2r z+(dRL?BPV%i66JQTwis!H&qp2U!3cmS6!V}eJ}})$}pPAcNd_n&ROmugJ$(fd5J6e z7vA07D?ax~AiN=rJB*S+{3x5p(1m6e1ub0czU*(T^A}^!G53e2ezt=uzb!KIV&nOP zhh@X+IyGjX3i-QNk%y8txwRTP@o!E3F&>`twx4^&Yy08AJ3}{~^~CSg&&+u!oOL=0kthv9t7>NI)x`8%z6N>^m~X3K9%C18CqCgBB}pz)H49ov zNqF}eieS8kN?a1%%}-eLxSvJq3EF&mh>31dqhJhCmi>UMMA`&LSj1dis54X>!D%KV zMHd3U^~D|wtRWQPq}cT)(TTDxtOP440B&*>5&>Oec69GbXPDB zRBO63r!8eJ{xK^zGeRM-O2d+Ss}M`7QHi+X>OhoDfI3K^d+O9I&I%+>qbw&4`SnO} zrlU#w3NLCW^I!Hf##D|%^_*uEImcGM_m)eY(yU7O*bvWqQ|MOd6QV~}WjXa+f0hZl zh^fI?0`iGz@Np9s`%-mrKWo%NwwdV@L!DG5R<6o(=XfLBX)}jJqChm>vERr3QCxem zQTXm#{xCXC0heDKT6&7}5a*G-M~Zz^709`{N7c1+pG znz-QVHy??m^AB-1g6`6se|ys9Kq9Nor;j<7+H;G3@g430%an=!8AQi~G`$q{H6ppJ ziTqVJnenAry8l_%)Z9gc?K}vTB72_D8R5El0J(d4%xO!yZPSjo$9ED3sJ)m@Fq@gF53giSXgM@qHlg=Fojn)-Of>n1A+{r| zWT}7llM7c(W9FII9M}A-y)rHxaf}a184WGIWkpEsJl))7}y&u#`(#_jau><~Uat3>1F9zP7%t?3S+&L0`xA$@K zQ%*L;ds0OH@&IhX)~iH+CjKgf_`QJmhiFdz$gga0UZFEn_jMVHKScI1`@5B5Ku%SH zI5o9HHk7vSc+pXKoIn)HCz6&{CG4jVjxS--HJo_nc9A|hpemtH``|;j42@(``Xs&A z9XA1; z910ChS{N#|l}igPrnVGfOqabu=Gw^ZsP8X6oRZb2+Z6whXb+JRq7SbRzWP^~dMMgB z>z-Oc%~BX!)WUI78J((-T)rY;@;dRw5f0g%w}&EYp{*{D+WVNNx^pLxS|fCx{ioTZ zhFWq++YWXn6vULV4t_e@J{`vLgQ9}3?gV{K?X8?W84DJ)=_M=I44j5n~oZgf31TW|X4j0#Qv3VC3#a^hadQLl-e` z%@+N}<9vA%nY#VK^C>zRbYC28&g!09)7b5rl}G<9P*}-9plmJvOTXxKw~E)NFH6)8 z?lhXaL6FBD1ucASZ>77RoCINB23=NYBh=I$HHP#KCC29N23l-Pai`{^yKbk=cv#$x zJu0Xzo+xPk)TABh>+_6G%L_l^KTbO>j^*ADqFH#uTIx9yi~FYF6aH24O6(z4Bhx(^oVZMm#1Q1mt( zXO)m|;6C4f&Wa_jF%c80%*nCaQp&*2SS059wm!2ctq?K`Yi9Nkg?bTh7itTS_>|z%U!bzL=9r+%j>N?q=Wd=ec zj+!ay6b+NZMW6GP?-!8r<{#tj&7PjW+ojBjV<9DwGM0z6UcEH6^UtC|3lR;KAR3a| zT@(Y=sGc5ct{bfG9%j#(;eL&@4LYgBx1`HKe1)?2RD08PB~H;LAdge_if=5LHG^lF zxhzAKeYHr{V2{?Jz`wBnQlTU39Gh1zAw6TD{X z*f!j>-9ys02yZ=HdZbL|LTbzSpv0>4=2q6NS``hHq3uADWegzW6c7H_whtJgu;zGI1NR6{rSFVs8%IpW^pfowb$m4le&v* z7Unco+H%MAVNiSb;-}dL^$3?A-=A;{#*Wp}+Od+*PVU0Pq+FI0|)(s0CYsZh;)joA$_te~~2b<0R$Z9sG_BEnQO^xvTpQ zy*no!Do4Lnvi0fYG(Z3a>IV>_g63?67|jY3#1Kjj_Ik5HMEevuEsA*MR^0J&N5`s_c4b^a} z@i!hH?H8U1eD2IH%RnGw{zA3-db0b0d4IvMu$5Ad@~h9&nACMHt&yjZx>V^y#4#YCo=RPfDK^ll=Bju$|Z`1ivE9OH7^s(xUt}Csb~AC$`0f zbw~{Dke?^|od0QHfn%DXM2~O1&#DD_wa67(@oWJ;RJvJgB;x}?br+_;F4j@_C11w% zESEXM^sspH=tZ$Au(iazrlz_I?0h6zw*UL;weCVV?(<7RHNe56zpF>x_E-Iy;lUgO z-weWiuO@v*f*@6oCAP|amRd7La)A`9+=!_FAg+Zr$^4AZG}sfXoqP=(LHzK&VU3!F zLJ}JA&2xI4QD}F1!uJYUyY0YT>4k=7H zh51=(-|(yo$b^a|G909kEJ=v>98@bO-t?bQ2cwEztNRwbSpE_bPu|mG`c8$iy!Uf#O8%YGp8J7wa<=#)9D7=luy{3@ySSWt`|$| z3U6vda8z~Ngmr`Cp2N2HJ5*cq)P>^wNsluziu+}YJ)e+VWuY^V5;5K7ZhW{qu3IIA zZZ7S*M{RC$&Z}!p$2qIfHoigSDQzud8TTn#WQ2Kck*Y_xb90l~l>tG*WNc;#sE>Lg z;91|Rn(>{xon?HmVb`FvH}XsMA3X|UNR(S|s3b4ja{JFcHvEn_EOg&jt0qlB1QmBi z2K6LCofuf9)q^S}LvWhc$^>J_n=JfTaMJG|e}06n{6Q|f;?Fv6?<-kjl9$9u7lY9KIum*P6XWX=IRTlUg z?B}{a&sR>~t)>M`qf_T(FhlWg%$}cFmrFVreZ%iE8 zjlGLqliPg|9kGehu)?;;av>nj=^N%MX676QMCW`6+&Jywgd51EBmVPx==g@GwlBLr zM&x)VOw{@D@4aDGm9=MDIe_DH3wQPqV8trtf0>2&Xe5UE40ws!%0|E7`x($Yl2jmZ zsEW0&|8zW`H6-TzawN00L{N+^#Nf1i^~zm{+e8hX_7ordW{yPHQL`J&8%Seq3us}*N>GP#LV%vw{Nd)DKP372Y0`4 zrIqDBx?KbFvGiPg^=Pwi%Ihevy!v}a9=M6NfFFekS^>?UANJNEG}M$#s+5|jm?7d` zyJ_U98H?0JGX@sL-R1K9E_S~+*ePXcwI`@Xb}Ua_3_&9pcf{Pc!XX!T!niDY(PlK(Wx8ywOQzUr|}6FOyQ3`v7^8CI3~RsJQ6v z<2l`h&|qPoGfHvy=KLPoo$h2Une#c+@;rQ1i>`M6f?oLV>>#LE_KvQ|x+X~NpZt0eW-z^NY3;IO-zFhNW zMYNlzS;n0bMAgsTwC7k?Qh6v!P+@C~5#UYBb4Ise#ksxJr%ZZ**qSm7{{VSF%c-sO zaAe!^r{A-;E$SBq;C$IGk!ZUV)Q_>U+UCIQl$tFTbt3R&qP;aWWG;Pq4k>YsnXq!~ znvXB9x6_l(6NHS_IvZ1Rf##fK%8oL|WP z%L!Qbs;*Y#;Q&ayAgBoXC)?O4c$@6A?|2rs8~C-1(QPFvWkk4-#@K3QX!NtPaQM3X zT+fts?5ZMGMS^T(q?ya!D*wSoD9(^*cuLf#(;%kxhA=#G-&aI z&5ZGpwSPa;!EyWJIdQ+eq5RZs#G^Kf)g}9iqBuB|B$2;%BoCGu^$Q%3*-9NZE^)P_ z%#Ks!3dutWVM8GfM=2^b;sT_hxQxLZ?qLG?SD?207N`qREunSp<%GY}0xl?jKZBgS zr55Y`F;i(yJ~vFa7~(f4Nf1iX!dGf}_e43RFB&9|U2n}4!QaD26`tM=8iZAK1!4HPcz2AO#m+~h zDM1QcOlmmx{((HK@tNpP$u4vll0dY<1% z!WtaC&7uMppFzieU5i0RGDZTzJ+8G5s^XZd9#4P7ln{47t|$0O+CM%Tm!crtc*XEv3@c^ zq?CLXO4Cl1KiIBPtrmbo`0yPeTy{5E5u*?ef4XL@aDdtmx&G?B_{x`12d z8^snl7!?$zXGj=!3(b?5WL zB2HoY5`|vgJvM;B%JXBL&4%z}KNE^ToR2hRD_NfGy?$<&51D<`25ccc19dvMf!|`4 zcJ2lv*dKH%_M-;StxQpAqv(6FsF8y8hqtAjo9`1S0@EhGMGSPByk=|ZKpGJ&Qd;YD z*AH(g0`1(JeC@8acWK&g1)o%~Af<9y3~r#8zPeEhxM4SF6d-KH&KbP@F|>Kih|MXf*#d{J*%ndO^7 zRY&#inCrWNpdYD9t%fqk+f!ruxd_t-@Rit9ND^q}bODhg_K@!ZssKCW<@~ukSb9s3 zy;4Sg$ZwBRO%Z^lS-j_@$XCNehof~WoX=HpB1hwMEVj^E@Ljx{OX-(cVu^2H{DTW! z_^k3#@e*DSJ&@DaYiAh~N|Swxu(>m()loWqE0*|h=5cGttNwB5{p6Qk$fydpk{DNWEYTZ%p%dz3IY5_Hh#HbIuP!$0(9Bg0J%HY)@O zsRo*NF}gEmPbzvWB|CFjPp@4u8FRregT%QLbBmv`$Lt9LqmGvjv7SR8=@Q-us!W9*%V)0DGdbutj&5`2^X z;&_?r7o$7oiJ(SIp7mZKC0K(TKa)tZC;BwdN^jlbs;>hAb5Ti*h47BKOrzkZwh~7n zrJ`CP@TT{LMIY(D$`W1*8E6x*114TQq=`n@W4sr? zKFuJN`LYL*t13qJUX>c<#r)t+LFg=6$P69gvUbI$x;)YQnxsQ$9$ewE;z ze8C>dVWX77P*IwZOlgA6UL@%0)?RwSUTK!%zB(l^GPeTQMKKapA0vv^lWzCp__<%@ zh@C4~)_C+K`or{8VWX^m8)gp^%c& zBlM7;#n|KFW{-z`HQbDgruy#J&u=-WWz|Zt+*=yw3dv{XrK9t19^{L>6mjm8UdzVu z6(9#sIb|ppjed_g4G5ps0YFu})cZa4)~DBKWO%3wdT;*|K`_GiJ~u)TNw=?Cd5*io zG&r(1c8}fTt&vsmBpzWBc&}V{-zERD{v5 z>J1fWSN#-d$o^RZf6D_Dkx8d~AGeezmdG{qP5P$yzzYcdg?Cd{`|vKkGV#JxwSk|| z8F!hE7}zNI(AoQfDu~Vbd&YY>oS|V(uB}IC&DPZ(Cs66`70LT+yjIgzC(DS5;YJ0; z=M%~xPW@v@yAWM@4hF%P?)r<)t}b}_$!WgamY-gbn3ydS1^0B&ho^hXNSF0k_p*R< zN29Tg>;*&1T1?dnJ*MA~SYwC%!%%{?j}qdoC-!Y2jL*jjPgvE zY!s=F@yi(*&kvGiwGv|G)Qflg$PaM{JCq=jCFn{&-jQ8${1kC-4JEJjyO9u-`}vtL zk2GqrcQ$p~gV7}6n9;;uFNn-ye|L_@UF4BLRr9~C9}-;8Y5c-sQ(W`xT`V`~tlrFc^)B`nQ?uU+;xTr#`@{NT z(!6xApFUqoz%5SM1G6_S;WzkphO0~smB9A6`b9g2GRE#cpyEnxEK^bwI3YC%i|eVb zHYzMNbP1HesW3d|8TG;Y6tjvBepU4?IqzXaHfC^^B+%ZLj_DQIF)1{L-sV? z4$c{`)LM-iGr3kxY#7zF!_(Q@p!zwM>V(e&m!G+Eyh6FkbD^4X##RLkftH!3UU0d_5EK=TV3PXeGm5+XbMurLOL3c3#yJS@cv0P) zg~!6rnmlc#E*jY%-rV;%I$X4F7OOs&3@{V?19L1tUZJmWKd(mGw`~`4UD>})QmlBc z$*DPE@T8QoLTitbZF~a{QPYfkiHK71>vM(~t_aPMh>F3Nv{UlS_9DuxQ)$wYO3wT5 ze1*us!afZ92)!Ci3DZ^jSjs#|#bdRWCOLSsVs&&-jPL9rLOk_tVH8sJ2WmwSy|EqW z1olzIiMIS0SP@2*_Rtu)Ju>u{rXKs~W6mO=F)Upma7De~f!?`;^@mI!f!r7bjFEA+9Eo6br@dsthc;Bj<`! z_!@b{?w7;_?yxH*3A{LX_KEuvx_YW^u@VgF6XSn)<4uyAX^j1rMj=H#{1DD*F%mXC zwEd*)H&FP!8V3t3>aUkC{!I}TQ9{ZP<)n|9A`vd7`MNKDDF8>njA!Rj(*BFPH$~*ZI(aZxfgPt zXEF^tVjhe_lZH$hzgm!Qg|~rLeU0kaFO!C5&;3D|+}#+Ee< zk!K0w&(dQ+SLYKr2lP*mom34zD*}C;7-7!Z);HctSD|USq^$Xyhi8_q$gRc)z5H`m zHt=5TV&?09MLjUWz&AaGlT@j9Ezc_Y+t%|Ew)f;rGv{BXbE57{PeIeN^8`Mw8W^lS z1rnJLBV<<}Z{_DM>&!Wb>*3O$W6+xAGiC=HXEKcMMBT?cH%fMKT@9P6k}04)cT-M{ z7b=%3G`p+QiPX2NVmmjaq1_5@hPH{h?%!r*HaSqT96A$oH8rVq>!)Yzm5UkC=)$|Z zgjWs5Y>vVjZ-sMRHbk}L89(8|5ha2bb0UzsNo7P=pd<}kdV?; z;jH}q-BxMDeC-REkbRYKNgKVZ#z}gDL?fN=SakFnri8Jqt0M^-Kdod_*Du}ZJ3>BM zL|;qe)Kal{z7m1+`&sO`ot`W`hfFO)Al=zWF$TB+Bo@jLXDzRfIF)8cdrUcd+t2QI zTNI1i9}L*Mt9nEXd#4Htb(?sn6_k+c*6ZhdgHV-_+>639h+-rTb>hd0aPa$qDoxXHdn#RlKP&>ckw+qELsK&gDOQ3kTs)0#Q& z-FWfnL5=cT3WYOPBOUfemxtU*5_XkG7=>^^w-MPn8_1-0HGoV2uVLG0Mpn8IBLTPw zqa)B#>6Y2!C(t^bzr+k9dmMFETva9|pmnRn{1;Fmh*zH;?^XJ7&#MNWz^&-|L3m6d z_q(7lTwngIHcub=oPYF;$d2{`NW-zWh;S@%G-xc$u$lc zh#sUJR4yni{cOc>VY|n_Yr@as?9fc`h)-bg?F`cmnMZ+GaW2~Yj~Z?()n2A@K$W9} z!`>@Eh&_@0r0j9h<1JUR&fV5lu^8V>%FCNU^kr3@{rwS34huH(q%=h0@LHU7-0<83 zK<6^5QfbdfYJKAZN8K=x=Xut8P$Z5T%Jlq$yFuvVmc#mnhmn!v`BHc7!OG+O1{);{ zS?5YzyQ~@drNnbokKfJa=qH7#a8J67Oq^EqVB}ySTG4CwabwJg_@2x;U5&VwXB#2> zwc4W)5eqG%hOO#wLMVnxYGcq(lrOJVz)Jn#{#5iOY44pVlsMNg;j2GxlmY0Z})=PCwZBPUSY_V>1i|Ruvq@%P8utS zuAnWb=^@9jcgKGuaFoKCA;WmNMe4Y!)WAs1nl$Js;VsKM!JT&ttJeuX%wY^6n@?F^ zZ&B&Nh}RSbM>;*lsVtf98Oc-(Q`@~+RY`W6x#~Xt_37Q&)HwOdi0nqknNscBk3?CG zrID7wu1Rb9+nQD_da(XOnU>n^V8~n~n3%pE59}aEnRK{DemIu;q|sjg@rh%j zroOCWhPGkFV8HNstU#Nfo+A0ZFm0)7`7a7{%{7#n+TDU>ES*L0z@x_7n%2J?EgTzZ z!{4hV0zI~Kt$S2yt(6M$)zKY~o68vam(bq&GQkU)tRF^YCayRhIjGTI9&?6|hfv=5 zvD$lK$HC%ec;&wQAp1TfniMwhC9Z_6_EnWaaqGu{Jr~&qelG48KU{1>Xu5p^2VM$` z#P(V4_mVUM?eWi`W>5>-DhRV6-1)VU-xkEmNJe%=McVR7yRnNNtKL`*2muX)Fd+<# zQ_cj@*`h$L(NcVC)Oj_k;A2A~g=jrQxNnf$K-4dR-J}BGxV8_qGoTjkHNTsQ8vhuY|wSS&Sx1-CpbT zGAh($^pK7NSe21ESkSSqiga-v`jx zPtnNT>PbBC{x$Mx;BvlBfxA^9jlXY^&@bUnpB_>F}dw=ZEfNQ)aMV z|5nU1`!hPV$%MI(!hs|u&mHM2gN(y%3+PWuKLW%4toC5|-M3Z0f7IY<>MUl29{WPO z!fB?{w3_M27L1Z52b05UL`v~2g4O>(?ZQNB6nCTKDB{JvhkPmT9t3qxkpK4T2|7f| z-A~-3=@FJugL8Z{!Et7LNF$l8BAnE-q~;-eg*p742lf?#CFMf!kkouqs$vyt3?*hV z`eBj&quc8#%n&SJX%F8E)qeF^sDx_EO21hiPG757n%rE&Qd1T3av1xKp=LyymqSD0 zMSWgGPSyqZkSiuf9ihgjp~klI?Q7ppF{G7E*LWHPIN$}dnCl9W1{Yk;voLyUw~&HF zcSTo_EQDYDS$PQ9HEn^^rLOr6Ln#62lTNTiY^5wGdQ|I4y}W6LFC<64&FFFCC=UZB z^W_%P+nReE(eb|T2Wrlv#MH!y5!RK7x36heRNdhb+mvcIhoftr>tBoJEg&O~bD!>E z=Pj4ZK_4sa_znxpl?yg$EJZo$jFUjSOJ44A*)&W!Vy4$if{mN2uO9~NVRhtqQ%q6? zT%c+^STc z9Ws8LKkT3H+8bH4)lml8H9p=gr98r_wC5lOpeQs&E)9*e=UX8arUsJD^@|<4+mnr( zxeyM?Wi81Nd!!Qf*)6wW+kK@j+S7y1Fue~ZYY@iLfj>gkua|4?wY`$Ns)ecBor5*9 z>z9~yNu+4oq%Gss*g^caMuTI~FEiCB&wMv^*8DMYZ|`I#8&8w)kbH+$OyxKr#7+R) zxvIkp&cG6a$>jxNov*E;1zRmeVdSkjN^>S?L`cqCH5oQN-<9;0Ya+TB`#7a}UVlp{ zanDLXN6S%MBn{ciM+4ZP+jqG+U(1hq<#3r7o^3R zosr+*^2K`x&nlrS96X%(yn;Rl&ysvMccG>!b7E=Z0T;hd5r!*RGkmn|yi%e|Lll~1 z?h$lcMrWN;SvP!&hBrSj>&EIf%g`J$_*5|su=vu#WB0mbV7};NWuSP+nG;ts>Gl-} z{~7mOyjG_>5i~A7lDvlnS8Xp!S3JgvANxCUEHKkI0yA!$ z$_m{!qJSlviHHK@TmMj(OYZe}QYc@XU@HXAxliiEjT!}9%njB>Ch6r#K+0I1-?5F}qhzLrz`Z-GQ4gM7`kt6E z+N_fFj3bQ`O3qYD2(fTUqFr^#ahiHL&JyvfzQ@c-vY`{-W7XmYMIirIW0Hr`Hj#<(y-#Z;b} z%>z}1qt5mg2#)s&4yQmJ_u5fwOHj(H#Vm1(Roi=vsFKTcTKr-u1?5V99leaUT9Xug z<0G^$^R&O{Fxq?CFv=v|UN%FOBRwSeCqKbdbg#rL*XL^0kDbcd^IeFUVp030WxM;% zF-Gvh$%9n>eQ$|qfO0GZa0y9S1x7cR6$j~BWR4r zNa`sNnSu->(*lpNCv$>w2=>|`$IfcTFVLW(0jOeC9XTev%?w;P<9r>OyLTOZ_poAA zFSx-(#+a1{h#1Dtn6I6dKg3T39VrUI`J_CP#zb>SyC~;tk`F^OaxDifjATj_8Xy0> zxX;kA6>{^Qmp$&y>(ZB?nmxdgM1;I3ws6cqlolW_mWGbg@+z4x4RQQ98Y!KqD5jI+ z(zEh*i>#ujlRcQD5{6^0LZW*TK@(X@nZ@dYTh7mF^H1Uoz~i#XAo)fB@pkZK(3@JO6y@>4 zxOYd+QFV9~FBe?PwcblGT^-+QVK}>s|DvG+ueMSZTRE-UF+%WcV>g+xtVw4f0#Iim zHb1Y>Wz&?oLSw&OzYK;NM9=iQTbWN*3<*+M4kN#&ndyV^#@dyTC$v$;Z={9cd-2(% zf*4xviL0qcvqW(BX-p+d5AiNH=S<@zFaqjZIhW;Nt!Qkz-}$fr5!?07i2FUz>>HA# z^j*+O3uInaa7Hl+pmEHViat;KwxR|`gOO*xUE!9rEYaU8N(qfEypdM3 z+n&ySi4L^FGB0ukyJ@v~#gAjgKe1Q%lL;{+Qf*rFF_r7yEdr6f&gX zH)fMqI^j2H!yf#W6SoNV@~fjY3c8wVjK?yQo|Yj;q~YV|2Xz(r}~N#$JAtf(mz zmGRo~>gA-4*l&as^B24L-Y$3CQ=u4Uo3R^=qt#qicf3jRne)VPp=*okw$M3KB z%aZlCI1wXwdR7oo1a5v9!#azVPdm>Z+M5{I1HP`KA zyxJra5GHZs&pTEDQ{KDuOfeBr-y{WkfJblWqKLm7SnhE_ec)_eR~CPNwjkn71TQDFYdT^WfFP#G-b61W~w^Y(^VRT4$E^o`Oju%gb!1HDo+LYu95t>4b$-sHGLy;+eJ)= z>P&0HLUFR7?{3ZRIDKLdcVl`lg)2|mlStxl-3nVXz|*U$FO_G^$H$lfVtVOfBS@ui zta0umm;-%Md+K(b@C2liS32zWziggZ5&8fQw^2A$O=-FShF4Inr0{0CVA+;>mvxCv0S*%v#(sux%%sG~=28=a%o zkSAr_ia)#Q$QxcArsivQT%3VEYIKqPkr0sZ0>PwXK=sOJG@@W4S@5hLDX1XUz(rri zXUH=LSI%`PI4gAw)T#SM0*rOAOvK`1bU^#Os(z98mFq20&<>W1li$*kK%hDWQw0i^ ziEksQn`_ip#&%x4jPZ+?pZb+Ib}x0Ef4q8#$QmG~pFczIKG5EU7#eIo*mWT$ znw|)6T^jARW3HZN`IgQs+^Gi<;xCtlT?6`topr|I433}gAI;NZ55S*eij;sVo&kuAeR1))+z4YiVyQGzT(5ljN$jMAZ^wLX%!{>)R9^Y6 z(8jvR%MMFRZ5R~|l42b@=Tkem2I}W=Xk?PT?)L<8;8sh}V9dw7tLcbV;|Kc*21D>f zUoWV>z90c&282bYxnH?#1(r8#+bgW0%re|3jgWAe0^l`;NH(ki6 zkCr$qvAJ{L3k<*gWMt*Wr4Gm`L2-B0DRzooy_TW0dcv`X6*}3N?k-oB?{(#p*M$m~ z{#Z1T$)e>*W9kTA|L^&$<$GsUrI{xtbn+GlT;#Kh>puMINHUbVd6-PE`iqZsYuIQ8 z{sMTEYn0qyw*2MAA{P8gcJ^R?tlQQGkgXAF_AY=*=2IT!lPm5cX;sKlTz4G=o$!-^ z?#8+m5m!|xuUkwIN8371wjePr6iWn3<&h1_pHEoEOU_}hOE#Z`Q7R)-U)L`}3L{chI_>9@0# z|A>DZa3*SlT(eiX=3OZ-^15JCW~Ivk^Wyq6{b^?uT2S)`Sknf&x9tPOXtO!c?-4CI zvF_>@_o^WA)l;t8n)WNur12>{(1E~^prnHEF}Hb{|5Zn5@KJF3;B!eyv`ulB_pnHY z4;%B3iqef(CJJ1)uXxTs9$a{Ud_*_S*C3{3e$CYU&k*;6shO0E`N~ey0*W)ke%Fk^ z0vkv7V{aP_uD70a9gGCbkiSQC&#^IlxiUqLt~rlL$UvwjC=Q8B5&Gxod{h07hgt`6 zrH(&hN3{_Yk@SUhTz!94^KQVivD=OJ6)a|cNrHu_D@#yo%p>~Xg7Z#lU!$miS-dhn z+JLxnL=$yHDSFxgRk26^Vf$Ci`uHDt`h~|t!*6s?+CqKjgkQCAC^#{?{P|u^?qEg< zY3o&$6LkR9f_n_p5WI6<4pFCv9s4boxOHol3P`#ja~J`*`%M?S%2h$N9LEn%342vy zh*Lnbz~LFAU_k-VdOTZ-(l%);F{nEY3+TJh5RZp?708iz2Rpy^YW-1F(&%>^R2koG zfpE*4=SeqF50os-^*Y_gFtsnhAY+?CXI<xFLW|%~EhX$Np_>H`+Vbdr7myP*gDw8=J_x6Y1?JooP@$w_#jl>}1L91b3S z_Wr}PpIul8GqLsH@#g|rlCo#|*`S88g|srhY}bp9@bS-!x+XwTWXX$HfPaKHQU1nPi5=xwbz@ zhxOwlhjD_SPBC`V$L|J~-%=Pm^D;4schb^5zQ4LB^2%tTMD%q&uOW(f@v@@m)%Kha zpysbwUpCc{;B!_}`LiQ=-C7QP4QOM9AI|Zm7LVvHPy-=`orfy8VgMhmoRe$H+70u7 z{1`q!U8#9_?Fj0Bu{7*9^2ftBx~7*66JuZsZ$*D>tYcpTAfy2tU~1` zf_|}%NkM@*0rMKh)3I3r7B0;EHckl!R-Xpmiu4O(+KxcShU7}=-tS{yxLp>agY_L< zWQ;C{wo1J$+{_Yl(D+2JV&7iFqfk&$avqcLr*~p@-GEa0<#467-e-x0M_R4M&Vq@A z62UG4JvznasUwBo#Y3e)j5G3dRPzuE>MN%yBdXTJxm#faFQ;Z zjf_cPBXQPl3&bRD!9xTM8pFz-yc;I0_#6tM3uYht&V!?5lsRvuL+shrYtDw(&CR^W zF}vul$RGwSs(hOvMo#6n0hnB7%R;<$gJOO?8d(|~CzSDLoB}K^wNGjFGZq3%Agy{i zzdOJ@P|{e*&F^4~57r^>TijNU^y^55yH9JNEm!ri4G#w|HNhR#0~i9gEXRK$C=dMTWP#onHer99Or% z-V33@7kELUzsTSD#K($C99tbbhd92W%^MY!OmHwo&{G3MnbvTOjAp-v1!0OjcSgzh z4|Vy)Y!7hYLW=sDv4}V14Wk8ppdpUB*^;^+i;u#r7D zTd{lAWUBvjhxVJfs|xo_i8}ALM4rjN<9Fj4t^SR`ggN+|vDdX#J0C)$G7(>^WsGEk zzc_A}Ope0`x)72ZmlbGv!c+egtz-_8Vnib6AKQ*RKtJ>?-hNUMy#Mnfz@^RwX_<&!V$CyR0Rnsd-y(SyubJL?vrS% ztIN*BlypI(Nkaqt_*2HllXij3-mNXCnU#L371ulORNBZANInY^=aFhZ)1cYsDo|s5 zG2wMY+1cNCS{cYYP9@GjSgUa1Y@V*fhLC{_^N%UQ6jaO&$U`Lo|Nqy11H}43p_Bz~ zso|VN+J7@3T(F}$m4<_4G)?7QnB5K>zN?!3F(Uh;sMmlz#|@xhc4YiQlDhm<>U{j% z_+O!uRx#~%d}zu|6wE`k*zA)Y5prq3__29q6qub+&$u=m^2z*svG zNaADQN-$_oQ5^&(RX7!z&|4(C$_v4D6RO%eF2M)}3TK7%M5(lf*YF4lVZBhYFz!u5 zGdA8my=%i;u`HI-*?vW{JMrP3I%a3D0?jsxAUxBYaOVM<(B^ap9A-N%ZC_Iy6#{`5 z8PyV)V{$Vb8WZB-WtOJH3-pakX$MuUvAHHhLl@RF)UgqP6^2K;a-4t#{}9JsL z4mnB#TyqBUBgYRin(lN>m{qS$jmteRp)qIrCseLbm=3@jO|)fje*8Dzx395{B9hck+`$gFDftNUpiDhGNrn!M5uj~a4+#;hPq%d zMQdhzg}X4WTgJFG1X!N5*;>i(3L2ggGQR@tj78%#%}a_N38`lHkmW zGd@oC6?2iRKLdg8GCHBw%wjZK@*;h9_!;Ab=#*G>Y{jejf?0c z9HsnDH9Eqg=uep($6_WG~R_*5-%}+l88Ke2@-<$W%!hnhNL73rOi_E4kue?zioY(qh^w z%(C+>W{NAY&JuM5igbA1+BFC5akRCvp@gQ&2uN&yJ&W3oKBvFo6-$(}nxoG(_}QVH zSV^k8CKXI!3N(0C1*M)^BuV3n;)jKt;)qAdvcg3;OE}U z^KF%nT)rqNl|@N8h*b0$C`8%+EVkG_E$_`YICm>USG!OltuWP2^c!xgh?Ui1jcbNp z?o;NXRARqZREUCd&Es}sam4L5exAYH$%0r*)wXxAO||P`7vdlAPcNOlvGtj&y=OoT zYF{+2kLLwh>|117&eQ5n4CyA=q{Pj5b6?uaEB4bheEZl+cFMm|tNt=2WoEjdrxa+v z18{J@nOVP1X1YXp{sWQl6V)pnV*0}vJD@36gyvN-y6)OsE>m~v%W<*(+=qv_i%ml3 z$_r2yQTlYT3@7Dlu``voW0RYFcMFHNF*hGy16pbSoXO8`=L@fnLYgg&eX3Fo*%DC( zo$wlg4L=rL_d@Ed*2R;S+vZrnF1vAVn={`sL-dk`qvyB!@tD(Oo>I@*E})yp5(uwohjR&{};|;zb*jn z{JTlEB4gpU-^oMj4*!>MpZ~?_X$P77FEmJ}+W@yyA&YF9h$Z%JCMf*X7oz&l6F-&y zM|1jjEBQ}IxqqO;x-RT?a{VHqXL;J-7M0j_K!H&E3dU=k2K z`K81w{~{o^dh=lRFHYv}zoBKJJdr2P|0H+!|8+UC>Jb$KDT^4NrPYSb-X#ElR9sRvxYm^?dt-sf7O0EHx0Y5QmJ&e#sX^y-dR4X5vXe7G{+7!(EpNi2OWy82X$AT*jX{%vOejIp?XO9Tnp1Dr)Yanyc7 z^INq}1Kcp9TjSOcp7%lk@VN|G05mdmZ<`V~15l9|`7z7rZcJe)fT9x`iewhXXt3SR zhlcjGhYs4{ybbwm4pbSG?2c{QW7H-~v3MF^5l-Od)Ba2Ti&dSd=rIjQBfH)Gg$E}* zLC1%M`IW8V3^h6pdI$Iu0!~oNE3|V57$UQIlq6qDr}+v@ zhD!Y__rWF+JL*=CrJ9bG9S7ho;(^GF>T8V)nL#g3h!;vB2mZM4``-`OFiT|}W-TQf z(JD46v&GHcag*OGG96xfeQ~JJB>wtht6|ul7pn|)0(CjStlL|F=%Df+u+W>jlgvke zC%C})X{Y}5_J76R?(v;~iAS#wNir(27K%L7R84$o;<5=K7_B|PR)(c?xbOw6>;&ho z*8URT;j4VPVL-i8l`VF~0$-KBB-?}q`mZ(aRQHrE-)}st!J?YSnHW5_I>ZYuYK)n%b~p$YF231|p!3#$GIWQdiVMZ?eE~ zCMmkXBV)Pxl`~E5b6%TeH=3@ujhvAJldL+a-35;so?mS(w%w>l4vo&Z&rwaLnXw=@ z%bV;0c~>IiocVtLYv+(e|GeT4BA0-`O!h6<(#>+6f1oNBTfkL!JUH6dc>cV|T;W6F zem*=Cr_E_uF{}+x+*!^obKf=(d!MU$Zdh%p;voUZdd|~V6Htqtw%;bWQI$n3e7;NL z0_~>pH_9-r^aGdVwKFCTRCNpjoPm$J#r?95fYOh(bs3=mOKO}8Jh1W}J{C^Jq&g8y z`E9_b@EBG$MGJdML3Hay1!Y(}pgk7*PX+N)Po@AmzFmRvD=FG` zSN>pw_B;P+Eo8hmPu$iDkWyP$%f_4WTuoO}yk#(4*rr_Jvr$~A5yj%FnMmfI5jpQk6#HLj({6?#i$nu&&UjNn5kQj^aeGy={;Ph5>udcBI^Yv{c)PS&! zYoA@;pR@=inftEkpDm9t*~r5;Z<_|nx8C`zZ=fF&I6XV@1f`AlL|b9WIIuXOOwSc^ zeOuKt`I^!9&l7t@(vvC_V{EOa@*rNhKWMYS*+Y8T?`zbGVJuOjXEt~RpRg)OsziLw zsYU@kCdHxJJY^z~GXt?}Av`{4*UUz^3_m~Yt6y8PUU5i`i0w0-hBY;R11feb{U7ad zeqGuf`5J07`DRr~3$pTMV(`xfb|q5qcF~>s&J2=Vda*V_y(j;4w|=wOS>JlS@yvb^ zV!ffl%OsS+!UuP&@ zZQ+=DDexFx5!)TCAg!ghFKhs)js=b95-Da@?1b!o^3q)d?!5Y9Td z#h2^Hgg-A8RyM~VTSKOlQ7_k$=w1vh-wMnJ`XuGKeo{}f?a+fxd%jY>Ew1*eWm zX4*Y}=w?umFi9q%K}lckW~0&**C=&f11lDY=_|K+=d%Fq41=wA@`D9JvBWUJ!(~NX zmgEoYWKE6}$hOTwrA&B7uiI13l_CNt(Ie#k$YE}>pMFuahsmB5&$ejL0pIs{1ReCB zDMtGTV1|UgfhcYk#A^n;{k%n<; z(hpl?PR-t9sjz|VS1Aeso6AmLNy1eulG(CncdX3N)(*EHSacn+DIfK8a&6a3JM>Zx z7mJxOl_Cbr?060nP~i`Imd_BZpevTt$tNI1^(M<qiC&U$?3BPEfPel5&;63|XJk6_>&d77Y7 zM^_eR$KyTI==zl*?8_k#L+dLE9N8g@f=;!wZKj@5kP2d~B>dg|Kloipw%t@MT=}#D zkQsh%aQkSS1I}V=_l45F{&9{p0=Nmm85d291eO@nM3+$^fRu2kp&1dN1FDW|ls;cE z=Ln3>0Lm>yEuX5+sg%=4TcPUAY^j_pHFgigSjM3Ot7CK zHJ3JEXF%}Lv=)R{9;$1Y$Z#GXM`m?52V#&A$q~WV!!RC-n&db;-CG2NQAcw|*gxUl zNJ%rd9+#A50C^|w)4#JUo!*0 zKCbIcS&tlTpDYwaQxLQvaD*TSvN|i)X?YeFDUKG|CYovTMV<{)~U@gxd zx=v6b{%n$*z%a?^4yb1u+xhZ zDUQT9Zm8%aYHd?D6o1kvg|G%BDuN%sH90?3RRX8y-d9aj*$my+bzPfE;FZ-V&~cP~ z_i)hUk{xKJoGM$ADXZ|g-0PNFbSnT_lXDqLQ5=0d9@qIjlER@hbQL%ITr+pAO=b3-+DxV>|}4-mwGKO+{f%!qx#=Nl4Y~- zbZ^o*e!UHyhs9daL#43dGDWxzFLK6SDXMJj=yU+C40@}^~EkC>^V>xR}coFtAVDmK=Jm8OMIR66p za9i9=9UAMq(W5-mb6+wWyaxdLHG|XhNz#j_1YU2 zrxr>&=}n3yF_I#4SkM^A`^e53A!SBQ5V1^2W{vsxkFl1Q2uPXoy_IBGF>?yWfthOe zkY5MNxT30DN2Z*jK&F6b5Q8ofzlA~h z(=9hjhSPC8Ogv1>){!S}XuRtHe+~E)zrW*Cn8J~}VxIBKamOIY1g(l%N1BT)%UhMl zn*L>l!#XvSV*(fzha-nGtU+7ve8Aew;K{Zcn|Hx{*q*$i#HyvAMb3-1fE-K9CYgGi z$4iO3I8#x7!=ig*phRKxiPTm)(7joj^YA9| zeM|TE9`-Yp3v-}Maw`y%@$a1glZ1$w;^qX;{zR{CrimTA_+M@d_=uIPf-9aoH`p)w zgMXsU|CbY>M_@g~yE#GDUsf??a(dTT*lB7-@W^5Kx=!}n^1#%t+~G^PS*pk9vx`jh z=Zj4-qq35heZ_~wfD9JRXs9P-Ts^&|Mfj;5)Gll`8XA*%nFVUM5bsvE0QABe8|NK> zNDP=6k^(JedXe_VzF+L1SLj`Ze)U%c*Q$e`^PTu@ZrhWNF#DK`K{*;0df$*sn(KL* zK9Nmmh5ywiAV0qc;9$)FesmoEvhPM@H~&MEQo{F8RJ~&^-UQrs{PjD307!Tt9eo1} z)VMHG+NNzXSARJvuP4ZBJ8AVfE!K^p@wvESe~)h>a+>sfw(4xanNlK~bk}6L;nfPWd~_Xb&RNlc-LKFA+*W{(o#ae#5g{ zS&q-(*H%#B;+(%;m0$1(zv#n3f!m3lK=PL}Ws-27>&Cun=f}o=$zMK&Ebi8ad!7F4 zN+vKgAg(`qjSU2mO~9L1fR_(OIzk-*p6C5&Cdmf(Gki_g%hHI9rn9cM)?XYX_H8iY z56Q31XKxRGxw}9e)vFvi4rv1>7dUS%DLc_f4m*f@l?ZE38E-g$MQmRrX`p05vK0g} z#=kblKBmi`UXEwBz0PcSv*!6!{zxcxsM_%3!RY3`yY8f6QwHbtjOy6S)6VS!XHU)QAn~isoO_lJvOIdn|OEgWx1zNIdA$bAAthY=8PE zZ3jbi&M|JtC9P0m$s^k*@aw1QbA#ih26h!S-JX3(2VwCRMh+OokFG1G9IRjG3#+A8 zhCU<;s2cR6^j)9TiK_F_3JzwN=7R*%yG0OS#7Fwq(WMWlMfO=~6u@7>ASi@FzP8AC z1xKi*lTaT}uO);nu5oJQSy-C+yU-Y@f#RpT`$vZKCW*!y^Y2g91{d>Q~C!-=|+<)H8o9 z48kg+HW4Yy9lm@KO$jCjeT(k#Aj&}2H;Dbmo_T#Xx0ol5RiJupVKHSCoqz}Lsu}^ zh``RY53WD`R?3?HBYwFC9%yWQTuBj2feX&-VC#)NtD|2RT(!Y?DP;&mjzXi$URPUN zjivTFz&RDqxnV==A0BsxXn`0;tp##)rr;l2uC{>YH~qns1eG)Y4b4mna(aZN_if5m z2n}MTt;Hh!nEFuCZklO(vDsu>(LthdFa#y3*P?rUell$v(Pmj4-YYaJ;xr8m1A~ti z8vbDE?%r0rO1W73<)qEU#%!C2e7)rVW>JY=7kNm3y%2KSGT&E|pxM<+(ArM_S7y1# zXREqLVl5}F3=L2?n*c}M$cx%&hHm(#zZ5YrrU8ywzPJ$;J{X{v)x7q-MIU!C8mcQ+ zx9eVk(n?iQ`U0eg12uMW0_iE7oE)nOs*vU~6T9=0ZtlAS_TQr?f_ZwB!n`Os%(N4c z7j@LpUlNZlVoUktA92Dy1=p!~xvZ{>dJ5Q{y~dpmAL=Nif|z~NN(TmOtT#<&cx0lS zZA24j<5^QjIUyxow0!@7tqbH|3HRE(AOS`cb|_&#B)sux|1fcji@8CKgC|2DcczFA zI?s_HenZ~ny+mY0<|L&v*NqxiQu^2juMXBnqg-=W^D10jEWUxv5wm_mZNO+3oZ_~& z>&>?7UMmUBQe;#!nl)k1Qk(UTtoPD4qUD{oV~R)fTbQIqAMOvkgO0$e`Ob%H>UK$` zD!#+j^t)~EVZzTxFKLcvFW1W#dJoe4&FEIbqCb5Ke+L}zlW9ssb8;J5EgN$S%vil_ z5aQ*Wf^M2$6M8~npK-|?z?D4pJa;0|s&TgE14uC9J_KW6ZR5rb?Awg9t;<}r2B97u z#AfKm#EGqDoTPc%&z?+Q_*Kl-BT{?flQW%nzI*R`XY|I&XfUUiiCyba#bTQ_HNQ!> z9_=_EZ`VnZW1E$EpJLHC;78eD0I$ZyE7F&#;AOxVebQ7KCYuX^2O6nY(7{O5CZ2(DLIQ@6%-w5Ot2?T)IV z=^kuXlk4mO6&4^6xqY`&EsoyjM|&W*Az1-d;Yl)|RmV5($VxH;HXYC)9T0c#cfTTP z9?;scso@V;p%VA1(Jmf%{_GSOIz{&A5&N2a?Uruax%Mu_(3=EPAWX_od^K5Pt$yd1 z{HROzXh&OFg341oJUrih1)rQoeS)Ve6IJGgFS@LntisU?RWl|0F3;Zw%}iTW+h~?8 zpLSA~+VS(j@ZcrrYUfjZeQygfe!huu6vd96exU5%#i zkt!m^eI7+jk=`?qX|{5&IxOkIKw!P`No^MS1xsKd6KLlG`;KAPH4>uf2lX~5)l#(u^93RI)e(gqFys4MtTHUu z*N|zF0!5f5M`$v$E|azPr|pV@djqTmY{^Bv>02Fz?F+xQoei65?3ghCinHf-`CZiy=5x0HO)C#V zE#eXm*`ZMQ0YugLd&V|E*>jxrY;3!3&VI;BOCT8-@ki;l1Poe96ZfvAv?hM4a?tvq z9+}s_vvPGZ*YDM4AI3@eww)s7eS(U}DZp6>OY%R=rFm==DkB!T$$Ynq zXb=trVGaPWl=1i;ZKB3U8qAU(^j@~*FlJk2RCCLymGIpx3DKyoBNHv( zyJuxT;2F|aE`luFQQAn4I1Y;<->VN!ZhCYlYXC7eU$x{9L zx&+l>F_x#!W;IL(En{pPG|cpBp}y{)8tWOn8BA{WJr3)(MU)ZhvTNw^rbE$o=;#@&x6PR6 z^3AIvh-DZzf)4D&E7w%)gi3~s`*S1|r>g$+2>9i)ilc1F*<;H0kOVO8-v^m1y02s@ z%CSQOBL3`$e4Jt~PsND^`*jo2sd{s`JzrM~9^Z2e3uRjH2$()p ziy)Xw_Q38^(FGYU0}^`Em5n0vBJX>{5sY}9&cn+fbS!^U+i|G=TUioItWNhiU%?mt z8Irmv=?^#&z>FvHc{u{r6n>V&FZY(4nKC@+F4tsQ_o5t5qm#fT8A6UZAMtLtogp2X zam&!NnH5ksi#(h(=;`;0(`i>96h_Lm|Q`QM%UOy?WBNKP{wg7REaalT%5w9$gwaHkWB- z$Kh2&z!*aQUm`LM?VFXK^(zP9d-ZG^k`&0A^2_;U@5z1K*1o?8-_pJA*9P&drB$0~ zoNyHsi?Pg)H80OntJ)AVcQTPe*xnW|zC=A~zf(U3_Mf1K&iD_D%&Vb#@lDyOesF$& zF?2g^DcjLc#^0{5vV7Z9texu@cXh{KJ8LdLWVR9zjn+l1kP$r`QMoZa!0w8>*G7&- zB`|aaPMO+hjA!};SfOz!eUIv4jD;2IB%J)+%22o!>NA_|Ay4r#K1*-TFQ4^+JQb#&hmHbC z$*P&}0mTV+?gOdr%(*vd?|mqegHY+>Eg^~M#fd5u&Qo@EAvhn|BE3&`@D`mrcWB?G z^X{9K6Gf!)n1{w=YnKJlan_7NQ!2`I(Vwc-NfO z=2IZ2E)ydd`}wMwDf+3fTCEkCJ?P=RL8t^S$9d7g7e4q9fo5hsomH+82BLfXwZ~6 z)bIMPsx+%~Ue*tk8^CP<`i-jikrtV1{UXA0v>-IRqt=l(c+!3^@L)t_j$Y6Vt?eG{ zr)9zyIZhUd_+FVij>!GUA_$4e?UzamRUaxfrsuacj{Q|{D(NmIRD&dSM+_P*;guOF|9femk z{~D>G^NV$KL@lJ>YT+zO=Spx#o@rrjk~BX0xfgz`O?TVto1loz9fQKbTj~nn;gd(#{`^X;@POaP(?YDn6R2e&bBJ3WtdDOX@Qv`DTJ7K=C1na6ww~WnWmRRmtt9W@ZFRL<6qKc)u&(Npucq^8j zXN^-54_Ef2pIjrd|hwzp@c#&wx z+Jn@0ZPtX#jCoe%eoN4gN~)FVNVbct2S#$Ky?8!wXzX*oUI`hb$=|4Yy zOHkf3rq00`CtISk_K@5SC$`5xyTFFTU8WpQuNx&rspR zSI#Uql+4iLbU;Z6lz1$?dW&tD4(hz)NyCP@;S-bd-iI|~S13AcKC`K2N+y7b`La<9 zdSjS8W_PMnC{iLALR8(Q@pdM=kH6JWFdiFC;gQGkOkujqU(Sr;U1T)W_CifoTd?lm zGs8esh;N$hH6T`wABq z@m_3%tVfP)0m9z}800HM)_F#_?W70GqB&$5rxJKOD22(8xbAow-^1j_ zA`!M1)SGa&R2?xyubi%R?mM9SgqO$lVX_*Abd8NRRdir>9+=gT*_}>K794xlK?ikI z4CURYO_(gW}tF@#|U8x1trr6f@=G zel4wtK$x-G8ZEeecHv13)i=|?k&n1W19mduOH!J?NYav z`x7&#g3AMyf9;d1^yCIiH@u#3Ew@G#ywq}@qvtIP#ZHQcFJwc1ba#vzm-kE4NqfD# z$6T=`w4DMY#lC&-?otdti#zvaU(jdn>M;GHX&X-;WMY;1LlexQPtLSIR~PF-b1d>d zM@Lm79}?g$A0Kv=b^5{P2drfX1)%byi*bhPg7lw5tQbV<(;q)pHhVrN+lDV~rCtla zPW`Oq!7Cj`(39qEoLiQb2G?XM$h+13ygt@!6LWAyt;yi3DxnD8?ao1<`gUK%H8RUy zn{-b;X-DcMAkMt_xaqbBW_Z2Hm#>z-Q zwtl=n^6Vx~e2o=2V(wyd@$Sw7uchzxdxd+JQ5}*PDw$LB)^>F(nqzC9sIAd}5wTgA zCFdQL*eS#oedIk7OE6W$6@{%HlnWQ>|i~a2~kTA=(iE1 zZjx)+SqZSyX@T^oh8eigNx6(2`%Tj|Vx;MNjyp85p@wzQ_rObCP>GPBl>$AEF;*OH z4jmpm+n8ywo~8)=d-qgpQw4&%AT6dcn5JHV-Yk=^*G|{RjIi*%1LG|NpD6)ba`ekj zlKY<0%unFGj2W}6`_&x9{&{xUzPNL)<1aI^t8d)wP{xN@fOLESv1RKT0dijAvnSjp z&MQ;7l14(wl2+TB@*zM5a)o0a7-UO}7=dmVHel?X2KN&i78kIj5|K#M(hb{u`mz3y zDZYjMfqC&7ms>!Nwzz^!oOw=0;*;EbmhTQ-^ePBj3%z&vjlMpv-Q=bGBPveL1<@|n z_sB^rdVj`@QjssVo2ld%w!V4}goFa!>=s9~o9RlKz{-;`LOrp{UyMsJ^t=dXhfV|_ z*1g{h3sAT2n+W+)h4nNSwtsu*ybpS{g>Uzid)^zoIV5>!^CdTED`Vx@`< zd+<)Blgfr)AMo`~aDQAl^WEqr>EM+Kpx_AG6&)*)qi#DMApYK;sIubP@}9drEsxl- zFq9i^Qy)#Ise!teP8hoZ4Vh<5UF?qO2T=+>)%`qB_DxL|b?;-2Kx(yBznkbz5m|}B z!D>H&t(?r!B?f{m)kjcDbn$DHvuJryC^;fn|54(y)o$~_3qf!opH-hxqYFzUKL>2W zvIrt;sIZ`x3@7mdhZZ-OPV#{f4^^Gc1RxsBR_;0|=%khbamMW}Ws2JBs5j}m>RRb0 zs zLgLyk9UT{=aDu0OhKKgY(SB{$oK5^>Z16iJ%yLg}+*uFJ&CHxA;DIAQW7g}SGf*!Z zf8_8qb&b;Fwu#$}ypDX9@c8l@qRxl>?BcrPio}FEcWRG9ziWhE{6_@jQ}p@;(TB71 zm=4>9^rxV8gQU}6L}xYVl$0A|t59I#Db`f${mZ4=&BqstdbyXS+dN;a!no-i5nsC| zctX+KDo%eavLT9pDfYkAowgZi`paN)u@3j3mUMmp4F!}Od@7cMvKzN_>3kP4&F!os zA-AMCmfiQ-H3Y$EI|_5ka)8GJ9r5*^#vcsWA91Y6HY2CF8Mtw5p z?;s%;U87bXVR8g>D$|QEeQKJ8FOJ?1=#}NDjdl!kLtE^?2Lsts+3EWT#tU^X+(F~c z7M2-Wr4=t~bcoV9Fc3PJXP)tsF)L0mf^nW(Ko=G1_I2q4aWkQzKF&lc#{58QNaq7a zUDRQe8pp3xht~Bw*ks9)z^q}@yfUOAym*43KM)v@3^p`mDd;<4E-8vC_F%Ut4Cyhs z-5_>u{M88Y2F)9CG>_TTzaZ21;1Q$!aUhHw@4*)2HK|5zAA3E!6=^8A+Wb@v0XC_A zz)6r%3OUDjfmTycxm<*EnimW_b>?&77|Q3-<*3%apiA@UF|Yyr+k;+_DH}Uu6YH_N znl@E*_u#1G1Yu5;MzJ~Vch3%*D~=*utTBN@iXNSPkq=ErZl#^wg6Sq|Iz$~Vd+!mL zWenCpMk?0oe9l$Q>1leW=i$B5m6HsCPbY8Od$?o5{GGOy>>?n9CiimRMu^pF znLlok2)~)BL&Gl{oC znm_pRi{M4AWQgv#&CAX5Y&Ds`5D*6J!%e8=_no>}6~s!B>QO1$&A+&1{|M(D%s*_) z6OG8G7QYsVK~$qay2_vvqBmf^sXLjYCoE>occ3`4)e)g!4v$FD$)A@E`W}q*EU>3Z z3@VMaUL0uEKdLQGy&I810YK#CxMX>;!PaR{s&t&*gH&UI0aXbc?Qj*8SAV*wJ1~qp zOypsco&jh?Ii|mnKj*#|YhZRZn-bbbbr3+nA{U@Pd|=_LtAwXrrmhQ;(v=gTv{SXl zD=>LXxM{g|uctL%y<1phE${d_HUk{!cJ?~5sTgq^iYy722u*#d!5!zdCy7@DNal&G z2GP}!uP0*OI5!QYiBD^fu%^^_jaI!MY7c(Z5+lJ$KoV61`hm?YCrCRFbxIl|~sTn47wmi#=~& z01`s)7yhr(;YWnedq~{U>+)dhBskF#tLp{;cOP08T&ePcQ=rg$}`kHemsiRFk?`72p-)aUtpdnL1bwui$Ls$1Nn zg7IBp2^1}M4NXrC%sqCYuw+~i#6qxB9~?jF`Lpx9Pdi9+vZEE-QX9;-h2K9_=?8xc zN9!rCYqlc2Q^1D9hYsE$1?U^%1rp$|{GM*t;*NEms!@uv6Hk6JZ96@yG^C#$JMp05 z1>8fRE=Awd9+`w}`~i=3jm_Ha?TVFtmU`g?<~un%z@_os&=l!y3$c66Wh58-Ex_NH_rKBC+^s`$x5-S-ucW6x?bOl*IW8V~;AgHf5h~U#u>b>hK*WmxLN*Z^ z_dBM34=PxwQp>|UBbdgq1go+*^%))m$C|5POR z?VGD}YRi=iiF!F~U5-Q(RTU7W>3iD#jAbg%{_8`aYQ<-yR{2fwE!CBPeeSAhX+6;eu$~lF)3|@ZS!J@gtOVja0BhRNyt4U?Bt#PZ0eHpZ zcb7#r)pcaX%uTWq?A69;kqjlt0E$m++OnuTT2c9h#dnPek^I-i*({N%#pv2>L*yzdY?`i1 zIeLBqlenau()YP(SgrvYVHto!t@I{`hnaG{c?&m&%(aR~b{`RA9$cOsY(1iAL^L*m z0n;*g9;3T8Kpjnl!`R{SZQ!M&R8?ZF_|b?acbmr^oiBE(;zS}6XbTfnPT{w{A4Y)U zzwi8AGYe8XI8u3a1r%$6GU51B_lpz{a#4qIWU4 zp!g19@{rji zK94R`Uya<&43;E*j>1aq3X3t9QHp$Ww7;2;_Yi>b-5u`vPbRW_vLjHkFnsGN_qo^d zQ}tpEpAY{)$9o>jZ9={rn6T>scA-^Zn9ho*zIQ5+p{@iWx0OigoD)_2py$>OgdK$f zK>Q$fVTySsjC|m$E>~}+SCZbsX%jCT$K`wUgYllJI@h84pZn8S_Q2M(QDrawd`L9W zvu8680eJNv6;a9cH`9oE)r@LAlTxuSd;&|2Pn=o4JXF@du&J4cO2}L)I9bXhnHW|; zvow^PbSV_C4lX}WU$!zn_Ov(o*&wFebDBn}>7yL6_f6cZ+&fz%c-@hY@`djG{$Ofh zL7Jw+j+J-`)C)1524oQhN&pIoIpfIFCxhfL0^WK)=<&&Y7k%k{1 zYMj>4uesE1o6dlWBRCXWlNBEwS&N55=FSj$W=hCEskiZPDY@P)Ra`sd^NI z(X82kmPCK70bL=w{*trBHAlW++eMm0qm{k$y}Tkja*b@=!JdQKvpeTXXC@r6()DWn z-ey+C;5=S!K>mK8OeY%x?^KbtuaI@3XrtV>R~uBEelXqCk4-31rY?*65+vyv6vlY_ z!RFQTb)@gAPqE<+7Bm&B_&Nb|72AeU_r5HT|3FZZ#f)%Ms66qzKVZ-RENP5$Cmp^G zb1O+j)T5(fKY`zV^Xs3eKsuFwkKVy3ezNB`iBDAeZ|>qTipcMutuj;6_;LdIqC%tN zv3;t8>ArB&lUHL8dFuMwTFX>VmpnhAt}?i@-QAxiqWnd8WGl5#4DBcVKpJ36e%efKyBoPfL`pvDqigiF$w$J&=X z9i<=!$A@z0Rnpcit3T$yC@X1lAFVGESz3E_h#Vf1{yz$0?vpnw#peM0p{r z>EaxV4({%+Bf1S#7Qw!@O1tJ{PkUW2=B0fD>eGJ_gR#z? z^`s7_w}%gX5_s!NP)7RdFf@^7i@C!uyi8V>22ILKhiW56S9iVMqnPt9W1HaC@<;Ch zj<{-(%V!H#-p`2Q^d=q~UqSJwlkX(6^f+OJB+8^X*`Y7i<4$buZtl z*yeJ1BROY4{A?`J=vdK=|E!-BoGKfRA(DBVchnDMxLcFFsa{si7rTc4b`p9W} zH*SspR#Ox@P2xXFB@Tn^KTd0bXp?jkW){;Laf6ZUVavU&sa6uFU#ygTR7-UP419#{ zep@_mNVa-8Wf$e;%wVu|yP#=Cy(1;8N%~1*BdlTjG(WS=mmAS9wdKN5hqf)lr~%Sf zO%#8S>343elO^uH)3_>A8@u&Qp--)!PT4Vm>ja}l190CycWC@C_Xa2mGJtmDsp4T< z0^Q;$t+Ll)h@cXBrq8c|@78J|mpmVgY2FCB)no-b9T42RC(X7-NAGmH7BtT5xo^SM zv0qEO{{($Rz;#1QXAF;?!+&I?C}ug;;0?YCA?x-V_C=;p{P%;tF*f;J>Cl{jfQg*Kl}X(At{=P;N?<&w$10;UO7$ZlFur97XQjC=c% zlWilUcYI|_*kzvH_V)8;$~v+&Pvv|-XsMAVOH^o6|037j@6L~(5z`N95&g%^d~L=b zhCC*5WUR-F)2anl*<0(*3-KPH_h|0)vLO-%I5UrfgPyj)kwuw_tyQ2N49jI z6Op~-;dfnXQDxkg9E9dwF)z<2G;*s&`1^)!?X!OK2?!3{0eC4CxoPP! z-PBzsijm_rQ)Dw_1L5B(~pUrSRldpd8B({=~#um+bA_&t?%dO5s&hp zHDoF8-yd_y!1+hvMhfe24r*yr`QP{y8$|lYkK6ME%T=KXt_%C!^yr_d-X4wVWqx2f z=v;1=W!K}Cx=WlKC$M5EV@^DN%se6tSzUvyz7eIQvPj;HMPZ0ZR9$GeqeJhH{*8sornM| zmk+nhu3)!UdoJgPCO&(C75{Nkz{k9T5)D8tN9j4i&JUT+wBf%`%koDoT^e19nA^ng zdKnuFw31l8@~>$dZJ;sm`UIqU+LG?*Z-AlT4}hUUq=VJJ;=!7$j4Z;kA~W!6@Y?tn^-;IF z8rL6r=)e9gDt5`Wt!MFj(yljchD_)y=V4w!MuWo2;N4D zeRzISM4DWR{0s5;>qCpo@6&z!pD*hJ-SDQI+O`8anr}?Xn};KJ{^hRzcyxh`4{lyY zF8%-f)HE!-uCPk3RA{qy)gMF$P5xEM`s>4h1$xRT)XOCg620)Gt>lIQ704s=2hnMi zNaxHuJ>X%ov1E{q{{yTw?~&l#EYP28`X8|f@GF0EfTchwH5E&$xuTAm)lDM*>fs+{ zu|FU3|LJc2C)3LNc6d&tb&>k?>0B=3BTV?Sf3eKp0g!ao_uqEECXafg2@4WoyP>+~ z%Kz=V$rbuuQS$q4l!@fUWeaOfT!Xc*jhSd=3dUoyCg(8F1GnTzoLdar+dP+`o{ zN0!LRAyz)c5s&EKygV|Qe_JU3r@QG=@YCkFxf}EMf8Zb`^w9ki5i}Ow7C(&9YevF> zlOk-nXLdI>|L@0ZUNoeQj=0EGHpG6m`K)EEqQ(0pL&UWtbw3ck3N-xAK($D+uHr#o7J!tVkMeN)Xnc2?z%a)-1AZL*X6;^ScC0&`G`|{wXO+Q+ojXS zdkd4_*Bt&gui3W%Qkyd02Bc5H*bkJa(*zxiOCj$`f4>oVxxZ(J={z|QUiDur2JIXJfJOB41zcPSmg|C(DvE$P?%_Rb&Mbc*WV{En(z2n?%?B{iN=<^9Sp0xA)H z+c9>ND!0qiy;E19AA73W4j@@CjE>(R;IXDZfHzF*yTA0E{n6ogFtKn-YHQx;iwY_lw0H@~cvKzQC*;N4ES7;se{)ep zQh}XEOSB7^S`UYm=EZ)UA8n$DhvRZikg0-(ec9QMmArNuRy$xMv(;8H`F)>wI?cA| zovc13zoQ~m&3#Y8X|yIBE6Q;MG}BQl+`=1Ku-?vB`q=S=753yTFjVh`3;LSk-yG)u zelz_V4tvM-B;ym~ulf14>MJ2(gD1Q7cI8G5PAC`{AQByX3k*c-iDb){S-4p5;s~HD zU;qTny3DrBiy#19v*SopOk~b;l`3Tqpg4KR00sMK=-{VHI$bfAjz_Uf0Ak%2Y2m9b z9&@uJ&<$Yx%}02c4D;NY;JGBZ3YnrGW(iN+VOO}tGGKX%+8SB}MAN8WT+oZkIz1N2 z?Xn1n>unWA6WH(VmV!C2>5vn?$fr`Du`e3dc$I$FY&X*fyGqrYx zpdn4--!6EC@bB%Nh5ZRgwd7{?t4mHtIEhdVgzu~2?n11sjla>Kel_D`G4#`(rR00AsXnuEId zxA)@z@Hf}FxgzR|3Ga0FK78DmZ2&_*Y)r%VK`+oe@8D+8Bg^1|(A#jN8UBs$p581N2YDu?70mce?OD7AjOuM~xUtX@#BgY$*gt}r^V z(y?Q5Z@^4#?4Jx94P31h?7v_7Uk?1n04gv_W=Gwa{Nt?znxwL9IDl;krfhp?Q1hAy zCl@=SnM<#-o#}EftHvX^W2+~k5||T?YWA#_$q{QLGVBN4C*ep1Y9zspaEQzoy$yg* zY-OoWF>dy{xl513XBt>P3{|f=6rioPG|n?GT4H}k^-mxCwG)S)T^x=3 zzO~-(?`JKSfvn-2efHVEy{~=kYsBKodMw(hSpZQg$RsXj<|i(@HEDT8|=3*=^?6}JB-nAFsy4dM7YwC zZ3ov@3Z_lKDfw7#9{A=2w`Cv-+SmZC>UfYSPm;3kYFiviz4v=JiCkFrS zcl-Mm8eBwj&liZ5K{!;%4IHxGIG1u^roiX45{b|QH$d-Ak8B&v;=DfkoJJvnnCx&LIEAiyq&ctw!o`ZE~NUJtwfev@Q{!@>`B-+STZTXBN|&60{`i9{1RnT9)s zMEq~&J`g=+cUY8H+(5&k9?E4%* zmL%8~IoxtUbNiP8{O5{j)28s828`eVs&F z(EWY-{mx+iM`roQKYfgWpUk`%1>V3m8_+aoiSIH;YV{CqgtTLn0y&}c<32Qi#nBVP zlV=Q&H)8Ml7GlAQ234rdqM*=-mCk-QN4ENXTwNzf=%abuRRI0_{-B|S9;?b_$2dOo z7xj}n_IH?GC(bASpXu$tUQ@34by2l)5CBI-SbCiE?yNQ6#iQFCMB#Ks;4~{8Zkjqy zT$~-v$3MO=|En<``3PJ`lNf+Dw%B!2 zK>I5+PX8}mT)hM0y>@Rk`Px|jz1@G9_fO0J;dK9fx?cZv=6}4w|9y8qiv2fK{k!w( zzmd83?}JG9XXE*7KtNdXM7g)c+U7Tw+QSva|D3je_C=3(GT9Af;G>(U>SUtbpT>F-hM#e23>iMQOb_4>GJEq){R|2E;Hq5CTF zWZ=y*4172NJuZYGW%}@2vY;UIZbO#qe8A_D?)L|FKkJITJp2%E3eKzfPq$Ci%Y!E0 zimKByIhug(7DAFA_q{dyH_1i`7a^>Pj_K9g4PF@IKcV}7c=)?_EOO&ol}NfCtKkG3 zu%<=_#9{G#2>1G%mhFC*L2Dr`wTIX{(D)~y16`f}_^4~-zfOP>J{NE`#uY<>%68zO zCd>?2g;zwK>wJAXei_BIe-vZ=^8+eG0a-O4pQGXiR{femrkbx2)Y64WV00bh|Ob?A4uA2TX?V zgL6Oge|=Ne*954=T_zNigju|&R_LY>^*Zkz8u`eN$yo<13K)ir)H!WU*>Kp^A80)P zh7;90_9L9aE<61Q5JPtYKxiUXy^kSpGo>St%hc)l&jr0NJ)#H8!U1VzF53WX3frLy zeuqT~Xs{?1YbB>TeG1`=13-7eF(kL}-rGy7kS5rVovg4|=S{X>Y;iE!C8UX2oVSiwj zOj-tfHY(ncO@?An&E^hG8>2?6Mpt19&)XPz{4EQ*c$uk|`5!&#tdbjdGYXC#-JfQak#yFXq0QLVzPIc76 z7+8=d0QiCkz6?40aXOV(jS?VcFhlsVo8@EvixHd_=q<*j#xDIz-kbP*O+&;bZWE>1 zo@eU*aMQwF zuQ%j!SrOMVZN|YHE$|)QD`$Iy%QyJ9l9F`qRyd^F*PMFpSa`lsEBX7;@1y(p z(s!>tP_N&Y z5pMh0?|urxY2A6Zbt>YfA$LrW=bk@6_ zX_k|yYUfX?Bi*x!hUf%;gC#y*v|rA?DrrhQQ2$l-r&53Q;32K&*Ok^ zL5CB@p*kopbx>c9{QmR37hw-1sfc{<+`a3$=nWuA#eQa7fO=_i5>z(Y|Z|G}q{pC-I?@lmILdTb3g;Xw^BkQ++Mo389{n{vXPMhuY!@KYLzD#U*J3LIgb1M7tAeIyRkfJJ?~twR9FbfTg`5gK zZ2qj&5TOEE>#_kAt!pWP`XXOAhO^d^H@oG?Un1IaS|~r9K!Tec9=|;jHjIir)~Hp1 zKPf;3R#ay?sb;%4pA}C$l}@}->V9O)6xIvju&G;eJMUwQZbpkiPz&2r@4ruhSHPTO z!>V`-_`imY@@T#Q7Jv{#oKA=(`{%`D*mb4}c%?E{SD%W9WBT>M>-lQ@;m}geypheD zyrd^kDnT967d){lNooeo3d$N>Uy_(Z)*<&_2k9tGiRTlxYm-1Uw9rZ9{xBnsUI*l4 zI^n<}vCwt<*s?55U(k7{^E~LIpK(j0HJe{(v)1BvYfter7_j=H9bk?%=Y22W=jOAt z#5QNkDGh`jG!nu8rEIu({;r+}sf(wL7w98{>m_(Gi|sulo&1NR-q0j1=GSvAo{|79 z1F!ed@rCELw{}lJ20FrSn?;F8?VUF`X4vR02gB2?b~2uIb)A;->FmBx+H;B9ei0g| zMl>(a4h*{0NsZLuGQbA?#8d66EIGB0!{{k5u*RjTu1%;$#7bc8sICG^Gqv&e7*tz; z$i1(iuZHsj?b`6N8@ehwUb|Wktg>5rXP$9UpgaatT(^K`Bo0@6f#_VFWnO12ONNsn zBz8k7qoNOL@!C{8!Jq>oAwxl`@Gn;5HCLW6in@tY;yTFWHM^#dPKD+nF}%={%y5*M z#1bl?sP)4~W7hfnG9kF4#-6uPBL6Su5#Yvhua+7vp1}&=d_%|K+AeJTwup8p*F$%B zzi|_V-k2g!9KxrZ6X&Iv9oa)+6(850kq1M&Et0XOwHLXerqXT; zQj16!5lQ81TC*ssPYVQ=%^%%>)lxN%G(zxVsJ~!>PJQu*F@a);NO45Ulmq}R1cqt}5U}wDaavQ~QK>Wt4s`WiHj9IflaRzk z#XwlwNl#cg(Y}HmY+f{H8VlGJ^W)!TriF-4C=7%$B@YzaLM!AtXJvFL#F8CWimT?U-xu2#Rp_;RM zUl5ZMm<^OaI9+B zn3DW@|IsT@h-NO-%?hvki6+SA`B<@MpxR%vxYzWl8*nK`q z1yIl`5J{ZjUc6)GJt*J?-MV@C8>5jgABrh8)kh40sV(mtbvx92AYyiTxmX-$h_WjpAi>Ez>I zZDdOKnvlv8hPTk-EjE7|tbnhaI#wBFWl+wvmQr_Qvw3z6C)kyHp+sOtT%!+TUdn2F zhu+z08%RM$E+AIK@GbcMxP2*y>)R)Nj*_?|;Ib;zL{)(II@pLUg};vQT+N7M_Q{N2 zVf*1T`Cy(sXsNr50z?(WUs-!7tn&9*AXTpx8qmO27r+CkCw*!f*9bvA?#F1PNQ%Qp zU_q}Sm6}a_G%|n5BWh0$#rz0OOWkpGFfaSr)3Vv3Lc2n5_fZE!FG89zaE(4i5& zNSTtPr1~mtibONx>X^PjwC!uCg3i{TqbeF_82=Jrqy+A9mr3Cv%+Xd^9k|EOSZIK( zM(#9!c9ka9mekqGQ(Xp_P10c~3y`?--{=b^kW~(Y1Dj)GUh1>T)_hp%pp*jHO+oF6 zFH<6##Xxac6EA~2LExqI$G}pF;ISEsyLavsCNDdaqaw(wad=7ZFfzcPOqQV%gbzX= z{jeE`d~uU#kx(1LqeCCgWMP^V4z0J!b^%Tcem*{nJHYmkFg8l*(70rfi-kQSRmPd? zoa6)ww9a>%gEbYLlX*=S`(1`d%c0i0A3SKR)JHAM${0)-FVg~LXO~e_&czO@&JF&+ z{df^V9`n*VCK|b^C#)9yXB{F|a*cW{b*q^HJ+ReBKy$R#kTgcOg;kAYDs)U!fR7PtO?LGH-k6D$1T< z1l=F>R*M$aq9TsC}@sC@^{RBEA2X$QuF1P;j)Y{ z0FF3t)4ly#u1pHf`8qzp(NlYIBvPw*I0`UZZYte92xYxRYWrgi4awB7{Uu+&ci{+Z291_5_N;bmrlLUCUf0y5cR~74OR+-RUc5KmO zoG<&VXxeB=#BGUnRcf4V_7$U1#i*nV0+XUhI4W<=cl|G`9`WaXb!D5Zfl6tcLQrP{l z&}@;+HSFR^M48J|_Y>Q2K=i|25ZE7g8J^wnyg#&V=Labuh2P9qlUZVJoyYeS9z&lE zSO)0fR=%$$+o99GG25Rk&LFw>dOzdw{%vP99X}E4VWsBRUi)(Z7*Gmd35HWOvFO*l zZ3!X(h$}mHBz&$*2*Gcdy8DJy7h+MeM9a~Eol}>bBCx@q^f#OqyOTR{T003i`Yd%f zUI&wCocG-xXq)fnvF@VsXap2W`0Q1Z@iHh|AI`HHMmkK@xxW3>)LbnA;cWnRG}!sy z(*E-5<}UAi9z6pk8WznTu;((f8A1^8EDhdH8$q-hV?gyu?j1qbCmzxBZSmq*O~qIy zfvUjx>XTXu%WGRF7X(=S)kj?>Km!usiG+OMhyhyaX>iLU>&ZsNb%oH`>w{QeN&W;5 zP8njJ4z^w7HpWW2Q8Gv~;yk7b%Ih476aYM#h=Tl6^s1P z$fB+WRE6To#MGUaeK!fqdd~>%)V<1tM&3pBZJ6JB7H@kuloR&NY2yPwQQqD&H1g%B;MdaxvFSJ1k5sTIfnd)UWee`y z$%HyBt)}Vx)xdJ|y9R@tvl#&jtx|v(kcez0iM#wtX!%teuHo3uqjVU5sFuTG{kXM^ z2CA8@zxlJCrnEDPl!RoLy~+1Z3Sbr}LJ0XMod_ zM)n|X!cBpL0_TTbO9!4ub>-e?JA1W?24(YAb6-)JMI-|5uBVIhhTe`nhKi4md3$Ux zvGFRW_B%FH>#My_F1cjZV&42pEqyL2DS;d!Iqy1$7<>w3NKTC)66c`FT&_CO7$B4; zF&I{~BNBhQ_aT!2O0d&wGz!PapSsHYs1e*rF3oht;xI{#zAxL~Gx!|LQXZ?uhf)cb zQ@(m(P<6-hIYcqt>I1)(1f95w={I%!vM<6Z4$We%?%c+qDL26hS$eLLoX-LRF`@V? zS&X-}f?WX6S1rP^tk5*a6!P%q(v8;WfslMm*}htUgPdO0Uy)B@?W0W_`1Kj?Z;GJ# z+f6vU`e->XZi(fz#m?(&l4ANovC{kMuJhIINVq7ERU?xBd|s zR6y>Yk4jP10!v~w*AYQ}WKAFzmNKwD(iavMt%1AO}iL*Y*U*D zRq#GIxH^(Ev^+Q}kWL--dX1i(?(CfhXxsLD+I~^rFNbb_|4S3yLzo_;HumglJA8UcYcIdL1>7VqU%kgmYoUpuX>n#Hgh-lj=0E&tEE$Bk$7<|H58==hpT?^xsrxiOf%=6(uaprjM zLr3;xNoYrrF%ct>h5SVP(_>UXqW&NO@jwMEMM8Ke&5AR zOJW7owc;9vDXx=jyLLG?-j}zi{#Iq2W3#geql{_ z(E)(EnoAuhw}4hGpd@!nNtFu3cJF}>SroKNfZeh`6NNBG99uPw=Ph2tC0{@Ob&l75 zhElCw?2(Du-^L5v3yoR36OP^UU_R%hc8Sho)}NH630yg~C{uq~%MHC+ibk^NY6p@% zet2^Y`+UAWWA>Y;uU+HCApm=p)a`9q#fLM;cIQr_YAi%C-IyJA%+8=hmzA0Rzxj+eD!7+W@2Jtb)0UCxO4615*3$+n5{ z7+oS9`xex4^#*F5Q(l(ja!yXLaLO1}<7=aJbg8|?fH}L^QB~M+M&CGXihA^3P^(t{ zH^j7$=Ytc21eP?zc?UkS9bS~`3U!L|@hv-hRMg_R-*OH4)pot%)58D@TzVpP)Xfb9 zp&p2{!X=QjI0c9e=>TIVd<3F3kGoSvMQgB>Hhw*yzXytpE?J+!xC{-~;fvAQC5zqq zg%f4S7Nmx&J?5dTX)cuN{?3bf+5Ne`3{jhl)6T~8PCJiz(vQAmd7eOLhIoI>7O-_# zJPW(Aqa;D85ig!d*B(B87hUOerM3yCy_^}AR%~V!@YL$0y?2Pb-z+6NCw{U5@O_j? z-oCIkf|{cOpQZCM0}7O=#mh5vB2LX-p47wn%4(n;7(!C@Dz25cqUPSrgt-8|D|4?b zu)cSb(;(C_`>gudE7g(fjR`c$;C}$GC1m|jnQ}esx&g}tI5u$j_k8fZYpoy zLCPv$Z(@y1Xs*!JdQ{}o;)@-nFbb%@=5&nf=_Qd48ek4taUniSN;2zu_(U+=NsrUv zv2O?UAI}2^*kX3Eo)BSRiqNgF9LUx-X!Py;@u@jnD&^I}PZU1Tq$IXRe_=MTIkGx^ z#53^X*U2xqrS7>I@4oxCGudx_QSgHpB^m2>=wVPH2m1*&%37`MWh(+SYm- z?I#mKEe`YMF`M!BUai+%+>{IvXTNanu`BJWLty<3)(s0tV5vKuN$_^F32!e+Ihhs$ z=Hb=1%(*hMh3kx$uUw64m#+49biEoE+54v&y&zobq6pS0-}n|2%b2Ar#2ejBJ-(`0 zr98};qz(FdnKoV2p-YIh>&y0tg#hWL>AEk;$oR*o(w|O1L?=xn(a>uiNCUcjJmlSf0Q&iz^ zs3Midtw79`?Ze4k?2$$V7DL}=rR=-49=zt${j#$B>Ht~oZcng;xl|Y3)o66^e?>2kDR>y19 zxh0qgjDn756D&-=t!#T3p<(jajEj*Jl>~@(CkVLh=Q6;U*{wO2GZjTI_JI}i-*n2o zFGjp=BI;mEbP2$mj%fri9nx)MuBJI8gfBgePS>$G6lJD`V%}69(pg==-l9E$zjib4 zIg#MVp4@JLUOiSdZxC>Iu3%kIkdqvJQC|8w7n;6&(j6E}(r|Uwv&C$AN+vrOig5Nr z35EQ?uQ0j{TkB6-+WD$)w~om*&wX@N&$phn?8^N5tUPx5?C>Vr`zMR}w$T)U7->j{ z3XIQr+TUPH3!j5!`-I+3Df!M*qnW9ddrN%Bym-8WETb8&leFD~lFul?xmjbVnp{nZ zXV@cFJD1XR{8{SkR(t?e#|WOZZ_;>DO;pOn6&gYFq_k$Db5A=1Tc;1_n}W9ICUc68 z@d~gr!0xvIt@p|NBVkUA8|cdY)^9YbTU##7r^lRc6M1{NNu0I2v3w4$K>sOmF!=1N zOjzTWKb4Syanxu{5X#8NdyPm`X(QGK;OqD3d!CT#758LEgO@S@chtVF{+uHmH;)tb zKy{?$39*O8qhKa8gacB@=h`t%jgu40W6A!2bS?fEnoNEpR1SAt>hUsqKeIPr9&%j! zW^c-Ot#WI&N1v?~3^QuctmoYZawB7-DKN@3yxZ<#rj`8h?H`TYO!#_UmowC{H~qM+ z?)Qh#_UXt8yxvV}p|nVEmx|q(w@Y;uDWd=y{$pDkM?HGYNaQA`5>TiMUUp;MBYmE@ zd3e6p;|(a$-U~Z~vu`nJ7IV`}!)LQbok*1=i_-6syL`=O4`i2Y?4qkjkbqHTv3Qn~ z)xdPc{kS}`zFwM6U{Au@e7LAoC>aR}XJ@>zjjp-m48S-j&7@EDh$P8-Pvt!E=5EA# zpEmplXVI<@y2VqrCvq9?Dm?>-%4;8QTMsRtLU#$cvL0A6*FIsg&$-WQhn>={4)WyJ zKqqSk3b4BE%eBI_??3CvM#!2bx%!|$5o5bc&6emSMbi+~5V^4#MFjSuCBJ$uAy&6R zuFbK)gG+(omv)N`%f;fPH=GE2F5l7-<(>Fge_&eB`iBg%u`nsE)<@KZGG zjFAVuR&nLsDOqR}soA~ouG65Fc{|+m#9Un+RmRANYN(vS&<(2PVxtdqx24i^GtqL6 zyA0FyM%Og#c|u;LaiRWO;Zuym2au){%^a@{jDn-cd1G*W8t6Z%gQSmAIP^Dup$?l# zqd^K46au5NxIH0Ahb}Wic9SX-0X-_%*IOqLD zJ;^6?Bz!(v#=n=6usg@w;1IBs#63^IJCp)n$@V%dxJ%#TL2z~);I^=p0$V-|z6Hon z2$pW}dP{%5X9HyEgTxJ=7UbWX%E_v(xulAEi>NPE%q|SY@i2FYOA0}3rM7J{35{LM zHonYb>&yn?(SKU)QUfWryR7Sm0-_*juw$LXWMIH<>_v%-h0cS0G4yBdIN3X~A8MIc zmL$!EMHNalaJFb1X+m(cN)1&qp@};RLn^HbE^PcdA~276uln`R3_N|tMK2|@S<(+a zu@pe5M0CDg9R!BTkW7DQCdcCUCm%^Ds9WrN7u?8Vy2w(i@U7>Zy^|r zB-|u1cs}CD!R$_x^`#4=AU45S6Xdy~HnB5>HJs9d=-;ER_CJ6$F`rt+<_W`eo`@gS ztK%Fp!2AUj{6{LV*S;4IUKD?ea9JJ>2h5TW7e% zc3PIeZ<6FOiXIWMpgUl-H(-2Emf+WE`5FzW3DU;R><=nurun`#yM3u_1XL~%Tnm6J zw!{kWCbQ520u;a)N=B3dwf*E%rE*{?mk59MZ(;1J<02t6Wy37Q8jZ+_8;&L0< zyay=6BDZAHMw(34U?=*QomliP+UJ#xH8=k)>Hn`2B z0dB$fTxxM^oG5=&p;fQGk=bciMB20(|SDKLjR_(p=Tm%w>T!Dq_-XV&q z?kfk`0aDMC9vBrL> zgU&VtG8<$4B7aG`r~JT?!LN5^2lcL?bR2^h)kR42!;@=A5(zn;C2d-pIMpdE9`1vh zqH;f47pA3J0y61D8Z8&>oSo7~6HkEk*uJpItmr26{t3}@)}nqT)c#93x7Voc{*q_w z+AY1#f|82dm0)uMhHyyHD;^!4`0{WUs~n82mnX!tPY}cp(^31*Z#^0$y!kCjVLfh{ zzX}6NG@D1uRoCiIFPswNff4cPZ+;@Ma|@c75n$1_^%-=m^?Jl{PxsZvSUFqW_ItDW z+#I=j9D;vNE#JKKDbU>?N?TBYB|9i}~xv>C_m z`)$yKl$pzj9)H7T7GOMMdw^fve*EHG0t#nIR|t zqsWY9MEg;nU&YaT7PLv8evtUvyuKr#fA&2!tnnqbj^}nr91=J}XO$Z9>&ncuXw1me zf}0(?L)^*H0(!jP$$bvCD2CRQ=nZpb>46;0@aR>W88X{@s{({Yi+0OI_43@!=(+y=6mmWk2gF=bX8PPZx{2&?FnMo%>Eq{*~)Fm)nwN!Sqfk zZrlFKK*obZ9M!ZcOPJ&CM@p4!xvYg~_zFA0BS7k6ISig~s(|Q9w)~?QHT}C7Jz-s= z{pG>ExBn697HfR;>8_p{Lg?>$d29jdAg#}43fgMmt^-AteuQ@pDd1Fw^*_gr>H?+> zYWp}R9D6(oQ7Q01FM+oLtpT^$lo361Wd@edDb+x(F7^QFFjw=?xMEgkTjhd1owP>> zh8xgBM}YvHDBy(WE@3PJT91WT_dGOWkK6&$K#ZorY~bdW=axHpAdIFZj#=CO>f#6? zlhty)jB}2<1Yjyf25Dw5ncvn-eC`ENxZK>j?4IjGukVb8zl;Wlmx1`K3|xr3RTAdM$-<>t=m9( zA%j5JjDxxshldP=vH?DiR!K+n95g@H4e%85b_Juj=b7m{SK$)y zm(1xYo#v2LduB2ZRb2qJE?$c9-q_FhieAC z^F6g9$&PHK@d|Lt0^lzt&!01fqwL-)W_Ytrw4H?_Z7DI+7dw-)6r5Q(cB_LnD&5SdPvC%n zk1~s?IP44nN}yvCz|Ng2&sQ4*Y#y_igiU1+n1)ZSHz%qIy$8U&X`4^4BANqjBKQP% zam*gUw7dXuH5PIrg-Wp-XD)n^A2iy89`Hn>V!*?7b0(t-yYXZ+YLU#1E%9(mdzx6V zhv|xrwZE%zq-Yt$h$>--gSu;@O2slc{Q|wM35qSZa_Uw$Dm|Ae&}ql=yqHI@5Sy@E zNY^fln3LL?@EMG@DAc7>#$QjCdaP3LC*3nx8%Op_xOJN6h!@<{rA4#Cvn*Z@AYb(P z(MP$gPtvvlI}RI74-1kDpd#gmU(P`OKZKzjbnoL@J{Le(@aAg(v5J)GjWUe$b)-=; z??~YpuGF~dBIL4>&JOWYD=!)AanTRGyYftz6gdyi?z7g^#9649KcuFO0UN6FR-%r- z>ZnSqff@zeD}Q#e-b*SEvCA_9_T#-ARrV6Ss}tyde3Dr1K2>91mk+PG7?W+Y4%m%R zHGIgc=lFPNoD@<%E*P;4QzrUD6!J6Ubr#<^`X;N((`LG1a%O(Nm?&({$w^qTe&=3+iFb@8f3 z5Xrf5|9`kv%SR5?%(y=~=rpu{Zs8+cRLqBR68x$wF=qUe-KCQrZ(lGWIu<+ki2Pd2 z8yZHtM0WT0j68@T-twI$5gQCJ(X%1nG$i|a=d6E3(RZL-+7uV%jP2hoT>Jn2q!*hAuDr)<)x_#2ZcEh0JvvgUICLWMz zym|SF4mlbL%O0b2hBFp=QULbZy)f7N)_k#@t?7iYoL~3B<2yas`~W1qZHcVJvyY@V zT)J3BJ*|4$DdmYJY}q zioO{xoIF7o#{1o(@6qmZ_S$E&<-*XqvyXfxa%55rx9m_hpN<&SLzy!AvMDm1(Rcz) z4lbK8Y52)8u#0}I73ewC98vAmeWDvF*0;_p{4UFq8+3NGRY@ET7KscUf8<@MQnyDX zsxrA&(rA0)YL^9dvj#JjogYJY`$BXdbkK1ersowy6OUikji;#*)2*CMUo||Q4VD@I zu7(*E;8^)cOxu4XNuEU%d#Kh7jdgv@xLRA#rd8<^ipC%fE)%M3C_C2UnG$=}dswK= zP^)VD5nc_w`r_{=)(%gi`}yT-wl^N&UI37$T|o-jH1$*Ok$JT;^QF83aDb2EELbdf zmqUV0REZ3*=Bnc8+W9m=%~PqM3Q0P3iC8Aj{o86juTo-)f)HGJKK59H!nD-35T6YR zeNH-g(^4HxbDl9a`Dt65&SEOLdJDf;fpmB~BqKKJhzRqDzGdAzzG?{YBqRkh3HQCr z%a%ANEr^6e_=8)U(^mba986Z+*4jYZbgt>j?1uu=?(T4Gcf8OIcH0Y_X*UAw;V*vI zh%CTBQKMBdKrI9aVB;+|9H;&K+mnEWWQO6{NxR4Cqz`VXX3Lj85PCvPVib464*L$o zG2!QwnkZ)1p3$j#qaYxTvHTIo7XCaUTR!S1V-NgxrFZve{@`w`-U9+z5p>Sl5`&UF zy7pmtlAc-ry91L+OwIymL;0Zt*9pvNix*MpbyYlZKcHOKS*qXE2VU(ozU}o{+iXvc z*E2b_1H}m161F&O+({OM=b_@2{-7mgx^Vd7Fv2{b>X8;~12@;fh0xZ)6K>>69Nk-_H@PzwQ)O-L?mqldrHUzLARMvUexI~x2%n1GW4U+# z>8B5}XALD51^p^n{b*yo7tsw_YzWsSXuBSxv?V|vX5x@~t#qFuk?oEQq_cIB0YYj+cIE*c>1(Gw>z z+96*RJo2S(+(du$^444%!6c-fkv23b3l5pj60VYISO^)YLeX+*jvK^RH81EGp z4?VSJRd<=~a72mR?ct!foJZuHsy2z1@1x=j5n%HuFw%RgNfCZveJ{T#9m^~ZvXmJK zuHS_$2VE@tQAlryZX}3!F(*SA=?bcg!~-5`v*&2_UV`+TP-f(_LkU#?qbp`@1>vbn zg%SecZ&>OkdcU1n9#9muu{1Fd4#36|4%J9q7_c}ffQ^$r=rLcSafAq;q0KL45f0WAf^Jb55R( zR^-8Ir2A`PSMMUhWYm-8E92>>LyUN~nf}nnjX%q@xXN3gC#T~iHF0_sLMf&L)ASa~ zVedN^roTbqd9Pb#S0^LdBi+I?m!wDJC+C8p61vn|s6hwmB1jS3{FOy-`jxAVV+l|T z-0wUZ_o#m`2(hYllPV?}HhCp1u68pVhv&GvgUM&CS8(|!&(2iKPJTXP`HCqA?9it; zAFOUh5trZLDUo(4iQQdJaKI>K{m$Mr)DApcnANhycAm!BS7-Jw}CA_1I6 z?6bB{ZY0>dJ)1t>70(SnaPyHc`Owwp_&Os~;Qtw)ev$<>9q?yv+KjvBpxTlxcWt9h z5lL+3UG<4S?{U`>L?R_}1f%XzzPo{T^Y)9>4Dpfa^~WG>k+<-?sdwRVC8dv)MD5Pw z1S^lS4R7Q~H(ok*4H!t_QPaVI$r|&NO7-X@TaYX!UX85dYW~mckKGGQtWEMV3&H7pN#_WI`Y*SYMb>e5Evyna1iVWZ*u14_rcV3ZX*^|}d* zL|gzM4gv~p)=WUMnNyVBrNCps!d7?$wF=`^A4sWReN$qUvbda6RBXjl^c^%12@d5i zPcV>idU4ysVzt5Z3>v&z-xmcbO4i6B$qvUTK$0}(9gaI~RWFh2N{U|V6>*g`oUYXkc(iBWY8mtWF6b}k)7}C{c&gLsHrX}<5gcw##n~x1!)4_By z?G7B|4lVOk1{W9&rsm}chIwDFp8B5RdX-&#yllN8I)U-JoEeb~c%d{f*_Rh9<o&izyb$1fM0?Zao)N_m_Iz9ItJhwEJBZADLkd-6YjbQ_tJNIuJ zrR-WPc2kSj4y(akTki_iRSbTBv!w1_2_ZV1p3l=>d8}KdrY3D4*)!_xUFcWj81@;# zwd{2^DBgU{*32SePBe%6FOr#IQLI^1%dOb^=K%ElwtxO61i%5>5eCG=0B=>uvOEMc;+P56Ru37G2q^ zefnK@DIQ?k)m>el#XzjSW~}x#=_@AjNU|3fBdk0txKWz9%-yzS50X7hB`Zmuf&RX1#r+)nV6EmF_jR=Zh@EmWpH zNM0^$BWF8E(Nl1blYctzC`TDC^HQafrV}szOs4#Z)@388%W#k&7yM*++ z2^WLLX+l4N$+Z1u{ttU^9u9@t#}Aj1A}z;KA(T)gWf>t$i)4$eAx4pPP|Cg~r6OdH zu_nXV#yWP2B+J;x7`yCy_MP`0GftiI{C>~5-oM`Wdd?r`;>yf@-{1SYFQ1#C5ZReU zofpIjH+4s+-gIymj6nqq#*gCeYCViV9{tRfd>z@FgVYbv3V|M#1a2V1(brt~W`(G( z6yzh7!&VIs?Pr*knvCiqu{3g7#$ZYZ{n~(E>I-VWsvK)5Y~RF_T)S0#x@Z$o0d_b3 zRDfToxA=MBSGUrn5|K~VGSz0=UeYyX|3T6COI(*np7rLbp!6Si_FPOElho%*Y3(Sl zHnrg}<|!KC3zE>ao{&(8+;Bk`KCg9TaW2hCI?$eV!Na^`xTItDSxOv5zt$zeyNXRk zlgDy6aLQmfs;SP^#$c)Z37-1lP_rzAMriK@Ty;Z@8^30{hhe`xcs5J>Onj7O@iO9p zbV?j76@{uheQo_-iA&L-aj~<5qhpXsn-(Le4N@-nFe{va`yzkTdt(}6`3-B*9)R|K zl&N}{eY(%pBB!RDBQR@P^OS(c4n7S;P1f1xm( zsyz|5UFkfd_Jv%@zJ`g}5S7D?cV@;01k=GtfcD2IZuR8)Nn=Dy7^q)iG2#MVZR6hv;r_1Y{fLe0VEA zl@X5a?MYU6g=P_tyr=WT5GJn5|GatZXvoybAJTcI35bVkTv5rNUNZ<9(h- zqLKtOWu~d*7CB!H8CVP+U=TYLu zpP$&5-b~FtbyEg%-4&I*@slS9=7uuqfY-Xqt$t{@eqPAYLm`-#AH4c@sN5^_*h1k( zSMoFp)RZgZ;R{ZJo{6`FUVA8VJ#S7>jN_RfyR&DIS7-iy-Hc#Ju^4}zIv+l3nNgx$ zVyNk%Lz}r7^=%Hg+y$q7E3dlZjH4j8xgXCOsPNV|FMa9OS-Jw=wQ=wt9SDI=o{HcQ zXjQ6=RTl1ib&&a_!+7FGojLE1W)($b#dTY1{R*G?Q{EEmBZ}FcRV*iS&*0|@6{ni! z7-%1+>J-=j^k7h>=~6_kBI|Hpz0&`B^&dvSX`Dy;-eBzFq@eo{S|_m zQ%B~@9vHtJ+O^5e=1n^%d6&uEKLTY2TLz_OZmzigjc5)T3$g!4E$w8%1| zk&L8R;4(v7dinUEjk(cg+)|^O`36V^^?3vSWsPE$rjk{=oGvP)1Vg}kQ+Iz^7TUbj zmY!nO<%Wf%)7N5~%-AzmC-+E1LGyVFvP1_z`eP5O?6dc@f$``-1qqLs`q)31^Y_;ng@z7i>IQ{ zw@TWa9wK$$AJ}{3>{X)LIzH`!{uW`#HpW3<6S*Z*6PlmaxRL>|$nL#WM}M)!GXWUc zISqpV%yUdF2i!%pzU#mMG!o`;CM2SMR}HA24yUWGa6@RZI!g(e*H~DJ-s{NK`D1$& zg3@OyFTp(*1XR?{EfU6+GRI7PK3sRLL%66JC(Kk)&Q?i)#BldYOO7fZFUTnyH*Ji{ zw@;q5?eL3F>i2b-V_Q%3Z+RQaZ`3v_=&Jj;oQ?}wG^^$^6A+c~(Wc61 z&^!yjyl}S7AOq>$W&=;*QnD{l;lQ6?s(Y@EyZ+eRI3_YapXXARivDD37~k5y%CFtD znuTI?iC4>D-nZ*46ud>8oIV?&Zyk2VCNxq`sH!Ni<6)j&r9h8M3X?_1LcoK%$GWrZR{ENfRl#)B=)>?G(P1W zEVwDEPJN52KZCqTgG@tSY2iu)+4)Y|ti|{lUe{tXt{9aI_Mq|uvOhOBDY7Ann=jiV zuO~Svm1tc+3WuTIPJzH)*L0pU((c;z{pNIn+5-UU_Wna=$MH+%8czhEm!iHhas|q# zxUA3P#(1C8FV}wqePGIFn&%UMYdX`t^i?R(ag;owd%4zD04ftF+g9Oe0(_xt9xl8p z2H+ti;haE6b=!yJlx!t9+5tc#o2-7wRToValNC?IRT1(O4i+gkSMsJ9`b^}eidQXC zlD-<`U}u<867H(741BY}4T`o~MYLfcsqn&(jwaW8IRS#3%$J_r*c}(TFsQ`-I4sw2 zl)h+e7KGYox_6IGsWGLa8`LF3QS9#>%KH~}yY1>;iCTuGPw;+CL5Hp&ZDnoJ4^$c+ zW4UKTy?bvW_r(VpBO~|S51U1Rj!SYs7mARGSSg3baZrpF`z#Lvp2V>UfFZluqmZOj zcdNl&x=>5OuHBzMKp$d#Y23@HW)R@d^zaMz>OpmBk>L{$@|2btA{v+R1Mmgr5u0sTvdc%uQ+=)eDd?A0!^SQWi52K@GH z>Kj?9oH+mm3IIyzEC!cYqhIeLyGjlc-Hubc`;a_@6x@k9xq+aHj-(Oqx2C0En$^lF z2)UG9ZqZ~hbqz6NyjCk5_+fHZ)jP=#l8%6SQwY%K6G#6 z>sJYu0Tta7-#Tdw1{6mt>tW$oyu`Hg@`%WIq|wfd5|9b;c`fKbwdsFf z%G&v0Nyo#2a;-R;rzu}BZLgBB&S>mYc^1sv`MOCq<#wz*uZiYItLrIop~2nu_Pj_b z-5`#+bZQf8XP9++s%iZTOjh=2h{lQ4g!fnRsA8(F<7U0(-$Sx0s#-@^kk;5U)Y92k zLqKc8n~4y}eT<44*Q1z;>)sj>VJKf*tmrvy+m>X_;QY4JW(LzH*yy_|bsK6K_Py17 z;V^nso(_+EmmHsACf=3$tS>ve z9-7yT#Nh;McI4eJiHHE8kWv0y8a+8smRI@UoC>zBQyLTH4a1q%kIK%?sjy%gII}I* zmn*H+8e7!Z(ZO{MPVj*9mhBnKl@o?TXXFNX6tK~0%KAL6pi+7^x^OfxDJys?1REW- zayJZl0GjA9mp2$@mi3{FDN&Zzy{*O|^m&t3r#}Xu8IMJ(cGCwmcu=-@Hkpnsg{0*3+sPEIWHR#X&L%O` zzCXmC!G++-qnZLcz0McJ=mJV|DmMY@SxwAp##gi`=t^Ns%X?CBTkS(l`m}XP{%TSI z4;`Z{#BtTbA4An0+-s0IsrAq$)_>8Tu*>cvfA4ad>K%+@N+9&+E;l~%3o?*`{JK-b z{JQP=pZRs+^<@}2LAw8;G^nJ4y)@m7|X{R-|w8Hf8PhWgGNJNx_c0_yOU|@lV-GoDu#vLHEney@R0_yJu(2 zGQ0BlO8C7kxdBzaE`F#1O7=HL{FtyP;K(OMVe$|`z1b$ID6S%gjX9n0de>0M+aHHy z8F>(@$;fHAEaVmXN~4V$EZS8v=`7tzh8y;ejNcFaF#2fy%2QAsfkG)>hk$fxn&p)E zD%x0kML@UgQ2;~NgUo?fTJt0EJMh_gE2FA>K${!DFFe$lVr>~!R^O#l$oR0v&~Hk< zZ|rMF5`IZHP}AQaAbB_89;2Cv!2@y6O^O65;?WYI6`7|d*I646eD)(qwXNbq(NXOW zS5#S29?-A_TTUz`YRN$sCp(k0A9xw_PjIEAiut`!j89O8eU5hwWkBv|Wfo6K&;&%r_X>}YfmDxh0+z}z zua_R7W7%s;*$Nh%>P8tf74y)Sw4Cqo3vJ;7pZuLQPTL}qy>^jF?e(t&GnCf->gg}M zw|rtRCnE-^(9bLTjYYoULqQ_n#b0%+^RlEz<867?-bZODxAL$Rq`?z_dzf$5lv5sn zxfKXR$8p7XY)q%No9%Du+5hFdWQJycZJ;&CsJ@ou!jFF^G3IY%&kWyiB7W+~3e(B_ zjnhU(1OnUy`?a_X!^N!(GK%wlN!U`8U&5zNR@Vr}eX;r)a8BU|$D+p_=Zw&fw_w2>dU3)&VBtXBzetQ=~Uu0(8% z++v4o^$Wd;2y4+(&X$Gh|4wN)I&KZFjw>!}xu-o&FB%g_zF&Y_i4#BG9HIYu0x=1q z5cqmzmby5gc+0}sIV}$L+ffkhs|3w`9JB2g+ywr7O4N;zj;NEuh>HnEBYfo70T_Ox zwK&V(46=O9CTU)Lv&6|@7<1u|%HZ*DOwQI=E@y8Z{5Zm~vT;LD?>b2>)CZB7wOFcV zAmh#7!RYg(9_s8pVtkbq4zuY8RK1yL8>{mKlHQd^f5`bj(wB}T&C1gz)BU4st1laUv{xinek4!51sPSbPCIA!f{*CTFMEyrAdiToKoUVKN9 zV@@&Git#r?C=_~-Z>2F&Y-M58ESWdcbBL{ud1V`(S6hrI|!gZkE1pF zF9Z>IQxkBCeJ_XA)x~+9kCQjCpBohjB8hPs(!ZdXA=C1=3F^P}bX}@;@C-a|KFTg5 zZ>;B&XZF}r)O00h6z~SizjL{q?jiFORH;NfP*8*v-Cd?x$sd(VLZudeT4CVEk7xXL zLn&l%+e!6Y!n5oz;Zhb&caH&9FqQ>(6zp1d{Hd3YEAS-o&Bw%NO8=oyEA@w0yu#~2 zR6C}cZqTHgotKGny;Ij%c}yIDFg3F0>n0l82Tzj2u1SP4M{u;6IaVUNPJK9YhowBM zsHiBSQAr;_w5)+VLYt6H?|1eUvi4YG@NKIUFQ`v#o%Wfe+TNDrZap?3YaU#8w=2N0 zIlX8`T7z|BmXQtj}8K5VfD#SNNh8{#+AC2p6CPZ ze+r%g@5pirh$1LLVKTYk^CQ4p-tA<;LGz`I%-Qvh|5@gUo(rf7M)MyaN|kn@i6R^qyBs?$4-Tovx3n zi;FRijzSZ{(NQ31TAwD1Qy8u`ah;Po>9Bv|vs=ISnLQA>4T={S(iM>;O^`e}=l){v z)z!^}0z}kRQRP{SERJi|M*cARk@mOdofqUFAbh?Ao}vWctA${;5mh)-rEOUx2pppc zec^}&qVp0BTr47g_nd*VLl;MN%gQh@qCVH9t(#&#eLHjdWG<{QGkgt2dqeIDEj{_62;Z7r{J;e0S^EP}$LV<=Y#qdSna? z_LpWhM%5=v?J9koWX82kLlI&xg75NoNE~f`OQH%<2?}uc=uo%R)%%H79AP}kzGY}v zAo|)lqW%~F^&q}Fo*8Ls>Hw17bz+O*bq!xY(0uXY#UuIMkt+CEM^ zY{4)_XJ8$1UD!`5Jb$(4FxdsY9Lvh6;siFD1*z;Qo^->4jg!&LgoA7`4B>y#I%JhD z9zGoqkJf}v&(Zh^or2^C0I4zaR<|K+%my@$f!R3i*T^NSG7su5q|4He`(oZLvkGG2 zEypgo5e}o2@yo?T6ot))zdisgfeQr=|3g&4IH1s8mz|G-C!?)(v4q0Xew zGH@-nPE6&G0a!bCZICfxuMVB8vT5ekM%R>KKrZqj)y1vgGXoa(z}KL4c^zCx zQ85{71SDSlW=!uA6-!E_j#;HYJ6cs7{S-LgVY7{LydxUaWLWe#CUOO7LM89ZEG7b8 z$Dx}&^?@KT0Jkq-UNKBShP=yhN=)j5gYu*oZ?W;qdG&h2Q$9XOoX6$xPCr$xZ z(6dM{x*`u(st;xIN9xUiSH$FkrlHwE2DzZtNClqDS>y{-GBqi2=oy=6e({&GcD^Y# zaVxpB%dO>CQK@D>8}#el{4JIB&j}7NiBVM8*v*$dmOo;M0*KqC2B{jE6vf#FO^}!< zhnb1zE!6&C_^G+57mscj7b4X4&E&rBu2>o+fGW(J&tmFNgc^`m?u_WF^de zhUJ5XG=QAy-S~z1wR3w6wmtS=WEn_x+iBp^ZrPavR=g@SpLyTdo5)wTRZQJ>ecpwE znYTdC--OoGr|ulfD9by@fzR#~&VDd+zqUbsF?{oJc`@sjp|QLS-MS=Hao9l`5LOs? z`7g(o2&?K}Df&xcX@84B_^6=SwPnQN8(3B)TgKzZZ?(kU2}cKiz>?@3Ouns>pqcgs z7orxZ!;Pzhj_*wFXOgjGdpMf~;}7MZUU%$m@y^rW@6bJl#DCG6MEDp>Eo+H7FzCozc~0_APfI`D{=P(e-(uG;KcXe>`gjuXd_ z@iWtmkBA~>si@*_X6ngEGI1!455ibb7&`SmD;uDeM}*f200mYL6d0jD!YkZdXreWx zI==Ij55ytQdhQJSmX7^ZU2ZAP=-g)_1Ap#0&&*$8%tXRz>4zsB_uQzw7zW#SC92+w zqv(3^fB}f7BXr*=EHUy6y6|V;+QVr+Cf*I=;BTsiZ~ z#uQ=Wx-vJSo@$i6r!lT$aL0) z{U$M%{zE$J|HHwA361Q(n3wDe_H6 zKFS2Z{Df-!706NI6`aWtl{fgZ&xXRR|cEd+QR48=( zHi{k*g5eVuD6i&@@=C-mch#B=_?-4JL5v;-hO3mJ3I;tSgWzsdja`xeVW@ zum^LdQn2^t4OGM?#APorp66KYUB|%T5c9^hmC8XgRFUC`Zg}M&fYwQ|2cUJh*UkMF zBDZX63>WUBi@ mYCB`0$3j~V+t`>()9o(EXn9-qR42A)*)Il{NTZsGciumgQA~zqBv2np@ec!|hslzqefD zCMIArU=Ga4dun7vW+K_cyzK+R9|K~#HZ6z4Nc+*02xQnY-7mLj6OlcC%T-6oye-`k zGdOXTw>^Prr`@>SIU8*uWL=D;y3l58#^K12)Cf9`o*2agUAeZ@=gSk+R7c0HZXnt< z|4iLA7-rS+0Tm~3F|Q@dq{-4FKqA2Mv2wLEO#=9#4M!z1&AJf{Ir^+Ne|9 zSxpLUJKC)Rc4c7xED-~E@d9|l5{JF9a4Ki=E1^HAKJ4#Ir84u*n>v!fD~U36jv7~{ zF-!3kiOgPET@9Fi$QL7bJO1WV!gC6F&GA78R{C`Nm>VQ<>R(%Wq z4)SO+ovE+_NdHk1zv92Rv0D&sTFMv3iMlP|)y6MFk&da_@( zKcAFzTR13Z4}Rb2WYNXDKp5zpAtwV#f*-B7B797X__4ak>!Djsss9Waw#Z0(G+HB5 zQ_LS@IzX01PoVq%t_uj~|E`Nr8~C4eZ42A~r0Z{`3lyZ}|F@MPV^pTM;vAVbi-PRw z9aJJ~{RKBrB5lw1ok#&Wx!-Ne>7njneKCgIlyB37>d6kuSc~R0sKaNAQ5Mlx8d~#{ zY59PV3N+4cxffEk-{_g>k#5#y2Od2w#Vhmg;Fw7|6E-lGQ>a4F=D!&MUMmELDEEhB z4mv)!jqi`{WahpRHo%X*t?oO{BcSA|3l|lQ+KKvR8JVXYVsHb)FX$02JJ8?l z3W0dDZ5?=!=EP$58w(ShJD%;B9U2fp*FaTtL7Fq`;{0zp(k&kEKLF}pp}4`?TFyU> z{kP??@#NRK!`8wgF)xw8NBZsPpx&>@3QBo}5L+AZoh~0|U?eqn4OQK6fJc7)*HdpZ zqCFXCNE`&ko7tVNzW9Gv|G%sMpVa>$b-VumE%g?1HVdGKd|__xEu+w6@#GwDt0m3{$6H!p}$%7%+C#<{`7ovdibFBE{L#ox4&&IN)x4xigenfe~9s|`G* z`vfD{oid-dHd9^U%i=mHAJaPa&5v#Mf>TC8XwGm*$y%HsXMt??AgzRm{i@@dyl281_4H-?#Gve2~8}*F9zu_L+=MeL@Spti(0qnD=4f@ zm@%r5Gpq#}@RLhrgp&0V694}UuoAKe_V973{|?sditZWwl}A0#INyW5`4XEZ?F`q= z4QXckFCi{JK2sDY;NKK3DbM%wQbN(^pO>IyvUTbgg6XF7-c5vnaF-hwDf;|oUwF#~0%}s#7hLC)pbsfj z6V)65L~HH@x?Rh{K-URLIdhz`yZyt>c)!h=4pTXZtC(%P!kWeBNRfeA0yzwB zK~z}RJpZ{Y*hLUM!I8Lq*B`@$*pqdF5FUOYhUH%26rJnPQcpZ4aQ(?&-dY0RCA71qDSmuf1HrY+ZzCeCNp2 z6e?4?a%Mi>iz%Wo3^#u=M_^VIW!c7WSB2OtdrxOS5HLG`yuEIpVuXn3+T9)qm;rvW z*r387;;?dWCA6$^-Bh~>sP!7kgtv(KJLw`` zuukx>pcXqAKQiq84jzyke**9iXw<%A6Ybk9FsxR~GXoBYR0hie`%Y_|!uyL?@dNIM z#S_{h?FUbwp#1Y+IvFi;Kg+f$7Fo@KXZ#k#W=g#kX@y1XwU^6lP6iv^ZnwMfaKoxp zWOY8E)7?G^Ci3)qVh(z3ghk`7%=%hVNDI#z>|zkC>ypustBYaKMqx>cgf_N?jnjOD znjgK|$n#2Z!|CgS>+CmWlqQX)HO+3n>x&U zM_<)GqfmjHXB}r>u-U$p&87b`JBFn=`(fW_8hk0Gz-F#p)Ug&WS22^aKB~pYD>`h4 z)e;=4S4EwNlejcyb?@Qh(Rh#4@C@OC13M*nuzzi_9_V`8ne{xWcuEUE?{+ED&@!)!|KqBMoG zfi>Jnh_T=IJKRXsIWP*7sCk#4fOikdFbFQoAGp)f@q-++3z{|YQ4kU36CYkauDW2B zI@97&7I{!hW^-8rs%hEO7d&s;X%x(q21UKU$A)wK(N_l>X_7nC>Ot=e8CHA9IT@ML z-X422M7<~Vucf`RueL#|I`Yqs$0A(^PhOvyd(G0iJ_6+EQUkE-)D7#_N=tt^bS$I zA_96oz>A{Gtb}$Cwd<_DOu{n-R4iUcvrl1BCm7CY8u{JnXNPySIZozD_X$8<*R83O z=0d@fL)J1p^UJaqyPakg$Ro6>jZ8}eFraUB*-22bq&7lyaj|5xsIEC*_lXLPl+SGp zby4xcJ1I*ExOn&tv!$3hG$3-sHXL!@kU*i>xrcs23m@c6c5+MC1tZW`}EqF z)>RXS3zB!?v=bhONrdLBJs$Puw^@=8f}$-iAHW(busAPIZKmCOGx~zgIk6kQDDf2~ zwi+i(?L2oym*sxqo3XBTjw3~>0zn`AVGPG4{z%+LHxfMZuLtBZG|QhpgAhUnHCp0? zkl(D`=W!UQ7)Ew4e-Xspn+5d@SAbMj;iJR%wzh3~V|7YfhTR=J<(wy^yGMBbwT5<(+kdC!j*6o5#Uy*R#8Qn9w;es|X&QE~nFIt!d`IF`tCNDJnu(oagSq zLlf-@r6A~pyhZx-$a87Vf*4fEvd zMNZr4b=7<$ufjwj9Lqv!SgY`lmpJ`dBiLk)uqk`PQV~Z+%60b zO@GJZf%8Hr&fS{=)WenO2k>NFde0CeeEGdgdaQ(D%k}Fi?~f783Malux#+WfxVXqN%oAmRe%9Md*u$+UTVwQ&z-?1KM;|G6LQAC7<(PO zCKMX=mEUwNwkyx%5_Bg1tBDpHcs&i~vQQMP-t(2wGK*{ht$=}9f}(xkxm13m+E#kP zw32pz>}61kfddMO9UzS7320HPW3_ln0L)3A&B>EV>b%IT@JS&S9e=^qYN2d_6RjgB zGSxSN7i|#?h9$b9=5V?n1}gZdcu&sTQM1|Ok5QiFVyDz9d27R)Sa@R(y!4;7JqgAv z3%X7>ZDkzFoUc~iY$3{@%P{|PJ(?;4=<`v)b1@j->9!LQML! z5!}TGbX>A!GNW#F>{o*Ub>A*1@_tYWG;6V93UFkdjP&<~6k$nAY6Kv4?xJu7+*+V^j~lUsoH`dTBo$7d*(5 z7yvV^H%^QI5JZt7NgQAt4#gmEnlgVzb#~sg7F5N!5-^kBcyc;2z>sa2n)l^QZ)u~! z;9k|**!(aW&xPB%OlJ1LPgfswH#ZS5%fp4r10))!4lv$hQXY9X3pc76ZkHENuW-*= zsjfT)e-Iw30LjEBjLNXXY1HTNqqcPg2F8hk9P>@k49)lx=v&^4XBK^gvnkR|_b1YE z;N^v9x&JhJ|0jpdQdqQUxSIQd6M%Ms#_y)0KcAMScr@N~lM(P7`0b0PYOL^T1Eq%#2uLCcDYsI27*MQSnF0+`@yKp{gM zSMs$!R(v&eN0mGZlG-^^$UeVYfA1Th;3vUt3u5Ivq!}t$kXn8(kB&`~J2~ zUvSgus|K5PK=wr566-yG%o!>$#qm$;&#H81OG}&qrHUQ2#K1TVbmG&u>$tAZTQl_V zwy540yR8lsbT*imOcya@KQ>Sv!WEE{pqF7!KoDk|IFMiY)!3mnTo^3?hjGX;OSm|J zt~7OJiTMk){f+zL0RfOJ>*oHHgXhiV>FQC7b(=L03Q$F9UxR&gMAjlPj=avi@u)H0 zTzPi5_$?LR^iEsYl2kB(1jd-5;6oBNtx4;B-<@0h0UB5KF$!;&C*Sv>_=RH@y>-`X z)z=$_n>(dnL93PnklFzcO>d8pN3^7sgr}t%y5Y4cl0k!*0*m1v=~|FhBv=y_{~fJK za3+6w;KmQEh-Cn1hRcg{ELke&ZN3eaD-)C0T$@#`1`xBez)hF;N-bV7bxkq^;ga}YF^j5Ny*hiKcTcQ$I4ylNys!eR9~>r+h@X$Fm6M3Cbs4c70`zg_^k|Z&rgcs< zvfDLS>&9F7e50=JF2;3c^`UqZXY(!XHmwX~VWT>(uBhU+eKsT=)_qBPh2?8_gvWX5G2z1e*rl5;JXTNr=egf! z-kK~snce-~Ox-s^%Nz z;ZG*A%#BLqCkO07>n!~`O30)81lI|8tQf=6_hhV|1ZN7f3j{?82D%=;-$FzC)%3V| zQi4qI3C?pn>Q|`@e)158RO8ABmoC@Mb%&%uFqXAA(Vk<`De}^Rq0(56&YAi0#kKkd z5J0ZzDN^@0m+27x(RE{ukRM2r72mf2u=}!?6E{=9_%T3Rc6Qz3J+)2nC~n~;fP~Nk zn&N@GjcINNje_A9??0a!1PGC-og)(yow<+oGyK6wont0r)LE;iPTW*9*LhVgeF#ZE zKpsfnZW8!`{5e3qke@jGnH4 zW5Vb=?3Y$)IxkXva{c;ao6Jf?%Z$SY2<%YFBDtwkxym}^fe>h)K#HMSs*qz_ngUNF z_tp9QIuOZ>2hZDh~P8_{axx*Old#@@fB# z7CC4?P2_Fhy@*v{X(WRbsV$proG*XG4bpV!VH%?E&{;t6&0Lgci%Q2|01&uh*R0uu zDLHgn@HC)+qh}Vl~N{+9Bwg9C3K@>t zXF7O}zdgZ^P=L2IHO2OZBXPzpup!W28^WWFFv>CHuLv6u<(vJ{TLzHY(iO%Y7tr4u z{hW#XdUlSsv@P3gAnjpwaG0NkY0ydudThA4Th?7}-bqW$UaeBU`%M#dD&K=k)D!o* zZLif;>*U-1=pT{O%}*2-bM!%edL7iv8%^

    -)VlN2R(3F_%$)h30#py?3>JW&T;m zQCk(BcB`fSWS;d6z4kCVz8!k?5)!r&i~b<7cITwnxsx8$ zjB~OI&ZdWQ=0Y4@KzM21VT?(>j5J5cI+PWV#xaTh8$>z~fS%YbCmHJd?N z@k`+0Mvr5+Z&cM~= zDU1mPOeLx9@|tu&1h%>b-w3 z^1rM1{;3}SRlWC5hWvN+-amZle^>AQ?rWLLa9N}mh9uMs zes!t{zeX1m#BpA`R+;o~XHOrQ72Ji_^kZZEFS;6PND}Dwo$L~H&$|dZiQ-<QO4)ruJLhuSuu4pq#>moJ_lXZRy_2Sa*(b2s9`j)FZWBL6NnnbY<# zM0>wieYt}v1NMM@Kj_vZ<0C>@NU~nrox1zWin3lK=Ldq+eONKm%y32$R4*;>9-Z&E0p_ zJ8XM2_P5%sw((vPv*;ETj0>oB(H++rXFXpVr)_EjD4m_f+e@GG7xiydeyKF^|kq=|XSFYC_BF5!PD$mTGg`KYgz8Omwpb&eIziXY9}!QI${DeN~0qJDT)7_@B)w|=(4=PUprqsAfQ4# zDS7C2hy9b_rv#CX@$);9M28hn&XH375(U$sXPXEFfo!L|RS{qILjI0wF4fu59K zsh5AxdOVfwP|kCEb;1ukS>VU2eC4B)t8}akBJ4-VGi> z0Cdt$+)WOc-jo|Zc7wExGybfO|Ac@&Yu#?fAwEa8%@$S8+X`?mxOWm&p z5SKoC70~3bcla&f+21Gve$BcAcnb1BnT|+aIB~AAJTcr{4?^2N2=Y zFi^#MPnm5OwjjQLzfI8a4wP%BzBHkY9qjb%X>i7)VgR&cr)|5<5H7MioQJsX5qBWJ zmzc=gh)eykZEw(&kI$te&HfE@0D|O?7-#VAcsFnN5r|!O$B>1%lW#ziyc4|tA4XgN z;0VgFM*0<+I3a)JHXz?4^lvlaf2UVZ0fHQiktNJffK7hk1!r8Vcul`E^&|-vzO`_if_5~-SYYJS|BWpFLI2mC9nNXu43KUl|*hbR#NPj6@Jp>3ma*zX(nkcB6f%5 z;`X@~ompDhh=Tgt(Y{ZG08p%eslz1tzMy%%%)q+Ao$#pQ<&GZQk>NnewRJ1tGun(M zq>~zaZ@QC6;q76$fA!BEsr{@M+x8NddXnX=-3le{rHYoc7@sfgHekd*g$4NPM7p0J zI-X848aLy&7pt8*=twm3|74H)PO|C6>T(b$cXm@RkSu#GCFzold{1ic6r@VBz95~< zokMmgfI!{Ve3ju%huaU=q{NM0e`7G2%rk2|Z+EIHN^5XYY~K}b4naiLm@z5-#!=eN&*UyW~&&c%2Xu{{t9 z)+es@%lbe0mhjUhsxQQ~YNUfnYDmeNpe(+ZJWpI{h~Kdhc)tNTk)!y>Dz`Ia#v$7x zId)cQo2CD`*7SbTT30IkzziUPaq68(=fXO7Q!MOgWk{FYzq&mq)Vmkx86d`79gkyn z3afVS6=5flJwn73M!={6il%{;M2gs6#!~T}l;&FqX(=Z2Zu$k%$M^_Jji57X*vip6 zvLOJFA@WnT@J%8#30Umh_F}W5Wx6}5nx{G2qP%G9UPSayqBFONf!D8S+3}M1pCX;? zl+S)^(jlIaR1IzS7ADCboRP0c+F-%d>L+HBsqrP{_=&$r9B*@ye3WE3 z+>%KjdnmT^*p?)g#2J}Cbu56HCo1c5Cj%449oI@ZK;jIu!Q|~|%bO4}5-bB-Y2c1_ z3&;df)oA{d2{4*RTuW{2F0mc)4{`uJaiWgD`QqF*OM&%?YyGnRKU6)Cb>dnle#*M4 z4(p)Vvn5n?qRX7gPQwbj$TUeDRpY?7t!diWNvts;LfO2%{y*2cvc1+%9riSebS~@e zMK`e>O75V$pU8#F8ziBVyi5{fO51x0JLNidx!v9-d%O=rJJDIv6Cjbj*?7q#&G#Q0 zBWe0r1=3Q|?Kv;ki64uWCoRS`dn=u1M{`%TO?NAwqeSd1Nxi2e!_YjuR-1;4z0ot= z5DNyTohPz!kxny}pwDXv6g5vuxU7W>KY>_uBuI6>c=Y|t8v>x0Hd?*_F=EjsxUdCg z_=`pa&>lqflqvY6Wg86XCP8Zq_s}(i27+Qp^bY`HM zkYm1C(|*?wOjTDJMv6P7)&yXB2qMM{(5`3Tl$H@~j@JjG=QLu~D?a z!-N`eI9!*yH5v^X^lrl;6Hyx@Q5#P>>*`!LX2D$XQ#6u1CsoWDp!m+2%TFW}(OXTQeE$4dwY8xKMX1Dq z1~=`?M^aKU?w5HOl--UH!D@+lC&%r=;4!k+xJn**r3?9}i8g(zO#|_<| z1i*~I9Imor#Qb>_@{MMuA-fv)rIg1l_N%utG~ck@V*5??J*Rw{22K6E?g>&mpLL>- zlT-+J76lkL3RJWwg)YOAdTEb>KZd?|wo=WL#W+r1=N$|HxlMvBd1~lfr_1U@9Y9Zl zgC^0>QPGY6OcU%qr77tVM-#*cHXrOa$ z>srJBiu8`xqS_Ppt?$ub>=ijw9swt}hquU_pK2}x?xzgWSvfj=pvb+EuUJ)o4x{m9q}=J6>q%wy*mX6_#yLZ? z@1aJKl@QPmZdST&cZapy*eKww+3u@#yb-Q0bw0C3w}sNx>pzG>A|&ij57To&mp}Q< zR*Q>EEa%N0?zY-c8nJ9@%^F>6r>WiSt(^sLz^(OmRq1E!+5eWo?a)HCpfMBhJ=_Z< zFOj$spzr}jcS!eY92u{7KSP5J==THCg9~{420?fP2=sBi&9$f|=L&-@%+F60FO9&? zx0bqDEbB=XJQ&sfIF(Xd3EKZr-FgG>D_>}t>9{Np%xk3~*J;F>enxhZi#* zEBRR9=s4n1NqK2oS>{VgruX1BJC%CoHNCw)Fo-UH9ZXR<(5T1 zuq{x>{0S^nxp(E^&;ZSFog)n|p1y>&eqxKI-$DKZOPlii7@e9b89~7P-@HM@-;QW2 zeQfe5NR`7H$n!o8SWK&VPUm0&1V|pe>FY;9Er*`I5N)&ysErhZJ6uR|=`3+|WupVI z`7I@XL4Ydqxv4&GcVfNWA7($Cjy>58W!UDm{mnGT7u$QAM^GESe$wI#EF!1}!7%Z4 zxC1sA4I3InDQEf`1)}j8LQhwkH42tzrr=#Gm3>3Js}irwa;HTVP`Zy^szgO?wr@Jx zjcK?R={;mi-e{aEd7i;*%IL}^Hst=v_^)BTl6&qoAf$lgQf;S6K-!6@D^;uAaNwo# zWuPlXPJ}s)X?H8zHJ8Si7=TBzZU&A9nk-A=keOOugLP3+Yx$y6aC-ciuyqnV6c9-Dp=e^r*U)y`rI7IFSw^5SecUo12{rh*HV%_`V*^%KrZsFF? zj+!dDAJNrD!=~>YUIthpnfFatuM1gp7d+R@N}dx z^{9$-UC{DX`uvPC1vrF!SqRpuM+e4J1V2VdxVqehZVcR$Q;+e#-oO9W^=I<30{W3G zW~F^beF%fQe4DuovI;YAmj??b^UY?Xfp$GVq(N2=dey7;Jn3Bw*D$_ev$xL{ulO;GznW0v=?bmLmRjkLQq#C~e&9@%oZ>|9?3c0>AKx`A ziceRa_V~6IV70H0R>7Q?QVxXzKE-&@(yyh%HZ2Ak$9nO4OvcKu@>Dv+34=}v~n~hnr z$>dY(qY?B}Ce3#p1{cN>9U~kiEsG{Le6j0HYtRXoL+jV9Uw%<$!9pI|2#w~sD_@XYt!nlEu=kcx zRldR7s3IcW(%ph|cXx?&_XcFsEhP#XxTAI@6m z+xwLd@NnOA&s=lOH8VEGuLr?QI+dHi*4)ya&R&P!jsnQ*I(0HE)>NM6DSe(cK&>%l zt`EvfcYQ}@)|Yb=^Ava5?d5R>e}^}wRIw76;f2BinpfH!MF4kpi|)XJ>x@nFJus!O z&t+SV!u?sio@QMZ5v_!Ld}$(jEeSreeO$Ly<@XR^QN*uzj+k^z(ZG_Fkdm)VPQZF~ z&x)PGtK*C$#YuH#?Q0(|0H;=wv!s?aZf=ZGKP-=%%xy1k2s6%G@khHB2tTdyy`F@J z`4Z%Dx6b_>u7Y&ot=_G0*Sa*VQRxvnbITj>{N%L}zoLS*W;N{k$)$K0W3CZ^0GUV{ z8(Kf4Mb2PLh^k^fUSq^Fm^aCpcMV)9mG-WQPY+j4pla`Z^pX_~_8V>`b<2^O7t9g+ zYi>-r4r-fgEsYZgmk#*kmybS1$Di%IVM}hf1}h;$t{}8GUdX>llY&!TL36LbKDSZf zBT@JR8-)Bd7L1vSo@DXKB=8YKOo{OJ0~-CEukusSfw%tqH4VLh;oW&cIr;^Tw(-l1 zK-JTO70r;)92M+0hIMR)TbIcLV2yKaO)dPGQT9|9l)kmj(K9SG55p3_DTS=(;+lum zYbtN$;0!k{u?H9SfL%2{xhPlNq@*1VyO2R4jbv1YXGPPsKKcZDNF~jK&{Xb0IErVU zn+K>2Xh1*vRtoJR%MRwJ{bWU+{>>Zp30h9U8UxM##60#?2@)k^P45kIVLAu`W6!V* zE%93;4LRAgJM3cd%4p#Wi%?35b|&YUs*YXWZ)HV;dy~tN>B+xL zdRzulMsOA>eWRz@_u-76D_?57i+!bWaL;Kw*38vKlehKT_en*x0wczJO-Y+tWbwG3 z<1Ylht(a(WyYjj&<2&>9vUVASZCTsxD$TU>E}c{VHn?tz<_uTA*6ZI>6PC$s5(H@_ zbf~$Ayu#b-n@v`6=GKqpH94q`HoI2RPgVqByNU<-Uic~XshtJ>Tt z7gDE7Kyw8v#G#>6#Xhgq_{=fzvhxl`o2-04R7v?l8!lf$5e&K9sT1Bev$a`s+SPQD!WhsXnyAp1!l- zQ!HyAtRgQ{zQs`EVXc3uUL9uHPcjb)X{ph-PujTcjJUk3Oe1;M*Hi6&;3rHr$_)6W z-ZNc=5}jIduf8zc%s)9UBq!vU{g!C({$kF}sh5f=ndlhr>Atk7=kuAYYD}I{??EGOp@uB> zN43XBhb~_%d=neH1K%`R+;TZYmXrm%+0$(-0CUQ?6-sys7*nybtJY)6egQe<6%*MY zAU4&Lwn6I!GYM7L;0F-Q+=CK>+|R!GziUepFRmksxd^U33Bh8-m}Qb0egaP`_ziq> zAl?ZF=Me~9r@X*%W6b=5w$I+72yxvio*vIcsj)kwc|;mX&G3gC1paJ|IL2s%zkZ#} zjESvz>a?AwVzMH-`WiMMR1z42U&!L7&+Zq-Om#l6%8c!_uoUWJQ5Cr8cAvWJ1wJ8g zZC)kjb}li{&0>JySLyjE>bh69)l^G!^CueK54u(nGH+pXtj*5~0?TXV5 zKjw1}Hn1`e7Di_Rz$01Bzre5Rh?Cd$INj|>((`L0t|gAXK?GCDT}&S*l`1xfY}}k9 zp4Py4#JhwrUg3gLC{3VB@-IoQMp18Jzys@0+4q`P4n6VlW)8C~9j3#W2?``?@1axS zjD$oCXs*>t6?Nzfb86`~yfoJ+)>^V|24wXrLF@9&jAqtOYeM_SED{!9{aMg-M_-}| z8$K1rIKH(y9-~iIBw2UdoZggI+3F_ne!<{i4dTiwY-%m0xq)qLT}>8h3-qpvCXJiA z{nb*nN53$|O;E9XJi9Uz&->GIXXNEen&zGe@FapX#jo$Am8_y4ARDNpPl%9@wpw|y zPfU3~MNd_-_dvz?eUf|Y5{G4&$*Ia@QxT*ynHt|AJ_X*_cgZIUf;T($u9CXY15zF6 zwc3u;L3#_>^ukWDp3-kWpzUTGZt6t)iN(9xuE^}yF)7SPrH_8^uKJH*>2MIYP6{R7`M z^k6 zI*8gfHx4_{F3hCOHz!z#pgm7t5G^tQ1OLVhVnRE^+7qPMh6aKDhmF*mTOc>b$0)$q zz&nbMHsLrD)FR2~>GRsFa%h$H7agxNf@>{%{4qiI#nnqzA7a{WMfs8LHXyQv&U1dyEqNOu`mrM zV8IEp&P2y@S9p^E*`jqqazC|v!X={6LFVl-fr^s|=;X1P)u3i5<#DhdZtHQ&RlnUj zAXMFu^dSSCRJ`CHr-iBI%u^)%h<0bta6eBeIwKwob5A=r=c`TF?F+;rmT>{b-t)s!K9D2t&IHpyAM5~tR1o9Z*j&!rV9 z=vhiK9tf*@Qtjc03JAmuH%5(tPtIz3Zd6i|`$5gWLP)5OEjZ*J zpQS;0oL4euPA9in-A<>`oOeHvZ(XOTpijt^sp}DYqru{w3p)2pXJko}dOJ_h2&ylN zDoO`rJf-3j9Lbp&!43HYOOpIu{PY8K9cThtzOkERX>BxcUwUqsd@BN_nSa;zuI41u zjoj_9{6pyOSf}*ejPYZOO2aGi30w`bm}(J?;~ohAKO3JJ|h+ofQ0cG zAGNj|DtE1add{D&&~TYaY2t{rjdDCdU;b(>y@p@#BHh}EYZw;HLv+jQ1UYuvotPd= zN>_K^s@umzA^C)ps2+@Kcg$`yj4tn8!qPlc$nJ0KSRjt#$G=Bb~^ z0ADLSXMX76cED6bq+F#bgI&{^hO(1Az2SsGxe}a_>c5}RAvhldn+&j7bOf0WfIdlq^>v(? z*>kBuKjNo4x4U@dL}QSe4ar^0BoBkbP6%u{6&hTu`f`^WZa#f}H_^hhUtgl%1O+#r z%k>fGE}eW{u)7FW{GLT1gxmj<$DD1(VCVR1DmMLu{)NyL2A zQBZt28BE?>;q;DyFRHlsm489x@8{V|`NX{L;>EPDnz65!8p_0lfu z7QwS8i3;a5Q>0#-v<7D66?@$S6klOz-LIPj14O1 zU{76`X%XZYrl)VxW03N$3BBe?nC>3wZR{_(UGLLHA@jiX;=gAx)O}4AM(^brH86+O z{Bi*$eka?bk->DwcmBp0iKshl-riwD$rrl*Nggp$MFbr8W6act!sWf~06xUZiPS_2 z3Sz&k=U+qr4cz+v6SzqRYLY_NsT1`~W?v)&)IiYKk!^4s*7*0G!a1O$q`AmtXIwkh z8H9CD&6hL6cZ5HwESw;WIn16K=yz7qdj%{X)Dm7ZrP#ws27w?S}c@&pWJVr=u)f-5y zom)|CD#9+W!+TOPSk+K?I@KxP4%dDkT=M$pM9NFcMzO?ZB>okqS#3)m~mwdd!~a(&xCJ%8re zpG_E!vZm^0Y(}^@QI7Ts5;to$tp5y2wBw2gUX9S1J8e?O#=YH$Nc;ua7nEPYP`^x@ zDxssrue~UDYV+^2ug#((8^QK$XHg`RpSzgKq5;=VKGomG)$^?FS|chX*3s+PUh#fl)UzLx)kd9FXQy-QwpmkK*477 z72cqf2MD5|ks?FRjtzn^X!AOQAW=b)H2cZ@z}yJ(dP03J_S7ih`&vV!N!Eq5$r7Jp zpwaQBA>mN^%wKN1!_o0m;|k3C5xK4^F!B|4a13=s_WW7gr3s%Ek5 zZF!R)4`7=~hGL7AYt|4^Slh!X4&Ja9i{Sv?L+2Aa!J#fNfsk1z_&rA)E)0-@D&WB| z3mdkArf+H(Alf*u`FuwP&QQI@Dx0p~)~P&DyG%b{vbVvp4@LQ@gk3U1w>^#81O9%{ zLzNZ-U5Ds?^u9N=h|IhlS-$5aetl!aC2o4&;(Z&%KbQUNg<#=GmE+^zK>zK^2Uz~| z&v)~_UKk1L$C;{%9~Cm8G@_jSY}Kxk zo}mCXH7Z!?z)+&%I%mFC%G1e@Qbag0)i*)Ag+{rh1o~{#=fzD-k%-Tf`DU{xDAH1^ zT7+meD*Co?;3yOQzYKwu8 z5V)~hm>T?0pi1db+PGY7k|(?yS~;^&GwbQSojUb3HuIr!@}MJqz(a1wb!TtVKx91X zQ1!*<5HSyO7OOvbGUPwqElLk~S$QK28UD2{filQJCo`Vg;?^%BV@b|}3*c&J1SDz+ zgPt`l`Xd#`TW~4!?{P4c0ZI7kQ@pgP#^sL;w*h9iJV}r*u6@!rC3=zd?EtIX$wxjy zx`HFg!lkVKfc%v|@ES{}X~eb@OyH}-TLoO81Mb0hJCaRlW``S8B{3} zc_9NLsh6n6$Qu0jXVuM9hJ`{CTR0RI5Z8+cDf!j{9YnmL>q9lY>F9B4$O@0`=maggRj&#ur z{+v$X;j4zmv3Q@?fra>&&h*ePUNR^`c=YTgyrN0R%Wp$QD>aGLRGjYO^84R?aM7C|*$X}{W%`+Fwr)_UWO$>1%8-#A%n#FnJh zZDxnxIs&^Ef3ETA%UiDTUu`M-5~QSgpHDTXb=O*Jsx8VJm)ts(Ua_P`7t)X?_c86* zCQX+c2iW~WgM{4;?XtT$3lty?f|tv!b~kYE3tlpv-Cm_@1X_2ruVekFSV7xWoAllA z-z*5olbbQ!OlURXLK46~dYykexg-Z2pdU>i8!H-I$M41o>}j}#`p4G2SDd#^*oWYM zq5zFNTh1AFl$)?0;XRoiOik)vf+?DRWEMu@TSv6%KA_zz-5i+lLBa^rT z(Cl1ydJRs^nosSX@6P0+h(9SV*0&$eQ=~%fHN2@;tuu75raTOAyP#2CfaYp?`UkS4 z>So%K%F5o(pShjUAXGUSan-b_q&b=$Wj+|bcnfmdg)ZZkX((8TfY?ualFNN+0p>p2 zh|@qE>5cMXiwPIQxPheV3qOqMHB3?HRADmDb9Jmohy0rmyA~m{AT|ml7$t+!sBn9u ztKgxa5>u)T1%`#6vL;5XxeP@%r*xP49m0%xPc!tFmsG|tB9$O?W!6LB==OaA2xh<%mLBS^_ zo3KOZ37L{`&$UuDsI*k+aBeRF=Ht(8$8IH01%98sd1iAm+&!=A9y!ULiagGyd2fEy zuOB{PV2gqhZ`DbPlOgWP%Khb1Ug9PgCd!S4d26we$|oQ~#{6!#WhJ{il2H zw0_ppq#DU_OmFxYc&%Q{q&uOim8qrFq@J~bO@bOuLP}yqp23q|1(e2GYbMbMr2;b~ zH7`5Cu0b*{A8*nY{)-s;q(Eb^J6eXP82^1yfnmJjY9bdF-r|J7>^{yQO0|ewaoh>6YH4wv-nh>rvecL zOb+~nXo-K}bM!7rU&W)j@v>I?=4ACwSwAE^@||1Nj3<9dFE2-aRCAtwEJhBB zow9(QUae6a=|IFAOc+(%Kc>+C|6&Tc7U?kQOHxD(7$IDCYRxbh9h>en*#h&Um>kwl289mw+n1Q3=-p@I~(KF(OBrK== zYj`6Fmy1M2Z99^=KvwKWh@8)6R>*BhsQ|OL^nP@0-1G-+)>L*7w7lR-YlOxo?kP@@ za;^mk>m*M5M~Y1pGW1AFtLe3<`TL!b{R3+c(b6lGiXB6@>)D;V3csF|i_Zn7Tt}ZP zwe@@*RJ%hC#!5YomlVg^n=+F6uU@#AkvYFBE3gA$DaAUKWk)-kaJ3l%cP$iH$o!bK zF{!j)MGI7*`(wG!K$nNAnXStns-Y10$LNRn)7wmloS>{e`tWqyyr`i{zBO#hWZDaX?64(|WJ9mk zJ>DC*yDB8ZSD+CiyB_59sa2@P0<8qC^H*kB@?t_*p^I6kyOHO^TgoS3%O1vAcJD*2 z>$>}Er7xqJhTdZrX_#mpfEcFvC$N=8;X`qfqAwmDj4+@Gp|Rk_BEFWrn@>_yBI%sB z)F^66 z^2oeOHMvP`PW|cy%vA9e)11}vx)SDh;zpP10k>P5zOcO~sOlC=i56q+~cAMak0L502)YFEIQ_-;o-qiX8zuf-H@S%_lmQCdM zrA=-8=cNgQP5^Fu7D(X*IK_Qdeth!L2dJ2!d>D&Y*N7_j`wa)@XOu50fLfIdgXQ;p zU*D_NXK0rds60l{>UXTEBt`X&8$=E7U%e-}0+PSNos&*1E~Vm`v*at1_dqLeqJKHeR^Q(%iY`YfqpQ=go{#on%8OD z^z+LukBxNPtWiS9R#bZ@%cPNK#FMkc$+pPJ22*kNXDeY3b7l#rf?qcnk`)yc+E(#` ze|(vUUK|#y`nJJ2`fhWP}u^}Z%Qyq>41)7*ALAOR(^{-wZ)nhP;QP!lx) zebyMV4IAuR$QvI@Gp;6?l{j38m9jtEV_@=AK>v_4yM?A}_l@%0mQ*#V9ZccHL!L5J z0A%`FK8i={)!PUqIvUVh_cq>Xp*Hs80YS&igm6;kq54q<$WKDv#+D&IF*ff4qFBEk zgf}0vgdAAvb8jZF17)fP!(O-jhI#y+Hr!Yg=2(=m8l7Wml@{He^fspOAxO%wbC3_=R?N$hqcvXayZzx@lb^Tla2D z!$;8mhDl2Qt!ZdtORXKAv+qFpTtnmhgLflM)gX?2P+)|em?hXHkW%f{hUs-L2KOiy zL{?a$tC$c8=M_P?WwrqznjrZET=#l#7ua^PyiNESkhc}TGemik_m?+pCu^Jx(x#vza_Vu_dBJZ?!KfCQCNb-`G#r5(^Ey%3NZE zTQdtX<{_;=Z)@0*$Y%^mtGb|ks*;LTS!QFLJ$C%9WIG;8!+CKz;W|i%5apsoXho2v z2~3F45Tp+O#!`^XKT-u=3eHF#y*xdY$^VW{_wf-sEe-xxz*Or5V|pF$y#@G1ehw71 zl_`r%eb=$T_STENbT|5TjwWEND+V>J{PC!2#S8RCEj32RY_mE~bFRv(?%P3@K!Sd8#}~5r={z{h85B7z&gGNX%P)5%}9h&i}|~a zOyA}U7w@fai9EEUgLB6PP@05Fzv zJKVt*LZZQ98w|+S;FC}9W$JE+$L2PSq3Zw$mn`S?%TGEVT0goz!J&PK3#YUF=6) zN+7MV932wW=Q#0*6=4ur($V61O1ziy)-X8l1e5SXPUxxFq-2D7n)EG)Eya1gDhg0M zZ{@?Gdns&YTx}LRBP*AzD1{6>8c(aJiZ3UGhu`v~3ge6fuq&rpJ6`-!oe>P5;;4*I zhWlMx54HWz+B%RE(1o2M2M(m#jc2U50%F{R27iS>B69kzY{~FN(!Q5mHq+RQ4x~`f zrt6OgfWdQs^U2LibCgRS%T~gcNOTZPcA2)ylZ=r_1V&x4T;;?{6(+JE9tSThG$mpKG zu0W0wDu-m!n<;X-F3vqHZ)7w<6 zsao@>m(DZp$3KN_Jhq=>DY8StD4$g{;&@v`w1`;tw7A3CPg29t-4dYQDl~{U5V8S^ zh=iHv34;^wvdysiP-9IyLJrG;TIyBn&J-OW6LQTUryrSft7NcO)6m2sl@7L$LZ=i&65X$*}x?A zA6q5CpB#1|9MSe6X{*9CY$T7@|4iuf;0fc?9kDKQKieJ~eZ^hO;J1HjTb{LnoD3*o zA_YrQgP3WDO5Jwm>s_p$>Fw!CnVS&noa9h+ox2~TGwy%?l>$2#TsPRj#R3(#4(DCi z7`y2b``VW<$yP2JZqUVR-kTFpBJ}1U|8QNR!wW@9OTd)K4qtrcgCHSgMl@f?qGx5LgN9;?4$VXR< zs^u4`DNk-SdA^SrG&Gu2*Age?69qN}#cmTcbDkblg8}!vFXQU4ee2a~# zVxTA^wM9wIminWJZI{XF@Z%>C8+!QFYk z<>6l6b7kic0C{TX3FMHlwAXEb24h-)nMnWX_{7!#c=@9xTs{(RD$gj3(C`s)Wq$aK=F1zsPWlSc^sDyrlPl#6b1PAZ;jg=BMf`fri#5j2EU~PIo*=b zpS^|+Z2c-lCfNXwCDBcmaC33a?CQa-)O_c?j=V`{$>_JXS1h~TVu0xI2Cm;4*PA`n zxi6uF3@=sJ|AJ2XQ%KC*{LB$8Bht_tF%GpSyoSlqJg&sgyp3w(Hg&oXMi`nt0&e#+P79YkH+t^cy$h0&W<1(EFc)3p7 zgOGB1eVSD!bxA+r807>HQv+*2v2iHsQyr6Qhq&a-f4smwn{3zP?priP)x2K4Ug!CB zJVc|n<4&Up7ua$#z~46z?-KLoW%dDIfj&K6Q zv!=v8th(Fi=hdlU6*#J|>{TeXX0iq&1IVXQtHBYf=>Ef+nJ#iR&phdRTl&ZBG4A}i zWpHz^LiyG0WkNLqyv2{`P*=ijtVRwwJaS%t`q`%A-Az!Jc`y_d6lWR^odyY|c9sVX zq68i|7?SFG=4%4h1WR4&6sSv)UfLPsh@=z|8>@_@KnsT zmQ>zFXTIskn+VHqM(=s{+uJ_v`G54bWWFecXsryXBG)2^Z8i?~G-@<)h|(NlEb1Cc z%&*x~vlu&jf!R$)y+$cK3WvaOF4|Hnw`yu{qp`GG_(O$bH4zT+x0&|s2!OO$=z24U zya3^4U&oY)sB1dkg7y2wXhjkilFN7o&E@xL1JSN~zG^QI7hn_eGTv_=b(4n3pRq6R zfS@!3)v}oFESNv`J{eXrq-KSc90+MQ_8Uo=WRx&DP`|c8n5WTH`sX=1^$UjD>=}RpnDX}4qPj@!7pux z0nrt;??;zjHhJWf1^d7KwCXcIoxSDQJ>h|juQRpH#5*ogzKr%NRLv4`lNp__0(WN+ z-ofD`ArOweHhjNl0mnyN4OdB&+b5oLbpt#Y-r^%^_u*QWh;)_+*E zS?+B!+6-kR$$h9YR3j_Hw{HoNKI5wkM$0TF124qOxNa$dq8goI>cngig z5zmL1dUki1AD|%{@5i%!;)c|%#zk^sj#$gFA(2-sQl#*ItK0sKh67v4;7<$nXWy%< zxD}s_XTDsx*4tow61%B??&IvTW2t1|>QaEVmtD$WvNSy-O!x8@$#C55U8EF}i)GGe zP-=Oie)=HIcflk385N9|mU|y{i~)(RT5+^g7nto@e(4DG{RXgeVA~^6{}|^q8&hKI)fx z3M>zDL_c(hd@7(ifwHS_NH+7UVSmI)IGN*bx?L#R8@^Cfmqy@(*(vA59~P-49}z_TDrk-F+K~17>4ap`CEHejR^t zlFFx0kWu&DzZI>FZjMB%4X9`a;#OO{{yfDoLm8|`{YQ zmaVV#6q{ju7#(;@s=}-OhMwCe9^htVLm^zBT zW?5Q9U-v&|`TT#(5~dRjY8Jt?i=k+Jm3c3s-fZcD+(mZ9yYD)zGFP>D-S`T@vyWA3$M@GOa1Tn?c_=rv(0frT zBz5UlR;(BmIE5nH?bF}_#2iC83UJEL!#i-$MphmHKu}r)8pp_h)t*PvXEpPkZUe!YssKT-xo6tRr}J7OKkP>7C~vpHTib)Z%h+5P zj%t!T%3;|gA?)p(i>M}jU-LXY??X4F%?X1M#l#;;(()|xiGu=TRBYXfku^`oeLs5O zXz$q$!XYK-Pv}cQv4>7}B&8#_AzdH}F zpZ;GGxH*hJ4q-d;K*lz;w{t z(C|Z3^QiuuAXbD(iK_}0hW`W+p~~)=jU9U9No`lyyby+BoXm>Gi;IWgxP1LtZ)t{8 zSZ~K%3w{4L9r3?@pd_pJZ5O&RhggQcKS{*;3BErrN#NU%zVL9hz;|&UEL=|MN+0yg zw5p-_7{vUrUb=TYct4lqxP1sl&fw7=Q>u_qt5lrT?DvQr(R3I*RZa2~88I&2Ixbg_ zQkTCas@Bwn8<3Czi&d@D;csOZ=AtJd!>`&U_DvP?;%jKJ{noPr?<=YJu{mb7Ow_!2 z+k}hVkSvV500SU?v?Xl03=JH*o`lejPm>faciHGIM4r`mx8UGQBBaduGj*Sa*n)P(ZA_O^EN%);x4 z%N3$uCBi$>urT9GOv57~LFFvoK(S>h#ld*9AoI;t$)2K#)4TW@(nxEe&r_Xo1f7c! zT>r}%zyHM)-49V)D&wvgDQn1g;mcC)>@f6tYtezpIhe6?pt+h_g;ZOvTSgzMYin1k z{GcvLi;YqF^mw39dws=MV109N`^P~-HjM)b9i`(ocSbPKw)7D-4E|~N`xWZvpY0lQ zMZLH%f}1P=)i#AwPHT&?D;Rm@YY)oKyKhFk_d9bUIOsz{>0a)h?^Xz~GTiG79Q70( z>dZ&dOHlD~HiK6whwrfA24__J2Sgq}yYO$ogCMHJ|1bOQ5dGsN-tkzo*fs$M&TTnX zfo@4&{%))g&G)jSKaj>PO!j6}^^>t!L;FG+(?&i6w?{Er6r=gi!6%Bbuol!>& z`F7n>jCqy4$D3sLhGN}0oM$-Sll9kNi|h`D`MT*@hwBs%8?g`vG;Qv|(?bQmwqoAz zP4~c|pvnL!N5wTalWbbZU$tuM5c;Ni16i;2WrHH@&GQAOeM41;h_ms$aP0=?`$kpa zwSI#<{l&l@@&R*F1-<+Jv%$qyRlD&#K5<`*1srV9YSo*P*U^MT=CIU`Ky*eay#>3eVJ@QlQVk+3&r++<+Qosud)qfpPENp)xqf|B6>e<5k zosu*-yk8>sCRQB<3Ft||@VDEVEfj2f#v#NwDrte)K|^WWcJqMb_1K!C6E*3bO%&(R zGJS~WR$an#Xg&v&C#_LdK}TL#d;Z+@_Smx`ZL(G69~))qIy{`)XILLcgoSo|%JAckn>m*X0U+JkZ}%6e4WQyMHn@(j zpoG^NF3&2B5*2SW(QR<(N6xQ0XXOU~cFw=Js#2Wv6Zp;qwW!G@O1NB%Vu`Y5jvXzAwn}{j3Yd zI9wgfQqx5P_^O|4U0rua*DNCIUNVt*ma9Ikz*?z&Pf+oThUgQ=Q|LMXJK{fO&3!ihqj}X0N;PO7^uz z^|cn({5>5WdBw;NB)FR?1>0Zhn=jC$1w=g5Svkv?sRw#A38SYPuV<@eh{C5FaTQUg z2OG{N%Uq0JS@QrY;oI+`*WsFXf$w{qqWR<|Y!8Avs!+Y*I6t(u0Qs7D4T^dfnCnB1 zuGkl5*yEZ$pd!4B#^ znA_|V(NA+KK7m*rE%X{t>369<{{zkOh=6mYXtJnf_a~Ro!(?WjZ+vx-!ZK7nfiz;V zt8R{OFmh=(euXTihbf!@p(zofH9bZ=31bw$*B=bO9KN}`WA*T_!4gU&JMTd3lskCF zmI@yCpWPkP$L@@%g(W(N#K)@V3s}TH!FZ}3x^93qjY|lH>W^xKKnHvc($1Pp2DnZ3 z;&fkQZ6mH8lij2n`5=U)-TTR+EwjFVU-ZW5F#mNEOwqNN5%#~>`|;&S(m&lyF4AK+ z-Iev5k+oOD0zrU;fDy)}S{ec?bvvP$Slylrz&ZZlW~0ifJMZT;<&w+xj=49eJlCtIExX4gY&8PVxI?L0xJxiLIYe8({lNXxo8 zc|s*Nh5v@MK@T{V zm#wupmT;)`NDrZ*(05azT^DvZ&f>Wn;L{}&u-_mZ-Toxh%}>k^5$rGX1~-PheGoH> z`J3{rQ|UX0G5!H$Vn%NtyC(nZUs}Qlx+!7r7kvevGRU%u(vt@+jiMa}zi~#^;^*i<; zn7^`WOjaZ%P89K#`*Zg!NAROoxF|7;hO$LK2hn)0T?tU&rd$*W~>$(x9_`DD!t;WKYe7#yqJ|($7<%>+?pTva00a_^)o7CgK<7? zg4$u8X+CznpdPERXgT9Kfo|vT<4eLzgsnYxF-@-kgr1D z!0btt?8eNw0nGbikKKEt&R!+0i<;_-t0fO4)2zhvxt}~XpNf9ye-ksx#`v4PXKTu@ z{6jNIN`FD5Vjiue)1%VR8Gdw%WG4wA_2C#$Y#orA=lCcJiI|b3+qD}aDkxpPaX$SL zIo0{v+&H!L_0#BqWj+LIyE@m&TQ zNmc_)0f1s;`@kQ6zAK+apf1_;Fx6KPqjf@~V|jfM#{!q~nEw- z(Le0zUd&H}mh6=8j5hmrIG=81Rz&(uVuT7JA{sk-F4Gj?J03JO~`k4WVzp_WTXtB67pE^KC@UX5yH3W zC=J@;W0CI>^5gRRSRecQaX!P|hhw&7JugVRIx<=N^wQZL0=uK} zFuuK`bc4v={v=$w^|}F3tS>kQ8&I#TtminLg;`f>CO`d*SP?*v$WZzbg2I8x=4Aa2`{yeJBx7jp)#EaFt-=yFs`c6@V z-ESF%92A@=xFX!z7g*qtt$0WZVo*AR&ab_wSh)!qnqJGwd^wbG&4jgDCN$Lf`m0ac_2D(v zQ%Q`Gb@gE3y93iX^FF|!768YkktSAkBkXFn{&mzKisFoeLaF(&ig@CG;I-qgcn4X{7 ze~`O!H=XK{5}%Hn>3MTuJSQ3AU>GeQi7cJh(RjrXQX(FaP~$#K{vM;bQjLxl-h{s< zjtq}SWx4CBgF#2G4;?E9nDoFv_TXU1+`P*5o{^89B=AIQ!^<>|v{_CLAm{&i;!$rV z4)#Bf4XBPv|4njEcW4nFvrSa^uQMb&L#=$B+A^O9p9jB2{Wg6M($A?S$_xdx=%#|~ z>^eR6UN7%cTf7)-UzTRo4+#1odxjFfK8CUEGQ1m9dZ^_Rc<<|CCm4?@j>bSrm4lmv z@-U&#lS`$mp zx6CWw$P=o*f$XuMGU33$a9B;P)J}%rvXi!$J7(EIr8oM78({9XSHvhq?4H0W7I+)F#OtggM4vnJvqSm{b=~@NNh9u;$Je#gz_R8@R2I zpr%S~Ei=x5znxN^_kvAe=C-1yqq;5b$1kuq%9_5MU_0C<#QZ_Q3W%`TxFcVqwI5?L z8Q{ZO|A8Ww+wH$nV!%f7|I%lLI2v=m$zcFJ1FZx-oH`XD3nRvUd6c2Ef<@P#_cv1c z)39yBYDwbeFmrqy0@FN2h}Mv$hv70|D5j)1=t|={BTD3J`D{H)S}Q(Bn-|&H))_4# z;7O))`a3S5;Rx5)TOQIO=LkF~RM@))J#*4vktwSEXd)c(VNzu&Kz(d<7$d^SOn_)l}v7TePtRHLld}X`okruj zqK@MU+1ZW}yG?2J6uZ1XY)L&7A*A9OFcQQ1VkW?av)vRxvr zvgW>o(Ukq4c=5Mp(fAh&u^&{$JO8^>8t%WKKGt)Ox(T?)*yn!EAg8^Lr~z={5_&^B zOxsg!_FCok5+Jh1`rnn8vi+bg|N{CIRGyoZIuSNzeAjypg{&XtHRGw(P40O&~1`!*h|T&@4l|J+m5 zx2qo{g?P|h6>C|2?qZ+Mj)dzrj1ylV%Hvew{Sf^Hs9cWRE(OM^+h>;ddR`^i*%lSC zr8!d7osODx)C+m)9_}q?$LeA@0P@~J68h97NhE#uRiP^6S3X~OjMigA)f+CqdUbxL z`gg*Ao{b3n=wCMpA};*j9u?d_i?Y8R3wU%qMm_LUQXIs``P)kWw+npqLb=%gxS;}R z(O8VXZ!;SI?A9dmS-t8D_{$K2UWmwhC*DfH^2!cULN_PrK2?&C8cS>)%Ta*w< zY1lM^ASs(p36TcrMx?vD&TQ1@d9QQ+@AaOK=fn9Z!u)2|tXZ?>UTfV;BvJndV{siG z%`PCgS)$dI%+$Zyi$^&E<(EII71PdpY<-MnD{N==^z9v{#=jT`$p&4xpy9R@p2=O(x7#^ChC5+=pY8++3rtZAL4R)|8lsqi{~5Hy zfROIfeMU9-@~;F578RFi9>tpCcMG(%;AW(_@o84vrKGjwNm6a8Q)q!#GTbd1qKXhO z`qlJRF7z>A6UxKpXQw)P()kSa(Y=LaZYwok!Q5E*Ck!m9B4hQ`(Vam!R@zS$ypjK? z57zBJWPpHO={!#k#R{d!j!?@$rai9*0_r0g#7nJu|)dK9ANP&pahX3lf5+C0q1DSahn`yb1yFiLxFNJJdu*g%Uo=9khk4 zQTIGAA+!9Pcx89Ew|6shgQ`EYb!)-nSb>-)R7feLdXdcu)$nng8ex>$AN&#E7R`yh8#rf!n^<=a;%H zY2HVgvn{R5!A3;l6M$BL*5_cCz#fAWntSmUXgn3G*J}wzSd2I{JXFYLir;C+;H1E+RQ6}jNTZ6PzB%p zpIH}U#F5>@>iUl!k%_$W)S(h=tOj)mAQ14hAsW5+$-Xi^Q@29kKG=saxs0u-N1ZG4 zqv)g!SF4nLu!FPp;`6(~&?Y}j5`#G_QqYpUp|@nOM?Tz)2QWkO7CeKVchC|Q&Vd@e zH-`J6PU<(?FL<|Iyx0ms7^M8X9s6A;bTc8WcfZL!==vZ+rwh@}JOvuI1f2 znB^-8V&AO9v^^vFxH>Z)9N~+d4oVkTa@8%*2E6sONu5HiumpPc)k%aLzaBIb0hx{1M(c5Z zPbCE#`JclNBHIW;=U*-ZuQ_{pjlVu8RC7G1U^R8rhzwU21{tXEe8>66bu|FqyLqhr zgr5mhqdGvAm~-3N`_?aOvsggYBfb9W7iZ7KP-_?G$vfq!ah(cCq97acM{iIHpXB%E zDb+v-H(`VrV$MwTAeM!^EB2|hDeO~`isp@^vG2A&E4B(01d@g=)G+=z3Lsr zr7I(a+J$u-#+2NgP`6Y(qlX`W1xC@i&xWFQw6tg;^q8*RlmN|gOk5C9Kr%{3U<8w! z$=Y?k+X{TS^sUHkimM1TL(Hi#?MVWhI9%SsJVBt&yE>`@R4SsDu;q0sUc4$q5`%eO z1BOhK?1B=C!LJZXJ2w;=ga<3Bo?A_yB@6`9=h;iNdHl;cUJ|)XyQbw-b{&efw9+<7 zDE@UBMF`>QK!ep8wTkY~TC&%PW$TsS?a4Wm=hdjrQ|ZdGJ)VnXNxXcINxy-l`5;c) zX0A{-?_~E1d)xf!9u8|`k*#%`^pI+Ag5@BGhq=#PwD{^qw&%;rD z-%(vx>+#ZWHrZw*0G;1i}-+ zfES97phQ_?mFM6tEq`WRxOqN#k*shH>W2~WojfHXC~+jS95qSbApV1HUGq}F&^fF%&MfL#ztfDlxD=zY$Zv}OwgYUY#W0iDJLWsz zgE!FO^m(dZLlyXp=(u>o&gj3m2lYaXta$;6dHwB<$DbmzXxT@GJLLOI0Nv~fC{_Dn zWY|JqS#y1LUbt_d@z$;^qh%si`#CjscTUP8#?pcO!F(wME2!hgF`sc8R`lm&Kh@*d zj^l&J;?{rRKGCmKY@(MvtvlT`T4AxHCU*pOMDC0~?nx6C9@@24A_%Jqqwk@|9%JZQ%lP9|OG@lF^T6Fd|ApjyY9$~YB5-XIj38JXEFXDtKL7jmy zoR9J&a$|=JaF2LYgf>cAL=96|H}T~Kn(lu2aU4MNy3O?h6epHFW5)NMCx2&s5C1h* zBch>2VeIc}K!F>>B0=-t8WugkbAezcc3y~ad5;-`2eDQ=kvzJzd4xQ7e&GlI{xdpR0QtEz6AOgWy#0qa2sjmDU zXXODXTKV)|aIqu~EV%KQ{cqJS(meOG`L3TDyK<##tWPncGqvwTk zptx?}z3h_9lUI@7p|3?L8e7wRuiQPq)^$6FXS!W$_AL}zi}(N;KK{?cN`tq(Huh|5199`1!3-|i}8|zB_jBVG= zVdER;Qn}D~oquP0KD)Sr zX{ery%iA;bl$dgk_2^JDp7TLffxcg!Z5{UnX`@D9>M9~{-&a+jaCOpQe;7ZAibho> z7<=JXjCHOX@G_2xzDoj~>kG&aRWy-hM8>!J^~knRRIJUjndsR@qU<`~E4RQdL~RWY z;wQTutDO*u`8U>lxgkMGwXezYM!X(gOaXOJFV?eube3lNOpEj-mzQ{$zb|->SOiFu z#7f3XNb;~jciI&hQ5HisGSKd4rnfrJ@;2yrWn`+(FRPx-Qr{9Mym34IDI$aQM%iZn zTR-EX&y12c`?XAo=J{@yyo%MYTOzw+2_jMt7Vj&ga3z*g&WrKICd%?S5cj>HCAnJZ zd2}(tBPK~O?Gc_J>xHrLfUvoHVpnkEUB%^O#vScNYnJ3{V{G&`lSLupyJZOlv#ML` z;-?fG(3x;Va#L<2@vn2OGSO*R6_U}&$aVBMI6C;Gx#&KNVFW%Q4Q4!rj)8532vv5A zB@{#)tKBq=y*1kE1d8>>0VzuFTW-Ioih_T9}f%jW85F8NDsb=#0YHvh(Y++7qs1gbRySRCMq?#C}{@N0Bs<= z-T18toy*I12}X!4hBd@CnBBAhvR-)RGLQ(A2}6 zAm9cJk$TdEujgOSyvDn&p+)g^Jpz9Q82vQGZ!wyvgSUDGrG%&e-JeV~wj_ z`TMB@PXI@R=X(MIOK_|^`pnMfx^0I`ST6PswGW=v*6E4AfCZ8uTHFVE@Hmgv+`k}Uv=(p8D1BD{y!oYSujgczz2l-~1M*TY z%AOk2Hak(q-6Uhl(m)@^s!FaiV-H9_sGH~{GO{h(IB0%V{*|77)e#kcZm{0pHDA5zAXesN96y$@OgyqE z7}am*JiOLfpVVm?Xn7ZgE{S5kNP2-D`)-pW(_ue4x*ZZ1&wYlXYPot9|L#lXJ5Yrk zw`@YKuIHZ3Oxf4S9-*@1c>KeukXfJ0%5y~1Ow+x6tngdNL&?gjf>%85JQsBaarD6x z4pW82^T%Fi%l<#cWfg6LFMB(R!+b&t0-{Rj`^RcG(zu>VTO!b>G8nB(Y zi*5vQdKR|aas-t|nilAA&UFSC>#s9lz32nCSgYH6O%< zZ>C9^yS^vQj~IVC$Ace|lhA2?KaRJi7~DVlJ71;IKac#_#^p}K!f#G~5L&C&FLp92 zY80rAeKAB{gE7(2^TdG#)^V2NR*~Ja74;0V-g<%U9ndL&^R`^N&+Se3yynuYmUKO_ z{a5ToIXt;3Z!8@=8j2mV3vx{Aj3Q?9V`)v%Y7O_;iu*f@nOwstS&vsEPPi(26bk1$ zGTx-+nW1U7v1a8VsnRHwPHQ@N{ov=_!;p53+ylfXujqWS^L{Qe ztzQSa^TSpbn;s7Pq?`svFGGz|sGzF*v%H=5_riz*U)lsL!7GPNZr{HN$~lOTGu9X3 zMFQ&`i<=XcG!VCTi z5vidsvlFS~3vLe?k7Kih*!#zYT&OJb=B+tQ&ZDe7S(G7)*H;7DI4FR%;ajkUkwe)@ zy-zD04!Tu>IO^g_QbD?^v2%0byKyg2W$qQ->V*LdPWe>yBqV!Dok7LMfCtqJZ_Qt^ z`u8hV#u1LK4O2wOQ+~d8K+P)c85_@uoFD4((@P@%!6xgQMRucRPbZH)>51lT+%4|4 zimd^zn$o@{@|yR{nhJajlHa|DO)ZhbidUz9ftr>PdS|^4Go>aW!9C)TZ(U7{8e428 zBVVFmFdrtz?fPRPX|^S`)C6}xK2yL)q-DOCDC@K>y(1>;W@knE$Z~vVp`Qmh2ce(R z!V%xitaLH=J?Z+m1~7n>kp}5kolj}eiCR`kA753BRSC_T5MY4ncB>tBy^C~;?MB|E zhZ~Ld*yk@!tM8>}MLYEUdDpxV3G~qQmjU3`3l-hJ$=kR{V&Wjk01!+SvhEyVq(H%8 z!!QV^aHBRm^>B6o=EEe}pMYNn!&(qVXssJgir|G)$QRT+rptKHowFOO+aZbddgSes zPD3J$MW;nUsk;MfthzNrtt&}{l(1mFcPYa(676itUl+z6VL@Y?-PDygD4e^*s}CWH zT>`KU2lb=b^8RCzary3a|DMl9X@D3|5KUq+SMC;!?<5*GEc+i6h#Wjx3K_3fnA!A` zVN%^1o;VX-DOf5w+pUr|PZ$_fb|6H_A?14*8HWN^BTQ9zNBYl&N4sP%m4)hXbugA8C7f zD4rOhd-o~Mu(C{^X4dMm@@;?hXD3tYgScQ{8d_4w$B-3)Bc+7_(_B_J*T-+*@DcAi zmjaYdrMCTPOXDRi{SuFF;rGOcb6N-Ehit<#q)q9%b5{GD#x5AUgF@!`utd*nb;Leu zJ|$J@&e7Ld1WsQ&wqNpL&1+S#O;c*hdw)+)g7jY~@9p@jeAs0{+j~6HsFlhJRrB@) zs*h$oP%=JWg^=T|GR4sGgEstLQp>H8r;kqhOH1NLV6;k?^5>>hB3xUziLxGTD_rom zAzP<`Y6}V;U;bHER_2*=yvtCI84oe!;g6WCF2M5$$^9Y+3~tcgfAv?`;Kw8pi=NP& zV~9A!)4wXT2t_ z5k;`FQ8cTUmMDo2T$(D?e;{DcVA>KbRu1OKQV}ZH;rxCDu!cRw^G3UiGzk!Ku)2L9|(SAUsD)#EjQX%^ec+7e1pWXD+Yt(PYCi(^guNk zXUpx_7k}?$(17uuJI9;k@xDe`%J31i#;VOon-E4g&eGXOO2fN8;__O#2csuStU!ga z%&+rNAYPx!*`pyLy8PM2@N5>Rne{f~zHS;)u9h|jC|X;PImTE@3G>OLpK9YAq;!o_ zY8iaSOH8)?RhLJTR8n}X$_cV$t1N33IHrzt@}iFeS^mBPUVlY9ti0NhjNmv6tQ2N7 znT;4#+IYS_PG>HyFT4<(hTbxej_S}QP*>o7OPKpz(J%M7jxGofGO>jLPg7jrv%Gyo zi*YFejNPdM$J^!3yM*%j?m3gAd0pr2*-OjoY1DPW>l1+v$`$TDUOj=dL6+kjP$A2( zM*?fhChn=)`(K#zSM-+2FklvslA3L!#0e?;lLRWtxfh8gg#lv^8ys5@Nkv;I9tF9; z-Hx)!34S(cgQT|ghi z!M7Py!492q8;JFNWehv~>CLn13yR(69_;@Tmy`CY;(@%T5t9zAGUQP(GH3N2==Vb< z)p&vt(GTFclRMFRhBeRSEL~?>dx}p9HRMI zgu{jUT7Vz@@rxIWS)7dO%OF<6BTKIIe2nF}`rH;p5D8vVvBsC?H8CT#BPj}Nidco* zI^!hn$M}zIerxr|9j4QB$WaeO&wVjAu=?9)?zPVY5ydvo=#Mu(TAM&}Z7b_OhYCD( z*+8qTlLoEn%inV>z2vD6mz)OZFr6&rMAo;F z^_ZE0-vGGzqM!*)>bQ7ah5)*tta9vc4L0D1n{Vc?n9q%h&jWFDe3&mZ=S*1Ue}%AC z+!3qumQ%r@?g=;bK{K)teWTIRPtrCx3RD;MhiVQV?6i}ycx>-BX|!&L+Fmzj-wE|f z((yB`el5+BB3aSfWb$s;m&cWAspCvA2S7Ul$tN93Z5-ZNP+%5Ila)u~p!XH|l=jfQ z!gZoRvwR*9)rLKpdADX~VQEb$roArx0Gfi8@kY4D>D@8}a0#-;!GjSAI%uIf#p@XA zzz!ER|0X8A!L0jlk~2X65M$KtrFO26H$96ltfp;;{VF)2^j6FLJmvat5GtZNPT_7mK#)>M<)c-1zOZgtb| zD!})ww9zTM!$p9Q4*d!%Pu}c3#|)sK<+bcIgCS~6gM>+PGGvGT^`sx58h)Iqdh1{J+RYJCu1(^`0jV$|Fo4yf;%!Wk;L~C00Q>Yd|RM1L{t)|#?F?C#E zy#Q8_Y~ANf=`J^7RD9Yg?q2S5G2*(gJB+sv3gatgh%&nZXS@!yv`+1sTrL#eMg+Rr zW22h7L-8a6YsMN&+Grkc5kE@+w)D=r)C(P>Ka`bz7H@#~2FvuXwhi>KapW)6;+qNxUiw=eL-+VsZ5m&WMh2*~<@bPi~(A>KcT z!;;E4$sra>92;+7GyU~hkJ<_yAxLvP0(?$U_hB7v z2$Q7a0<&!1WT(|O^rU{S|QH8(FVgx4ffq~-KGcr>p(C3 z{wMsB&K7sSRTG)J(X4)+xX~J&k!bi!7EiUd`<;2v(W@r}7eA43RsbfYt@Dg(o$%I= zI5)K;=viqO%YK%2wHXiS38CuaG0E&xeNIEs7vr6>64|4GDv!M0D!}4R^2wnRn_8%b zX<84&sTVsYsz{X}%BB+OZ*bPNf_a=p1|C1TR`B(@pQZl3^T^H09C_=vPOBJYi~fE; zBvhaQER3_;KD$hvhY-mx$;ZT*NsQbhMETibeo;-QjV6r|zv!}FV`)-O!ri?$yt!&y zoH-}YuQJ-??Yq;8$@aY08R!K37P;!qr9Q#q_Ps$7cDZ0;*!LNIw#fJK;_>dRemGL_8G6Y+t1oJ%wvN^c z@ik+h@I#+q#_QK?EZ^U1ExWYSc;|S{=hB$M>PX<16uH>VnJ@&w_Wyh)Iys>rAj7=G z^pZdxc*TGQBvl`?(UM(9d|9!S3&j0OE|;jqoT$iBLdT36$RKJJfNEVzTZSF^8FmgI z)2*^Y;hHYBJH$^)>kJ>1Xul$8zf%Bm5x#Se>dbFD^oIuDGr1t-%8)X8QIwlz2l|pp zp4lf7^}CPYjjyCxoLLGbKfH8Hy^H_ge8$40Q~u;FRI)i$%%g-@ zXTGF#SNEgG2OALxB{s|h^v?ZQd=6&S=@$#(i(6neQDG=7YN! z#29~VS*|%bws}6g6EObD`QjN0qHm0#cDWvBu9u$e$ChY$q0>*omwDT$6EX9TF9x4; zlj7(Hd9^~6f>KWl4LkXM7}T9f-ws`=E{wj{-}XkOFf5LCPz$Tcu)>S}9GSo>g|$0wdp#0PVKenRJw z*r=U*Y9J#Z0U;7zyS!tFpHEWXY4g0B204unQ{$08{X-*+pOwDToPo+8xoO`T=v~#Q zIWLZ#b?^PGea=?WJg|Z()W~wL` zo`~Ep+MS7 z>4$+g9xZl5tc+UbclDOug<&_JzazO^1t+iznGI^lAAFs3#m8j2|3i{2EKm<-8iv|9 zDp2R>{_Mw&BJb{%m}^+zeCfbwc{fC>WiR5kO-6iXjitJa)mA;n;N8$5k9`K1&GI9c?V+6nlu&;Z8Z-~+Pwl+ z5p2ThGi8vz;s?2j%YUW)@8a^gAX>bkPLNdRYFI`$Hc}!z9x>aA)z}n#XN{N$*0Fsi z?HO<=cH8C7c?E?nz!sxx6T}~|=$PMU;1mzp+saFxK5jEE;Wt3wm%gvXh`Ermks0u!PCWQm_t{=l>`Q_g4jGnPfW%PHv*$gXxk4k89)SDM zVI9Yrp)z9VR`*P`OtZ7ps%9mGs;??>J~>JpR?8Uj5Z0F#ZimG#9=|N31Q$8rGYhpF z{Oy)TM(C*r$pgQMKT-AH1VyayKjC)}aQks>9=3PU-*&<6xFmxn`6iT_NaEynkIQ9g zc8GWTg(ZvYuSD4g9@90>i?YpSf*{GM`vhMQhsK4^Ts61D)OqSbCnY;ByIa-_1M6jV zocDL5Fpt3Oa{V)&+i6Xojh~)oRm|au;p;fMQ+coWr#E2N9@v+sU$;+EaxFxra15`9 zF+20*OH+7f1ley+lwZXWR`HS!#h=&RZpbQSrM10kF0gRw(!X={_+;jH*8nuX)lm3r zL3%8hg76EeULJXbN1JE=_t3(PAkn8{_|xbnSw%#P$t4{Rm^_2e?^=Wi$P4U>n>fCE z*=u)PGY($D4~I8Y)urY%u_!Bop0@lk$Oi)yyZ~__R*4CnM1FTng?<(miho;uhlx|dZla% z&aCX;x{hyoHXOqsQ-C>d@cD?`?+R&pAFm4i`X36JV(^CgPEXjn6ZK#Ewrka!5B}?PkK`!6?Dy??mcaFTD zAw}e;D-=XOed3x{@I=IRY)z&7rRrcx_E{8kA-`wnrA)~xA%dfVHiX{JibftdPuE6a zd2dibT}r1QxVSBGgD#omU55{h`eocti;z1VMZz)priPTJoGMEl8P@#<6ivjGEnKcb z-?orA%nWy|wdy^%ea$jwnRNDawZ9nM*00%3;6+U>{lUd#CEqC7CPwf(oiBk6d2eX_ z;sA8v^5Bd8H=#c;1RN8Pv_E7wO1TDZlF+r#BRu2u&8h~Fq9Dl_D)H~HI*1s2UbFR2 z<1P=XSK!0YI$sJYhf6-$2Pi@Sjgq6sAb1T=ErdO(x3fp2wo_^;`+q zEBlc8oXkm;J;3AkMQ(*GHA+O?VBl;iciZ)g;oKQ1X(TasQnfP|~mW#6->}?`bQ@O8yfk0Dk^&7%90TB8=wm1!92o zw{-vY?E&1sn*)ESi=(~pg^GtzEB?jG0A$+#`p;IP7&yQDXJi80KU;~|r0)24zF;We z4}WZw{c?NQ>F*KwAC&=JgB<%$3f_(GNcq1$sFW(xz*GI_Yq&`NUle~rKmYHG0)Z>n zjK_xV%J*odZT<7F-u+844p-B{TAN~S&f^(9&iw7o#Bx&($3a$3V^?|p)h(GpQwS%I zbruS@sJP!d_J-HC=^4A!A`0jdx49Yu9#B`HOW*5n&(ExL7DpCGMt;6iJ$tf#Q8VS4 zb~z*@pRzn8wCwm`UE!087`w;Ni5Y=kQyqN;%@{i+rl+My|a>;UzL;as)&) z5d@_9XClJ2%2!?sfiETEd~FxvpMK|^Nk7}cZTCfixEVc@r}+JQxUauzpe+a&b-AlK zGA-m`-uyA{YY{;|ukQ%kkN90Kv(qkS8VOFDmrqdt{%0Dx%y}A>EhLXj#qJ&wW#ly$ zf#Dm3zn{i^M-h2l#WJheL(mZ8_vgpy+DM7vhs!RW~`V4X~;FyBHpPVQH67FmG0DkQF?Y=tks|8C59mfCs&;yAt9FervR^0K^ zoC<>leA)0{=b*6L9KHKw9aUY-hwGZKgaQY5I_RI-eL&tL8(rkhCq$1o`aL)zTBx6h zi}CCDE1w0|>nE>d{|>ZAU1Hu#5J;->$PVeW?=`j?!V2j>i+mvtDpPe`U`fI4#wdTM z>4D@PhA70#v$r_g?MnG1`Wr_T;%}j?JoW2YUq>Y%d8BnSmbLi`?>}Ri-;AXPOA7m` zMDTYkcHDL=tdf&949$Jl>(h4Z_|O7(r-g^>5_*?>*MD{ymw*! z-n|ywn_yk5{i^s^2p%tfw%1bWU(Yw=925)Lcvc668P${{@1-GHF>8_hE7y5xxI83a zN%KU6M1Q`D_0%u44T0E4D}noR0WD?icj!VdknaM-GlIf!mCLG=?j1l@Pj%yGG^HdYCLi!)si zdFy(zx$mI;$#tgS%>3%fUAW3$X!PU#D{oG}i|@{aJ(d-G{U>*R&&K=_e6IUB4|8N9 zrrZTPP1k$rNCIz^ujz5gWh{Ke>T7p4PwC<|338xTUk)QC{wk1&!S6$4uI8vAhjE7f zd&u@TA`-^0iCGzC5=pO`kYak6dhwEP^cfMYuD)`IQ;=hB_StPD6ZoQIy=hSYY(4nv zbeTWWH1S&aqt@j8oJfRWh)Z5sY$oqZ$6+PLe^jMEQY4`CbP`)u!HW*;@hzTvhUwA& z&aC^7{P;{K2>yIrggh0@3?ZiHq(l+#k4Srw6?ncAx>-THd{GL@&FxA4nM^3-A8CbK zy)wu8^XnV%9Y}}VIq!IJeRGn@jgjb}gs8NPqd+yfFdp{)oi>6W1RZW*%+)w2>3$LX z-UI_&^jBOcBI6{j_@P2K+W7S~F8F7p$Wa|Jkj5biBKE%n_yLPh#ge7jV zTU>rxU=8JNyen|^c2t-8X9kF9L%?wA#Q!KT#g@GD_w+<2{hq>5)3-l%ft(OArsowM z>XgD2r@<{!gqpu569E|ysNu~3luHSJjUssr_}hTh`O)Tf?Tp4JQy187xg4zg zC|4~8{Tm+%dGT6b13%1}`*7eTtF!e{R)i3T8F$rt_RJr(vBKVGPBN$Vj4yo#cc73sI)|PHXuO~tG_u#TZ2^G0B zU*qu+oS~7Yo&Ok%D7@Z zt@QW>Ciq8uo8vkh##KowDG2EKvi3g7rGVBv@lInB_c`eWj!M?z)Ttt+BJ6Gf zJ^#v!-)^X*KD!ud%Dw(*k?ybkkb*0Y`PSY>q^8HJsols zxBXM2&k%)4iSukPzER+4a=48pB^?rexj_ao*)M?g5iLM@ z=>-|6&U!Y@ZS6ZFm%#QcC`uyB9Logu!oTlErhxH~4thyBTjAoEfaqwlpAWj5ZL|fpqI|Pha762~NCN&` zd*t+RtIn**6QGZSv!{I-LBfZJBA(9+Xw?clS8b~oYip5ui%WHOqkZElW~0UWz90ZF;LC2vV9$LEKakpztoJ-YmyCiQ!^g23EhME^R^mv>(a)y0 zDPD{!itO&g7K2fxrFj&|fwkWw$XLhnE^?dZ0FKXq6U(_+!N20wX81%a%;9#YHdC!i z?}uo9-_FxGe8)JK&%{fTDEtL@Y2oTTixRSCUCe6>1x)(B;jx=&rM<}x4U3^3?y0=? zqMvfNtXJ-hzc7J?3}!|5Ex?tz{r!#2ApcJ`qo347`7S)GL3V1Y%03Hq0d(RP0?$mH zgR)ljhQ0f;Z`}xkz2jJSzbGh?J!H`#xYE?kgIF4zmKS>$>Zm(>&dwuGe7m&c}o`aT#(Gk zy&KeX*<%kM47A&U4h5fc*G3{y-q#%Fmxyzz@KBfPfC?c!y~{}EzKhl6 zrX@K1PO&J-WLGf@Nkfgt`8=7OK0liX0XiE*(D$G+`=M;2^v&O%bS>{{f|YOxZV3qUM$EOsRPsz*zRv!coJ7-`Y7JrYkAP z!vCS$z>up*u|DujsAQDELhyrfp{JO%zs*aAVKQFQCcv1dzYM!SA`5L`5C6sZIv^$U zdAHZedE>R`^g^CU)uU7Q%fr$5ZNH6sYMc9v91oO*Uz*r~vc09yUPIkN%ib=HK?Zym z=Dh>7{7AT0)(;9uL)X`Wfn%t-O(*&HG4VngIJ6@9F|wEUZGCTB8Rpy;=QOyA-~K2S z&BeF~Gfe8<T=MZY<@EBAp~>3w*_RSS&Ub4v-ODjmcjkzT4VfOpgGxKCgNF{tu|GvqnHOL>#Kmr->JijI|ezG)6< zjAfN|ntG)NbfJ#bJQjhOZe!f(Kco2G6)Hv z^8GzKhCUS20i}7jk&e`_j~k8!D|6~IxDa!__>2=L!n;Y78qWL(5Aq-kSa4Sjn@2}-5QC%ZN3EL0YQ`D} zBYwnDiJ`ZkiYer z&2qkLk$C0m+i6c7pnj11SSpft_^qZ`B;?g5E1J~=MQy3Ysio+{oT7V|hjW{23T_iI ztn#O}!-qv#{QKoD+E*uJ%AG2x&WReng6aI@G5eLi=anOu5}$&Uso@XpbZ2A-g3)0y z@Ywi2{UtSe=)lHCgwdq4v@6l1wz#DXND-!o zmnI~6C+J_&3ZV>sl~{XaE)I38Le4TW6;R>HKJ@n-#>l}=KGzp0f6=$aef`s-%?C&mD+g%Al-TgC)xFtT{Qquig-}hbcyx=07J_Yb zaP&PeLNA+aGC#$u`0zy~@hKYi*p6kqDP6_ad_R%vtV@ecI}7oURz_T0Gn9MSnX_$4 zea#(w3?51W(9~-C!Uh$@LILc%np%|y#JOTYck=4wUBHWKEI<0@_377e7wjqVSsZtA z3s!=a1k1AXn-xYx*?=D$BE(_lB`UZaubR|7EpnGQ^fmeIO_bjzmfoE`7Q-I*)))G$ zBIh1zR_U`ES^Rn*(a`+~Xd>xmkegIOx*GYcUrV6tM@h-+=PfGC`$OhEsj{m@ftQC2 zy`8Q!_EPy?IH|KpPt1BrU_bX3qo`j_<?WAr_5Xo26Kcx$rMqLeFi%B9rWJOY4{Rr`R?d17M zyTYFZyPEbK4`UPJRC!1%kx{bErqdAn(k_=FTgmifpMWBGm$T&|$d4y+2rj-$yuVP$oToP>r^kl>Pc7 zrH?xrFDY74DlL(|`>CdiGcioFggypr6g0}>JZn$$XBWj~gH2&f40j=GN%I-ibZ8Jx z0sW*ReA6sJL3C-qx9lf#&s;l}Z?q&B)GZOk^+!ns<1z0{2^a()rLm0r_RfG%(7OfEx`nM~9K!Vj#AsbSvvUk#yqA zhB=9_(!6>lOzYJwi!40YMx+hJzI}YMeyxjj?!5o7xAUhXBJ6!;X2TcSg=@|W?RUHE zO$Jj{2-JR5jS>wXHO3X+&x)Hb)zz4ZRq*Wg`>4dYVfDDBnTX)rL9249H^y1Y-DX55 zKBiohWSE{&*@q{Wd-FYPx(k@4=>vkAQTk1@Jff)U6g@5bRb9JaHky7*^~oA7*5=r# z_u(c&pi(HsYyU#7GpDdTre>MC#Xqg=eb(Nz_2n1b2g(zI2dGMJh}T(p1rRlisn5Bf z)-31ggKjyBg3h5lx=q#NR~}+$M} zR+iYxLdJ-SpLk|v{_66a@fMr!l^5~X6PtXK)mC646)nX31i7g2lvTdQ%1*bav22G- z7{$nLMru|VF#c$^Pnl&hvsuxd%wI!%GvP=TV|x+)6q~-ww2P00My==*tU^b|X~?1A z!r^;pa4`(?2|4y7yH(EE(pCASNLrKQn0ZnyGu`oxdk6j6FTbZ{>x~Xqwu`*P;})hC zulWGyYvH&weIcwOr@{a&QjWm*e&JMk`FS_c>$bfnEN;irCoWbt_}qp<1~O6B@%dx` zoMel|Y+Xy;PrY{Dw@y%gsHtyfF+gSIBl@KV;3AlKP99{hEBNI^S3qWkh7l6*mXhgE*k4COHDehf5(Dpg_qyQTPQa)4 zwU1`MtKxm)&>c^H>RJ@PD^6tsDdmfkjno~X-@?Jh$c(EJA7dsHud8jHgRbx+JDgm3S zRChm%n52X7t)Y;WT@eTWm(Ai^x6UJ6o?ffxL8QBx9doCjC_Y`_8ff6Hep%im^R8pMpGJz#Tl1^&4m*DDFQ&3!92O-n&`00Smsd@!O9_ zk)#T9hPt*^TufklTdw#r-m0e0#t%Bm05VYR^whZg7vwE3Pt&=Fpqy?%M=Y0&JO&+hE@ zgIP^xo9uE&E>`EvtkH^(j44H5(|&U(Sj}J(d$8cEnEs_$*Y2J$bWc4FcHK)Xz$sIZ zcIwQyv&J1yf4x#)Ir*JCx(_`>^X#(?p&hF-Tv%$-2Ty1&+gcRjcl zo~qfwv@{1n-OgrXPe@E!rExQI6Nqg)0Oza%tGT!h{$%`A%o*a zfQ2uW^0nPL_mTbi2BTy6B&d%ZvU@8vZ$7Eo`y|xm)FwWb&l2 z!(}CYztLEBJN0RogVXJ5PD4@k6WLj+ zrKWQVh0W22V($))z5Y7cm!{YDOp?`(b2! zNr|xH3)gmSAE0mNk>$n-TTkw+*Nm+}S;rN_L<^lcv%l-OXuZYNa+H4@99LO%ADOv^ z3pOB9m>H6B+giP+`W8c8Ec8h6z1di1jd2&BvwXvJSVSq|9>tCKbqzo3WN%&E`rS}A z$Yw|qy;s{c8`<{)`gU>t-0r|hBPa9|K2xWvlp!9!?kYn45cb_!w{pn(8!%n9n@ER$ zgE}iWff0aT!fAn6z0L-*;>&7Bfj{Gguhro=am0YUg^}YV6`T;YTvlR#y0=KPu(~Uv zHWn+pXcfB%Qk$sPqy(KTlujF5HJ^>g+@6g*y4ddH)s;CQUaL;t2 z;b|~Ej#6QJJudeLEjvN_YY(7E0W*!USkAPgW@2& zZYUFR=z2{_YNj-Bg6M1BV*d_1ARtr1;p2AsaW2H7i=x<2;p@{rOn(4O{)(_w@c?AD|zbp-JOzn5}I46)Uk@91qNEc_-2ti_n5ebj2Jp~( zufBIg({>qXbPsBoTVyT(H2>HCV(+b^qI|oyVZ|gwK|&goPN_jaI;FcqV#pDcR3roi zl&+yWh8$ujX+=RDx*gI2DM9J>JBRRo*1EmF=XZa9d~3aHy??o8WUlLseeQF|aqOMM zi~AyMrH0-v`HfM6kr{PSTfW5B>&sQ`{EKl#j$#O|?KSyos~=$${CM$!OIyGEC~Z@juRD?rYyTwrBc5wJ>cR@-+`S1lwLlx6oAQUoa z`sPmtS`ceX6GWj5&opMjhT~M+6&)G!U=IG)P1lo(gclEUSFi4Z&;)hoOT_XJu(p2x zv{|ndT)$G_oG#|lQ5F4Y;N8}8le78SE5EJql@;aKf(PInY;Y20&!;yY+6 zOt;YOSWm8*PHoL=`)&O;;_~zR(sD3%k9=<~sZQVPj`CuLc!kNsH+8ljR?EM2nD3t9 zU)OvQ0~=-FB`9WX@|-s>-Ff=^olp7O;;StN^2kC{e&ZS&Uh1YMf2nVH3e_dx>fZ=T=n zk?dLgKe|F$Qk&QDc=Wz*bSwAN)Qh}pY-I9KIdBzayuhgdS1jm^HOc0l7gxbpZjlw+$Oy;-6xXR)_=iQ4;0f->>`1!%-7W;%kk{7Qu0;m zQ+$U4_b%OymQ5aZCq~hQ-`9kzBz*UTc3yH%>NzXf2oz&d2rWsE4}iZC_UvjvIBpEy zLkzn(w9NuEryV||WwUA7-SLIjQP##knkY`SJKz>hPS2CB<|@ndd&Ao|mT8|mTI?y! zNu6!kiuJTvr!&|m~xA!9%CO{ z{Nf&|-FurAQILD!tvE?E@nmz>U3Q#uKLHwT>;>{=o8J##o~$6IS!e%H)|O)x|h?7vy1hBz9eQls>J^MHgNT`C2Nm zU;9fuoUb%gXs|K{`|gs+Ylo2{Y>-0*i^sUnngXghCJYudS#(q=rhG-u7}}=gi6#wD zzN0+MQxMRAfN~c8qXZgS=y}*e`%Tn4sKT(RAnawMK@XCHKwn>>5PHfI?_u&S-#`u+7O5(RU|gJ1q_x_J_@ zLg<%E{LtN12B0PnPG#yI>CsnF*o^cOlyL5M`iY}>90b_RFqt@)pAW!W;Gv+B=oIeB0Pr$63L-#u_Pd3R!1T104ZYTdql$=b#yjWk} z=o-|7+&(X%4dgu+p+Xf{O9?wWN&h1el7?+vJo*YKe6)>t#rv^XO`BIiz-Sj0e|Ulo zcohYrNz|w(1E$AgzKKOb1hEbr<^2Fa3uKfd;GfC|6)@ECiUaCLe|GAukrY@yQj69* zCtoCErzRZ=nn+;NpZ*q1VB#gw&ZEw8T?kO>;COVW&Uyi)L}l=O-aq5@1G2IaEQ&nd zh*2Qmm8QSuGipK7*h;>0ty3rE$!=O09G{fBs(KGb2 z(c*`zfi)~{0>Qb!9OriSL>HM~mBj1`;vSp;h%)(sNO!|-5rYeh!2U4Ba|BTHVSa+b zd_UED`%v=11LweMAZ$bAj$io+yLgR>8wdr~l%~%i$NLW6*^AA(l$L8AV}<}M7yNLB zn#TJWoOCR%-WlikM8O z#mNYEu*+}$&7S;dv3i-o|KAJIlkkzlEVB5Fo+yh`IOo8BoTK-ChTU!K?(BnA9;1`- zI)?Vmih(yy^3#TqfyKWOv36u60l_#84w^8%e=gO@#_nO$?|zn2AeDb4I%?oaqR|Y$ zE5W}K9Z%Iy{!lWr41BX2lF@$Rq|BZJKQ+^6V?G+g-CiI{ghsNDhv=|5uQu+JZM!my zJ-y_;1vtlID2X*jS0gS@9>Zx?=)vh%pkCR;f%5;p_L<@2hcZCsT%QN1>L0)H4Nx<# z2G><3jz3#rCqw4-Mu!~gnwYI>eloE4>EL}E46RZp%bE;C3#Jryh+n~;4o2|1FwJE| z4{OSVMh@Z0`>xCqff>>Iw{f0)vws7Kpu|QM!O_U=E`$Hycw5wxo(_!hebVMKHw+3?XZ$k9X1$Y*-WXZPpD<$OWdDH7v*kyLbNnVD7?KEOM`&tn)MQ zLmA54lerXtLtxkyIDQ?x`8tvKi3&aCFAnCkc*8sSL?$wYzz-k4AuT?B@ayZ~!3Mt; zO`XBl@5uhgN#+HQYJXpFEGkbVm%tBQBd;7B?}iI@H)KdgKb=9_P-Q}2h)%;JIG&2 z=FAML{n^?4cW=8?YV+hU1PTPXqE~0jP;*wos%PetMZC9uKF(-%d-3bU$vcI^b^Wf@ zwM#YvxqYt?yVzHXUN#Z_b8wZhuk^1>*%M?z2rph`QL_`OjC^=3I9@7}5<=MN^?+)y zY0D_pxUTzeCF}Y957<*6@dN$m2Lyg_S6j9C(tJN&4sZleEI${bJ9zi~Ghc@>s$GH! zG|sK?oC6oa8U2WpJET_&93G2LYt|l@NCcg3KU(<)xb5CEok{Bd_UcIACu{tc6=KA2>^ZLzeL@uoN$fjL?Y>X@o*qseK+NwQe3hZDDV4;==cw- zEub{0d`7YBrX!{3ZCT3Wvn_hGslHDZ6AC;pxjR!ForB62G{RJh+i^Y_ulCceHvHow zfBK@e`YWNZ3Z$vY_~PMQKU0Pc7$1%0l?G5G1aDwDvX^M8y_q{%jxXw2!PEO|vlj|F zwnNNBS$0`dzMgtSpZs15Gq_^gbvn7E@PqOMBy+_6_&@+@%VatWTg$cx3r(qM2t6>c zyM0#{0}BhoJJB=Wzy;197S+YFuyl(I7G$7!s9Ij#y3(r-77+yAPYbH9#YqyP5s-TW zOZSp-1q`*d6YE`{kpDTLc0{Ky-OY7Qep%=r9wcOXkO!3IKn2ggE{F@@07r?SZ4{ph z9R4T_dlCN1RW)w95%m=H2=GC;J`N&rYIL++6YHB#HWO$?GNRs7?Qd0&Hsd_B@^(NM z$m=dr3!jXBEewl;6Q07N(_8(5{mLa{t;80*0xjaY@K~)61@hI)E4>2vj*xeO0^w`% z*oo9+i#=M&WlPB_0hmcEnm_xIOaE!5IEc4iRau4-D}hoN&Vj$Z55Q*A{pTVP5&w+*v4G1h2 zJiDR^SvO(Vhd5HtT=}!k@1vd^h}?u~%TNx(Jn%F?P$|uXk?!OMp}lfw=VG32Sso#J zN5M5&$rat&;{}e*y7#RU2*s(hmf(hN*FXb2Zo4V^-hcf>`A)oR`08MH28~|@9zH&oPG?bz&@La$zvJ)+j(D0*J8#7$<&38VqM0{6| zQU-^{u{L-h?^|FphdoI6vs8@N54{0?O%>iVhg$L;Q1h7fg3qo=;HgQtfA(B5lMOuZ znkDd@+FvSDoE%;Nuf^K=Nvwz?bge%Gd)r$L{;X30Z}y1XypxYCP{~ zu3!;fV8cHE#zCUpdAKB!R6xnq@H%XnF!yppT}J^GmqpE_)+x>1dZ4?uXpbrHgIo= zF7M)zSiAtMmy#V=mWErUh%ssLZDLCS7+Mrz8mM89ezaKePGmFk#-9Ul_x7Q%{l^Jp z!{jBiu}}8b!|oUKrFRgxX;csxt9$4luY>+(DR$WR|KqSV zAvfqXv95+B2X+_vV7E*`%_h_%fR28X(F|gFENrXUq1>p*6d`8_R2z3X_11&zL1DfV zVFt)w;8ld3P6Hvru!cVz7J8z?c)T728?4<;d}J+vC-(5?r@4WAv3zkNZCYq$I>R9n ztttU5(qpUO?+pHX@Gb!AO$spl-3)NP<>r4(x+kw5f*FTp{q-FHuu9Hll>2zMtYJw+ z?(XK&1gQD)9SdUPuYq=ug?{+;^&LP#J_M!ecHA%kEK1I4K)+teXU97YYRVVLuk)q! z3;p;C><;#OF-HMjB-V(nkNSL{2|FeVew!_NB(p!tkiwkr#74VDhdr68ASUo{Co_`? z4Q^U~L0k_o4*PsQV<~%AO9aemsu`fwic*rQdH~Dig6^Z?EnYi~!hTaVikAkMFGBMUd9FAA7!7 z33bA5wZ`AJ8VmFL5C<@j5yjy~HHGUb;&KRY(%$)QP+ZgDLkv^aFo5T&T{iXJ2Eg<6 z8Fll?Y#;BH4=fXuhq<#Wc7kA3dcl{!r%4{55p_ih=RcOC-`UiFWY(DTW^>e`X@01S z_JI{mAtMeV3)ei^j?@#7t?S2Fv{9YJZ`kGx>B6q<*%q}s3%vj?h=sohU@PIF*uAP^ zgG=on>3$PVQHwkTx1GisyLV61j4oBlVFp1(C5R}vlFm5-hB4Z(!m_QmJtE@7#s`tB zxUNJ4>%SaPNql(MnG1vusR3EQYSqe10tDW^8%t9qZx2`3INuvsvAF=PJQaUvyPppN zUhJ!9F7HMA$)K)4g;7?{aSMhfIO$aSkK^++-uy614X50fJ`XM;clM6dgZThM5-pFFw~>H1ApS^*nmQqf6`x1vNq%m(vCFT*tu!wS;zXq zi2ixo@7-y7ciY1#Oyh*Wtqq5Lz&h7APXQskK>yrL>MWSZI-}b}Dk5PcYv`5AW(r2( zM_pDVRtkN_n;r}GojBvCaD2cGMsY^sZ&EXgfTfH3h!C{42I8x}^xnYOzVTLf9)H<~ zr+7x!=5XT%9>pAL=)2fgb7snnrFFtpLqoZRsNnl+`4ZLCVo*t80Fb2`ebx78vM8*e zUXA$M-Da19GJO6DTSF*BWvr8o5ThVwHh^~&bMw4NgFg+9#7BBPa6gGzQgueeA zkzF9_(QGdR6};(JI~7?q=tIK(9UKjo=O=Q4wQqpN&;;~^f3qc?#B-#KHgqE1S4Cn# zrU(C5A5IU5_8R%SFAUGwfhk`3n{47iR{`t3S`!2Ql+z5NX58$(9TgmT7`%dYOJb}AQ>eLCk+T(-m?=R^8WqJL-Na#L#2C#HsMk>po9}32`N30fr zrhzgrd(cLj{u_+OYc@GvzYP!zdQ`ZKsuah@<}qnb;YWD$s$?Mg#czUF)iO!ZGk{nv zm|4HpuiW+HP4MNhEeD|XCy9vvEOurjRuz8wt18@utG2z(NT1WPNi?Xilw)vG?J$4* zqdMy~rqY(2VC5)oZKMQDp*;ULKXCl@K;S52`#3lW7S;u5B#Nuie-oT*7opk~m zGW*e*ic)E{BtGc72pYJQqt3xA|1*^~T`ph2Nl`pJDZTPQAnt`xiYI`JH$ExqL|L9X zi;cd7F<>zi|8}E-+wR6|$BXsOQ_Y~6QAl}z&ai`#ug+7FLk;ZjLDYNsHRys-bugeG zk>1ke38;j-oh`U6X*n5NgU;ZE`w^#poaoD&7tgMZ_|u;A1;&*&uuOoXxb^ERa7%us z;Y;ubjYRpbqZK7{!WOP0*H8w)UTa!pt~t>~tXM1hrUBZyY^47c zg0cfU#UF$n-9NM@B{7`*4FBQ}3V_Hjg$(E4RF4MaY8wgvp-}yQ+AP+T7nUJMX1Co9 za2O%-mohrQ8g;1r4{KNzYqnkiP|bf+3&4B&zb?kVxfuVaZ-0;i#v3H})BO&%$<4hh z2(nTu-52yoeL*$|4k!O0%!%TcIg`U#4^CdclC7DlQ$feUZ^W2LP+-NbX} zQEJej!_^!9j0t4k>)hu0L3J8?bQ~MF;u=7|A1p7@hldFUOr)y-2LTXV7=RZUE;G;Q zb(ujdKJM&pPu`>v^{}3dMm=uYmG9S2tA5@!E+BL7Uu`0>9fSh=t@dth zm}Q%;4_3jG-m9vQ3Pykdpu2Iuzy+w+*1y{06s#%02kE8DPk!5bijU~-(FUG6n|*OM zUqM20hoV1HQz!5N_u9r}G>`WZ|8M5Kjav6muSzS z*?Hefb>ALGW8g_L!C>6o{0C_fqn$-%4UM3u+?RYUAHAB`{c>~G6oyNARC5>gP2J&r z5mKghtSeui;a(klrnen9?WzF*MYhc(2I4+eJL=C`q|Y_8rZ@vUCF?Er%vfes5n$g* zdy$*mmy&Xy$I0xhSFQfSZZ^#}7`UHW4A2*{w$o_1hh-xG8(CD0$8}+=Ztj(s5K{aO z9GD>eKBDF$jYhN+>$Ur3$`rkc9WJ9q&lG$HUa_l=IoqEQya?8z-~({4Vx-NRwUKw= zDX9^>@yA|i&Q%bTo4A`pm(!;l>4ytKXttBxWX8EL2cK}&REOKJa-I_(pLfy65+d)^ zFq7=a*q~_pqT)6I6v%dViFR>j7neafJ;KtjV-2_-4vr0sOy&+_wk_=L8BaZO){ECc zk0%Oz>K~K#2shy|JdFgUU?q(A*5_a$ci(DhKc3X{;my%_?mw(ppxPjY9X~Hzxa=X&A|8Xy&P3` zptvTQ(M^E15s80#uka}BQDmrHc$16>S03_$a=TfaQXLrsn!gq6Lyh}zv0P4#4C2%RVPJHGIT<^15EuPRQteflgo%r8Sa@w zdPgv8PjRc>5y~l@8^1GZUt`-ti>yGcf%>OZANKY)r^q+9TIcnxAEhy{kKeh@$&sQ( zRZSzsh?iCXti4dTu@F$%OZz-vh6|^$n*n92X9K=ZLl?{`SqyeKUQpxLGQ*fY!h6)F zgWDz`t}(Guu+N)7bjBBgx9=Y0xeyXXsSLE6rVC8CCOsxv9c_{Ydv?X_GS^*tBQ_!r1 zjV1!0;FP}`Hq7=d3r1U<5ly?t8P=Q)%b2yW)y}|jc`_=6KICD-mVU1f zLsuf|bMDa!u5-B!KTA##dyH)aN<>zyf^`yMMskAIbO~I>h!tW&PCsxnD0LtnXFn6k z4pdI-R0?e#QUk~tPqfY^(*0gp5g(w@S458T*O|yEcCBU{tmPZcGRiiGBO&2B z7ukg_!sUS-p9~tM4RVe(jHiowI_1^Yb2sl!EEh5Gl+8{7$0lY>I>7`Jr&U(@UQCl~ zJ#QfGOaPl{f9=b-t!$*{P+6}_{@T;a;&$9!iM%=5Nz)w(3wtBVH^QYU#NMkXN7QkN zx)_|Q?&Rm(=#{x#_|=y=H0JcknbTYrgwOJ$(rfL4N%ni=`dcl7^~%glZ!zMI5#Eul zl=)@Rzv`{4Us{wclJ}Cz+u4#c3Ysel)xJuIE!`-Qq4V60Fw|J;`XB~Mqc1RFp*|~H zowK7fc&Z=$;3_@B#B7ib@5-dCif^7-uM)0Q;m#q*m0ZmQMo zvpHd_E8>IE8>Y-v@t4L>3uc0VNyI$Ixsx!Wq5_tTss@htD}&?Q8o^m$GW%crdduy) zA5=T`w|JdVY}A@YWXD2#XRlXImg)-yt)+1_2F=9t37uIKi_#RktY5QU7P8>7o3Fg) z@42mQXohDx+YadJhK4FhT6mLS80F=|4fd1GDDR}iZ_0I=h7UogF1`hjOM872;~R|) z_3;A8V;^t04WeVbDDV2Xc{{gDyUXOBGHkx6?;)FXZePc;3j`wYhZl3kheISby$9ch zkaCn{zqKRPQ!shSHWqDU8<>}^S5vC;BHbc~Z&tYqtvjeEH6FKT+UwaEf9si1<96Zd z+jVjtd)zdaEEXkYR3T%@3Mg>OLBj2~Hb$&t=HOc=nNN*`$ExYv?~zc7T!jv%h1@x} zv0@>e-wVQlR>C-1_g~4dox6RT(XS+g#H_=agt@pl&Bc8txI3|sS2{kL8}RbR$#<;m zeRcyk-L|I@RJhX2@n$IN7eg@`cEais`&HA5G^Mv6xt$L(4=Wm4qq@`Tg}K+$C&oB) zCx4;!?Jc3}yd&tb$K=+G(fSA0aV^hVgKqy|&vNqpsiH2h(>k{k;_9_N)wu*0%5Ctv z?OH&L6NnOacnboe)I<7se^Mn8N2wAwi?-vEGW6JF`di5h!zLh*sOP@@%|SK0WPsFi z#@))st^<7KSYMy@;=$YvElgEgo-MlBIi--qRj+*e=g1nq$+~2<<{WvMwxa?-e z@prwOBZ}-FaS&;*Xnd5SDbj=trRFj#Lu&7R2v;wpkAF5t!@BBNJs8Dj_$bnVH5%$8{v*V*#dh2c`2bU-mKh1D0w1i#hOZFHyUr_?Pga4*y|C}l?*^%wLOHwGJ|Keo6R>=?Mt z{ccc@0gt?by`)*T^ok{yw2npeuUqqrcuXu@&8}aKYPR2ARktH@D)<`g)Szr39gx|; zrCmQhXepI+spGN8AXBxP_K*2zb-|Q1gVRD&aJ!FacdiUS6`Y7dhU#-1ppWm7YPi z&$a>PPO5O#O6Cvh8|z&2YV1=>csFBITcWEh$})7Miy ze%dL}?kK(JETjWzxM3w0zI`3VO+LAvcaO}iiG*;@3K&17Ww%@WiUy5J-@7T|3!92WyI2IuKr730X0Uy$!gn?j#a}Po zOCtIqB`$eqWy75%{ms7cs5g>HQDb$)4@)`=UUw!5*u+f z71u44Hk8Jq*`^w;EnZZSh3qh#{uPugZmX>yf{C`7m-nv7qQN8Yt$Vl+oEGD4c?*{e zc(47Z4k~Rak3J2vxy}-Ad(+%UrSkhKy<$|pNwWnAdhaod57By)JhedMU@n(klpEK0 zrtdc3%s3LT-egNV+R$v4DsxS=uwdBkv%rQU2HsWr;k|Dsx#lb~f1S&EX-@A#&)PH5 z!bC5dF7gPp+k>{k`0Y$j-jdsXafSx5PWd6t*>A2i#Gc9M-9|(#tn}IEfI>Z~b7N*@ z?KHo;18^Sgc-O_w3{TWC(*!XN14SS|3i8Pu!PK*d#@N~=TEg0KRdop=u9N_}VIXicStNq?bI&LHve0LBFPy`} zy=*DSjA(y=26spO6%#8j&=L{;l_Sf|hAi3R+}m|oiFdMv)T$#1)!D-; zkBp~3)w|>)W02(;!k+`oJwHV6xX%ZpanrIlewru{F_V-9P=9jdubwG|sdD5Xy`C2o zq6%B8yA)Zk8s9fX5zuki{~B6%nqm)c&EA8i#a6UC>;aY;f|1rRWy?#s@#5n*)iLNxusXO z3GhCi;7_mj1zyewuKhvHqzuOoo>rJJ*2QP&awYc}&y>LL351D$L)+wwQJH$+l1C)% zEVsj>LXc603#G7~)=9LT5@e{ZC_zn!e;SQQi^}pD8TPr9Lxykcx}nipUfyqJR)rrd z;-feD){ZWcHMz*%a@3R@e^*Ws6;-pSmu- zdBIA>4s%TJm!1;CEwq|FD>K*g0~XN;H5VTXz2>UxLniA!MWEmbmB_Nac|()q2shLN z3DZlb+Ca`$EPSE5(|M^`wB%XykM*`MwhukAdwRX)xpc4?(%5cs?IO=<(a|>Yh!oxH z2O-)P5^@~sv>k&;k}RJIK8xzT6v%CnpY|YiapiB}dvJ39;3nv!u((Ap#8dkUE;K^b z&GB?vpp@fv*wR`cx!H4qu%2FzELi<4S*7ftoI!QLi=CAYF3~cv6;<##Zp{{Kka3G%nVB{8wK4Rl;`ZI@e%me*Jrg`;VwWfw?E3aVfF2{!WRzKE!9;GT+XNBmU+@zSifV5KaCi zj@`H8qQ)~e!E0s|0n!cAkuk7bvBC~A^Gd{0LGC-{$#@Qjx*}>S4TTX*LKkPMAXLUZ!>NygRzIPk^`EWgvVzf26Yp7V?*!`;E{Mbq`%qujFc&yerflVJ` z8u4J4Sfj)V_{;^#tOZ7TDoM@$wuYyEdmn8V7#*IR`6;&4cz&+ry(8KalH++=vN%84?*7G>S&;y=v>#oMFc2E_c3~R76 zU2!DasM=|2Z3xG?X%vd>bX9K|I;UCFw zc8v6Vb2?O~O)&x%0#zk$QkcahsynY&%o2+OH<#j6YcV(Y&XH_r#&vG55RY&--)B#` z$lsx#O5^?nSWtRx@UHiw9MVGP>9Gs{_Mb0&HcYR*IuqvbSU%+wa(uG#t`xp*Nx5Fs zoz9|1O`sok(O4kLo@;&?$NrMnR;Huz@DfPm>AWHrE0hesT+>k?OI>XIk#wM8W=CxV zjo^`*uUgU-y_0;opQv#{?L`HEYR=>ddb=rV>9YvMmEAM#`k^^B#G`+~1VcbKdadTm z^-)m7J>UBW?IJ$$(B?u}i<$pbt+`5+J=YlQ?pAWS;t!b{<*V)1>_8U}WA_wJmo zz~SxKZ!d)vncg6~ky{@-{g%da+`?&rnw3sBLJfCSugzOqLm$GiS**ilzRf`+O9$Gr z8#>Cxs2V=x+bxBO@66`t8H}o8m~h2BN_TLqzh0@c8Bo{f%I2hs?5yZ8R8w*-Ec_DL z2q3b! zINexqtue4%*nHiqnaGU1b7kIlO1??ctnkzatl3nrRhd){0|nJ46jN4$#jW6{?~)Ro z=P<*lC^Kte)^x`LzYBV9Vgt-9VPvvRyMnrRjSps(I~JC4+4QNysKIS>3sAuRhV1&w zZNuvd9msSn>YdT|sfg@aQt}InRLY?k1*JF^cCBR_Hb^NaDy)JjYKN>>hDpd&nG*u3 zUhom;yD0|VEB&Y&mN;QoeM);LVm_X0w^d8~rMGq$NDz=w4=c2piqGZ^uWRB|xJ$MF zpa`%aMIvyn<9B99s97<9O3y~LIDsoJ5oi8A9@L5x!9(~vQ(z$p6ji4$c;m|KJ zu*ecPXM&()ODVm8iO&Tg2}gLURI!Bm-mjb*>z8-miq8`WNvxORYb2Ep=(5*vPcQ7a zJe^=m@fE%eI+e^k4fSc3sG*H2k_l<fw;pO$-R`m~^V%j<952p;Yj`;=AP-;47SSBF z?l2y>Y;p_mt$k*u=0ikv*}Ee6N{dVQo0G6nC+BzemyV*LH@_Wu287#nikpWyYfAYWH8p6%Ii)-%TTS;4w;UZnmtxP zSvpoX?Z@oNRNNVLB6?;ug%}r(5+BO025vuBLTs>$3Qj#jeY(b-qcY&n10u12+{6_4KWauwy23qwo zn`&NBgmayn)M=VoB8Z#-n8Hv!Z}~qzO$nC|o&5q|sVm=8Z*f zCOH6wd!Kj9i>CxW9JuJE7fxLw?{g{bcHDKkjraZa4prrY^e=yY??tOBd#?K0f6k?% z*2Qq9O6zG7ZAK(srloJOc(t-#{plWCAFB$x`D|v25{}`p34R2C?Cx7d!}SB536QJn z127O;UdrDt+te5)7n_RI39i?M49Vs=Ga^d`ZQc0DxeZ!6b?L{)hNr&<%zqncUX~xj zipTZuhKe8AklKB#4F|1rOCj0~^193Cik7s5Ev8sR62272g;xnFGy$5jp!eURjlguQ zFSk6VB}9S7;Wn0;XB~CUNPh2rN6oihp1w;NAH(?EPHR+7MfX>5!y}t8)Z9uQXTc4Coz%-& zk;Tb#3Q2sQV z@?Px{QC?@tTo^GdTn5RHB(nMgJA_j4#&00cox}`pvIJaOpou3Zd|Wf zZ~y6iI|hcQrN)LxN>Hqe8zjSN zPng!2?$%zp#YBD@pUFL7yS)+Yl|vp#r5Btd--}jS5H`YdIM|KV+K)0NnJm=6z+@4j zJl7Q{ncb{jY}BQ(guYH&0p?CjlgeawJA*xGr1~? zu~ijUyENk$=>8hAfQr5M%vnb|>dIZsk0ED@syh?kThH$A!qvIoC&T4qY*dy_qv82u zbQ*d)OD?k+H!$V`NDqtfqRlyLL?&C#?0d?#yXytB#B`HE zcU)s-paYE|i~|$o(Sk(Hj#5o&3>%8$72cl{G29?Ij;ABEhSVM9lS<%&3*Q>MdDDo~zv6y|_vf;U*msO(kS}mEnCkAn zE3&7!OpY&vvPtVtwtEp|l*{&X{YvLcHLEt|eUz#!RrSY(R?}yN?Rkt*j*I%VS_12) zS#ra07Tv6nWzHWZ~Y(n79(MS3RmYO69iDVZ(q zd?|v}HgJ^o>m(MMQ)tJ#UzSgl(7^^+{u3+UFD5S$)b-|9rLq-f4L!)}2Kh3`B#U49 zSv?tEW48P>Cj(OWm>(8xzXcoj6&@(_06xz9e;&N~^K1~I^Ys2nh*Ni^oGY+8#fh8H zeZvZ16SW}K4en(ljh-G^zZC#D!!qh0`%iLlU9ZD9n$V!Goz3L0*P<7{NlUi$Aq_4T z{&?Qi)9PyP`uUcbFF@c${RiH~9|WExJ(eQ8rgCB6Isn49!_B5ufwEcN`b2Y}i0d2R7))bJRmT&(^nDosz=dM}3%Kx<{~ACA4)D{cgtDeQrO@|+3;EYh zdO3c`=MhujaaR9Dh;wC@-yg7Q#zAr!l&M|@;Gx|`rQn(A#O9uF-vGlwl!PK46vt0g z{I8e&Bm;ZR9GJemjsIM!K1T^BKFHHQiL!_BfaSS1`8Oc^|KJ*(zL?7bKBmvzHIl_v zTj}RGAse-0i68{a|4q<>(8gqr^aOhRRT+x|_59~aBfu->FJ1dXoX!&l zpGfsZu&O}0 za55Oe)#kuJMC%l>RYDFE4PwVtIWGKaV?6qf9?SR^puRAL2dZI#yWjEC{gY+!j{@SY zn4>KM1Wb)P=A^)ctY-wTwo47PL5z6``CH}jgyvoT-Bf`dFGsD-syy@z!53txaP zcFF>@4?mqg{fm8m_b`R?Bj;9uHqfY5tw3wX`~qNl09%VIq%{R)=;s?KG)Dl&XC0_p zQqUTZc0NxH8wSOjjbqc+fnre5z?g%IEAIe;&y?1P15hRozO;oRzj2f`Ucb;Sz6MJV4KYPPvx#x`u-TmLj5z+aFqQ_|%m(9qx zT4R)yU!hV1P<4Pe1~?XF8Xv{$XKhpsW{|a$J3rrC4xW~|fS&s?R@-25ZCJ}FAX&Dy zJOmzBygJjxK~fV5s&ZQ!nAv}5{-R;{5VQ{sY=zTR>uD<4<(-29Y;adr<4f;t7Y;Tj zhIvZpz*p8bLoJ)StX%*`ww2g56gC$I_2ll&y57*A&cei%a&-;uXM@{Zo8Wko*84cD&|5(kwwMuM6=A)UU`(05;q{A z%ogUS$0#3B{3tV37%$&Sd z?O_0qsS*s@Es992fuc`!ujNa@Z|d2fc#b~^Ca5ndq*rp$%^^1M;Vpb3G-~@%jPa;h z)>BZ|hMzg&pKDv{yHM@1XF=*ZI_jp|EAR9iY;`)~JUjq*DYAl@g~{g8HlV@mGLYZN zb9jUxCT=tNwdOGqK-@1RBga=lg)pUygHdv859yUQ2MLT*55ZM&?*n9lhxVw-P+J!u z4-TzVKk3s?AshBp>KAlony-sSKQsE}OA7fk6r(Ot^=Q?mcnOW5>DTJgut&nKgR0%8 zK^&4YdHprp?)Z4If}cBdQtIc?2#hA@+D+BU-lzMQo(qj;B z!4!pHEO&uRcc+UzZd59$GU64pAEcQC!D}d}**A?xzt!^*>qCKw$*nMeA%pgB8qAW$d(t=!OD?veQI4ZRwuB6fP9NGBwSeX%|Z z@}#=H6c$e|?O<+LITKhtQ98W{x&rl9iF<9g4oKrO`3&Xrs9wT|J89HOs|-JGG`A*cqgIiEZ&^oEwI&&$LDyLr>+lSN-`vEQm*4@Su9vEH+xxL zUyc%n*87OPsX%oC^5$KLk6ad8%A}=1M}p7}rPhyL$DxjNj(1JST-*3+=Dv`^iE`x4 zA3dbi)A6M+vW?|S#U#M*_7ZA(1^P*K`L7I@l`1ErFoDHRugO!58l03=M{*WOZf#~9 z@ze;|feDcK{+}lk9%z69+Sd_Vq;aZK$-3L{z9DTkRauZ>paTW|1($7l@dlqi3G{Go zTGuN^`R?zuGVHRd4&%+v#l~9!yaGvl2dH}Kw_+3iT|~fVXALFT(MMaSSL>o%0{o=P)%d#GgWT1Wo6hC?dluglmJb{PMxY6%WREuN zXtH(t#eMZXM`EZr7pg1BKdUA%3< z2Vv%wVAa8%yj{88?qt@aV;sw*p=qlI<$(3N-g1_x;a;{(hCN87y=+801TgLhOc^ zM#(te`K<+JjOJ9ED6iHxS0MY@!9go7Yc}FGPl}br#7!fBE`J2tmXT8PcYEEpWym-w z5rkzDx-$Qbp$X3YU%YT$0EES7R}JGR+lsBX#m312m*n2idMW(L^r_v_qEeWHT)J+O zu&Yg7egR}#9(9yqcE2+rvO86wN^h~iE(Xr1c>G8mzvlQMS2>O*uT zjXR8dA7K!*>h{6dHbVSRy0386p_NNkOz~-eOK#flUAP=robRACT_qD9*R?t2HCb;hAqp89sJUw$1nR+EKYfuEDS*2fSaQ$M$@8m8-=`d?b^ApdNZyyvQX)pEF z(Q-YhQ4zeYbX2P*=owZ6XZ~{yjJ1M&c0C6A*0=*Tjo{B1UqHDX*;@eii5NZ>ZL$UM zn~G!ot3p0m1%fo}Dcn|by}*pkRnT%^$Y*yT%_ohQ(r@O>k4Dwq8-o<*c@`D`k3SVM z0s74$c_eh2Z?6qhfLawsUSzprBk8roi?RkufNbMG{qErcWoZ$oX}w!VJeSLERxqtg zV$5%ouM0V-!UKp4*u6Nm%l?C7SVO~SJHa7j~xI|E;e9jAmcKMnC!pXe&8m0bd7zYtR7Ja$gz!_H?8MD1-QjvNM z*;~x|S$(Xe3fpJ7{Ju7H70~~QtvR?jw1j{QLSpnj*dtRBSANvRf)-9` zfmqs|9t_X}rk`*bqTWW3c^HFw6`9s-=FSCtO4tkpC>(-X2@JmvH6%`SR(6E0`F078 z*!e&|meM#B!O#J>2k~C_G9~rL*EtAPl~ob-1`J3yi_d?^oq4hxu>h@tmll20^lJea zOiOE3{_9&tbB{oMh)3>BjA0HPCR1!9N&u`h{8rPtqzyPSUY?wv>vM;*4}Slc*d3rA zO8dSo@wTBtVsVeJuY$K?jOeOpMITj1_r}{Ryg!oCX*?)4`8N0xU$>)p3oe}}aO&Lm z#mBXHb@%4AP9*QO;0pWByt*pJn->|Jq%FxzPccXu;FuQdCoXr61b3E{bQX1%INLO5 zXxz4j0v2%hg0(Kw_f>1X9i4TA*QJ5dW}T5ismMkolTtgW2M3>;Cz=oEBK?zBAI~a6 zio+ge8g{XFe9Gc9=S#^#mRo}BW_Nz^c|Q+ zR1W5OE09j8(Ak%XPeF+ZFt3rRd$3hEr;k&LXZByZw)x{?5sq&^|B2KyHGlx+{s+*} zA%Z!6DN8a(P9vzrTXVc{UfFoaTzh$nkYqY}-;6!TelPBDWUtufX}xJl7u%i&Z0x-5 zH~BW4$>-ce@)g-4yVZ~QvyXkgJN-RTR(^|12&QSYeEsgpKp2aNx!cfO=~FRWfr+o}1NM9t+5f4;?xcj~gBqPc01Fv5 z1G`_Mb?@$nIb12F5tla?q6v7kgEnsaOEf|htMg#gR)?v$F?xssOn>bGmWFd4Uwj&$T)#7 zmcXLOitRXCT^m7*O|%@8OFR`I_lt%TvoDCZo3dbA1>9=^TbmV%a(V?0->oki!J>SF z0hVH!qR|&9cs>ckR1=+O!f=o3BWx+(vo;01dvxCHxsfl?0=~hmFJ{A{jI#hzWUMj^ zg(ePK-#FPO0e_djtt3YIo~V#qKA~b<*p8EMd=xx`)Gvp%7Ou-fEkQ9yJZk3ruMj==Rb7Q1G<0zA9R!H z_y0w7FIgCiY5DzK_WS!YT>Zo)_qM(YuVshq_SRr_DN@IC!)ar9J3W^tGIL3~Y~l`G zvEZ;7#OL{cctv+K>C!@ri1^n#wGvRaTC?ZuYtWL(EpRHA2Unr`dZ(2Ev~4nN25B{x zAD7>uMqGf-$q`d-;1tnCLtn+fx=%S#?aH`ZBcD`v_g7|8!}qcaKdTdk>96I6QtH`upOu=yeVb z#(v@KM=F6*ZZm5mLVoFS8M9E9)?UED+q zcCf@KxHj@Xw!O|cL9RouWWzvE!Li>;w%f?J-@b(XHz5S)Whakg?2jdnxb2=yM`|5e z?fH_jgR7@pTTP;_KWqC3T@owP!7(j4ziWwlpUCROh+68jazOjLh7)$VPg-Wrgj~|{ z=!BWGSo?kg$!6BYY-2vU+p?&-Ad2VSJua>=uGLB?0?tN#=g_piV#!)wEIIr_1p4!Y z>-s_->WXjXIiC2Q(P|j2mo*1H@56dVo82I%rmzpS3I+?*dw*u=uPK~gAxfVAjK=u6 z158udpj=ND$-?e#m|lo)4w+WPpgJE{p-ut+EAo46v2`pHh5r}7X>m~aKWGo||8K|21fQPq-N2sSOLwa46$^%Xm~r=Y(JNYEswVcR=`|=QGJY4gkEy^U z9F4f4gk+%CzvI-Qcve8a^#^8>B;^&*JDU_NIq`ltc(o(CDk7(1ZN8a#sQtc}idU|Y zLq2Z`I0U^7@k(JhP{~??x4A`mSO%40-PCd`exnwks~2fwxq5bA9^HDVI+i8c)HA*0 zBhv*;#HH6U=$XDE{sH51Zh|N(4A-Km_eU5mcym?%DfOY5-c1ovR!`CyH$_>%PI#)> zhjKF4)(B>QN%(0Sopg^ruYrzm z)yi;`V9w%wWkt>}u77YgV7q&8V5WjAl+`DziJZLCCt|^VwEnVYqJb zv1CkY))yN*dP34O@Ksf5PRl@{P56+Q-fXXD_@b4FU+9lX7Xe$#)R@QM;eKa#RFH?c zxX<|vbP`%t`hd{vf0%fAIs3h(c0w2F?#m1Y**~P8dC*{BNZw8Ft82Z7Nu6QeIVRv0 zoP(A)^>IK^o?grPM~^8_(5FKu2`@yL-WgrEy%SPzWyIfMJ&K#v?epXz|BGJ(#%ZLM z8kj_;vyG{izt9Y{=j*n*Ywu7|>GclL+o^)=v|JUbYvPc+;YHgXhP!t#{#v4az{y6e z!>8#|t?z8uD*I0Q=e@98&!p@ZiAK03D<2ER+KM|X_c;PUslDT&7^W)K z%B#oc9)&8P2}Eu^y^ip3`-X62D{LweiEQxE;q_{D^a(Bu+4ae7DaPS2(Ijpk?+QL$ zrx&$Qu$$@UmYk2D2+Tg8VP+;Qo_!Op#n4<0Vl8lzAslx==8v3S+tH|m+ZIHW)Be-f zws+>lHdV3MUC2#V9qkZCY!-aiH<4hj2}rm`*O2eChSEM#7rWFI zQYE6Moe%P$Li#Jr0>E|JUbf@o7Are=U1Zv~RStL9Zm{UiJtGBwY_N-!EG7V7C2Hm~ z8e>Jyxs7=sU)ROAzS{|&BxY0hH;!-F$D8FU?VnxJ&C-7pCL^^@aBavm%+wBQk4;#g zvMP2PVMZwCJT4Qhr?(T*y{L*X1J5M#hsZ>k!;Stbc!hL3{D$r|3+O7!D0O;IHAV6h zr^rp`=1Co*0;@!Yi6{1Hah~>uX*W1U`;9v*noPU*4zy|~lOxuKMwWznaaBZa=_qS= zcr|mH7H4<*l2jIJR)G5gPF%!g;*DoU#8(EpBkkjs)iF;#W}(f>h;N(RN61Z!nyfmu z;VdV=Bn+*Mif0p+jLdx8At+^OTg*K^(?qs%88VaKhC}pZl5bM;91%?4Z#UW7GfRfD400e;ssBkjO5EMgc~ccA0`zT~-B)kRH}XyT~t-%7r26cUc9|8)fRe8GNUl|BcW7xyJQkR_8F> z{JY&?kqm0YU)v;4ntrU9TN3SOi?4Uq zj0@~4(rO)a?YCc=fE+kCeJoAMFe=Ruh4OwjG3p` zU#Kg*UYV!SSZ`uv^CByG|J0(?>dz#r?oe{^WFB94l8)$rBe7=VbVsNePtsQni5uU2 zMci)3-%b@VD}Cn1;5OahpYwwMTw>%iwCi+VWd6CTsXftrdfY~pH4_&z3Ut4#uvu2& z4Mefa*GFCjI*->q(p|Zq8!Jla$~WsyD

    2FEnCZ?Spi_#Ph!Pdu3{n&Vw+Mvs*i< zKIGI9wo#lUe%dToT;A^bXwU$|pZk(&CD!Z7?HN*&20{jIm;D}mCnhtvI&PbcMpyPh zHEm_1^;jp)$758X6o1_5axo`Hj!+)PePnBBuVnAU zkFyx1K<_UFsGFnCX*skDpb?J8O4Zr=etZ(&fxINs0-fAqn00?7>D*|YVC6AO|J1C>8u@-CAwdq? zsy@G=qpv5FfwWzj=+5*>>9J!P;KRQzFw;)2XH70phwvHus zTx%+O(k+y#Iwa_=!DP@2+Rv;egC|?NMV%y{Wsd=S>WsdXtMo#ppif{11O1V}^wAZs zq{WUd$KQHE`wz=^ieYTCWmdlmJ7qh|5W2c?y85#;G4V_?U6EnBrZ~ZK-*SfvLu?vr zP@V1bez9Q=Quc&ZOqD5?F}~Q;>__^8Az4YHR=50s=10j}>wM}67qt7b%XGzH@Zw-f z^{?Nf1KX;iF95d5AQGktEJwE8WI=vL3>SmWIU{vh>y@k=Q zEdEQv{0pId77-inqSy>kJ;#ymFNM zwRy0fIoWYanc+`<>ZLNjq};Ci`_Hy^x;(h$rR25v1SeEllf*V`d43zL94!p7{KzF?oS({VA7ny(P;#+1=t9QcI!6&fy^>0f`?DCh}$SrPr}^Bc_F zo^i#AcAv5eR+!6wDGbK1xEiiU+BGV36S@+N^cjd{pG8d_FS7bn8ZcfDim7qo$iUO5 zqVX$3tTQdkEj=~q$oI)3or9BW1H9qInS8Pp>#;M}O~nA0j$xjuVv_8ZgU+Lb1qQ(1a zG)O>)ShKRG+;4lj&+u)!O=goki|XfJ{P;=~a*)9&05GUq;A@3_UW8Q4Pw$&w?Pjgs zHLmVQ(&qnWGjPAr(&RKKQfH1LiteP?L~vrtxW6=wJ!`IE6@8u)TGXp&Qe5*4P0!GO zBmITNK8MKC&wBlQr3D$zt%{j^m2BC9v7N*UdR6-6x6LGUe~r+$g_goO{dsz1>0yuc zKdb^{V?JP)UJ?dbkt#Yb@Sx;OG?S-c2{?jxg`UN9#!lF)F=zee zYQ)~Z#4j>klU9g0&|rU2^mGhIrc$-Q9+%9T$(z}EsR3TgGew-A`P=VT1*k0#5FJ+v z>&NcN@`n@3r5AA`B{8Z}Kbk!m$CQvJUz4#DS_<9YFm@lqRS#Iu3$f|A+8BA5t;MQV z^AVQDhI8geg76G)bZ>Ug3cXz$kRjHkL%cYYVeA0g6t6Pk*PsniFXxn14nfCjZ6~JZ z#JX-@j-zq6*f=Y<ysR)ns=^DJ;wPR zIz%ZECN7MMm+cHn+ih;iGNK~r=!s*UtVe7s(Zn!3&A5>I(XNTAeF+1xx9q2{Z+x3f zC^vaD6aTx7qbj?+srAJ|86UC`n(b#bQXrm zrH7U2#y0fmsd$=)&Rp9SZ}ujgWpy##rn#(ntT4FO)2AfKOu|V2=`a7Ph8p^H)}P+_ zyaCQfxGDmrnsU4F%sqTrl@S$BGR|W5&FHA|d5@?mBao)M^uIr`PI()olmEbiz1CMV z*(BMRAyxPL{7l4I49UxXZ4|&B@_p@NR1U9SH@KXLO+;DcwxTH8kY8cG0(oW+4BR~& zs~9{Rji6H?Gw@IS5D)Cp*K|PGPnI{!n4d8%Jsj`&#D3YrXgNHak64K{*FyJdF-lfTOh);Iyy`3XCu~cI2Es(Z(c5_h)8h% zu?LtZD#T`_tnMvi4_8TqtrO+xWX*S`F6+DH(R7`{UMiTTtTSy0;wc zoF%Lj+2%{msI9)$@8rj8W*1)5bGLc#GHxxiYf?*}&f`52#fjLRSUB=0QeJw|S-II$cWZC-3}pd=jHZeFlS} zh)>HUoqeYL;C&Kl?W0gBi@Pu z>+TaVA0g3j6Y2gof(AIGA=(<4|n|NWf}YF)5GB)N)HbJ?yt z_PmC{ima_e^08*i8mEzqd3U z&J06|WM7R|UD1oP#Y?sL-# zn3W#)JH-%n4`W>%wHeMCT2e!jnJCN>ywL?Der`-qh3!N0lYvr2tw$N#zS-twC0Cn` z@B1Y4ps;FgurA%d$_9^6A1d#lHC#GysQa}?(+LuTw$C!{hVR>YmHsw*Gf156d9>*1 z3a$v(OYTKZ=URNk4gFFkTSZ2D9oJM*%>&l|J5~doIs!LfmAo$X2b$bvj}uy!X`=Rq znawZ!84j^}mZIH5ijtI`#MfD~N@(jJ@VV8KOG6YJtZ>e(@kaHbt8tg$Q7S@i%;eiX zm^N(92<+iZsBATp8ZR+ge5i~8I-ERQ6oIy#Qc*v9XG_Iz;+O8Wp~C(l7V@vYbd>lzQVmy^Gu{`=Q=nN@(`Nl zwv|0<3u1q8C!`5Fg*QhSj@+rHAv6pOz$4C4;>t1FGU$?m5i>2h+@lSOd&Lu5E{y85 z&a{yC2opDk)h(zh`1RV|5oWch)R!!c^N)Qw?exX;;kiY6O+1NA57f?_D@0gV;%^J_ z3mNbejMtD!>lxdYuJC|2(zS<^|8})Bp=*@HPivHZtIR0Zjk6%Jnq!U?;W+r(qle>5 zGslsFCnUmM@$IYg!@jAA`qP!W3hFrXi zyXw!TM9QE}=aEB6h@ni3tD!wf>ylAYOPm&eG!l9HK#m`$@ z#Eb6*qZYCXzH}tJWqcQDjpbRYbgyy$NXMKUD5g0pG97;>-M{q;^nxPVtdw1?8WQEb z!GF#e5wuu#)`C#n(VPG?R*J9FDjoBwWQq_cXhS39cf>N+?w+yFjmDLktEc<4kuzU0 ziU?$>!mpU^*VV_%G&2b`N&^!;X$uO`rs@t^c zyg$XAHkn)Qw`uU#i){Sk{K9~3(TF+OYH+deIpnx!JkYH->ngSz5?-hZMP&WR_pAD| z`oi9NeO}mvf5EYDsY&bhe6;-pMk_nivs(|XSa>U+i||^42YlLA#gWvo$;r0Yl*s?+ zqez&eN%kXI>qC(KX4^+y{Bx9cXEUNdJ@-&yXdcUGP39o|VSEU&a?Q`*H<6IxU1qu@z*;v5%2!y#u)-YeFA9k_K7Iw4?Qn^BVWbl$ zrcXoF6Z|ld*JPl%YMFg5GG(Yq zarfU-wrwuaCUy_jdy3u;{w`!;b)#D+JyNQkgM?Tzjr0{craUprJyR*zJmUqZwv3McCxZLC8Sj%i87d9N6X(BcY+n;@3pQk}3KImGo*9YG*r+oT6A4$H zt~0XBv&m1lA{s7iLvE`iIiF8ArIn<)V3wVUzc;FGec|fHbscenPDJ|hF4>1LvgD%4 z-kbwM?2hy-MN{hO>EK!RQ&LEu@zR(NKOqtm)n5x;KmPLXF8T$NfwT&UkJ3N&Y?v$Z z2ZpcIKCMA~GFHqturQ|mPg70aF*jmW-`d9a)XSYu!L?cDF!6Gn8WFQ$`sP$ooX8_Q zU>x2tK6MwHcj1%sj}mx>;{sQnnDN+(@juF|?60jc&;03yP$s23KO(|ROn!>$6B@at z8*N`)y2c+ZN&ZF{F%=aW_|rP z^3zAhpWjsuM)l{?qjJWwg0=1b=(zW_7A{MAmGD-0cjU%(LF}XGd9`f@CuZEG^Fj+J z#`Z;xwUqFD(v;oN5X+pzWz#=onQhzFe~(l`Up3@>`A`Mz)uA|nWV)v4POn}|zITs7 zXV19o$EeZaALAv2)_NHU#{ACH|FJRe6^ia-rkh~cX1{mM;q{7?U(|sxK|<$iC+_`i zd+&G_VeyG8iX#fXPoq?8Z4sv0O)*&-Zhs1`l&h??rHHXb=8EAZs4B3dk9Kkvkmb;A}k^_Az8YWqzq=~uI7M&iHv#-cocOf!!Z#iBE~Bwwt@KOPjL z`&4%HvuEgru0xu4Q#c)eu94eBena6a{t+h_r(hpx$a|!HO5}>sCeOVv!4&VSyQg3U%eYv z%9gE{BGl1s^cp{mYh&H#bF*lrrV45KX3n&{%1+?VP&jI%>a%O8FI{o9NGit8U{+*9 zZNv+2AH_(pQd*jPvtQ&|tK<#Q9i(|(AHlJ~-K6w^=N#TX6hGXX`$8LnS>zb`cf>%> zUKoXL-#}oa2>Ub4o52Bz`P-p0XUM5!SHZ3O;*w-&AL4%rk+1yM7Aaky%pa8VWGUg< zCJEFgPE@c@uj0 zjNg|_=(<*G`|#%doSx>Y%=Zee*ZX>PHl{B$SbX4xp=KhjkcGUVgwulI#UlFFuf+Rw z5Vt$`-fqZvSH*tcb>xeUOL8S{zR+}tczDT5%1A?Gu{igbj(l!XP80vTf?^~gbidBP zcjH~zgHIMEv@F8ES%{vV=+dhuAuX0k$4(Q@9nS^zDscKX7xj59q$C>_#GS(GQeEi> z!Ie&P_bBp(2;q2LTZiHfnImn>G!y-dL~Zx+G{1M|-x(VYo+hJ5hgl;(T|XREV0;ev z+(X`jiK?}+GE$;EW1qEHC>yCldKw~K;%C_n#N$h4i~9l^ua;aY%Dmw6x3>lPaKpl* z%)mM_)A;*l#v{+6Huk5Cb`yMtCZ&yHn9f8H?`n3ZC#eaNiFA}qLV*`T$jx({YgAi(zL4capa??BgFeRnZpypu;?PM#(lpC9WG-}<8J$s=}s;^;7R zlouq7o^#}BrC`NhE{U2j%5K0XzZn5cRD^T{Zn4M4%#EWj6{4%Q5=NzHWAwG9-k}yU z&DJN*Hyp2BMyqKb0AGLh(=19h35*@3`1*^5t8^%-M{*+M+|`TwtC_-Z-$TwzQ?g~r zA?5jZ*bF@?X-fXbr1^srli)r7)ZrUeD6QjDqVJ;@YC{z8$t9N1cHUCW*2`KMOA47l zJ+ecj?sEh`r;RE~dW(}zQi4)}zf+Vbkcou^A9VfQKSd9f5C@K5{MN|*eRS@CjOUY0 z3X>b!p>#`C+TtB|qh({-P5_(@m--ewIw^p|oG_lC0`5EjuAXO?Y5To!9E|G9HYs1K zJs|rP=0AZ?F+aw%c!f}eS)-=!ZrgEXH|mzEV|Xr}wJugNrTuA;nxk`Al6G8QYS{mr ztdxcNf1IpD&T>p10?PFJyE+8rzzwU*v&(n1k33kTrVZ23UfvptHUUn z-sB-DlXG@?@E8;2I6+eNrT&a2;3O^m`R>D9n3yH&$^p%xTdHv#W1s3byp>ZWi&}Rn zBJ9H7ZKGoADx`_GgtXOifk zY%E*Wd;v_*YLHoNCj>{60aP>tDJ~0Q_Cl9nK1|;*i8$$_;8?YS6;rWFuF7{~2&@nw zo_5sFKNENF`7)Fw^YhkHIop}r+)Bu`2&-nZwrClD% z=;W34Te9+Vg~6iI>x-Q$*?sy8>$84l``X3N6>54!N|L>ME~N*DbN`ZZ$#yMYq{ikl z=DW#vIaVh~%3DNfJNYA4Q=f~kP_xIk?Ra!5v6C(CXN%JXg?xz+*8Z);aBCpWwK<57 z#Q)AVF1^=ZY`2ZZKPNoAf~#nd8)QerR=Yd9h!QZ?S9SHO@7rZxeC%;bqv# z{^}0S&@@fVFMo&i)*JbbjR@r9brXz8Bp1kNzDBpm})ehD8XZ+rtB zQb%t5&3^z1g?b0bk$2g@3$i8Fv!Bm|bQP@N*o@&MPPkaH9@V<&GVy4Khd%dSl-l&F_Oc!!)aQgv+WiR#1kjQqXW6Rx5o^2b3!*ZtYq6+aE5ZR-TYr=6u z>oaaERzpx)&BzZ;*4-fQ=of?NdrN~x>@DL1g2YIzKxy*mxa^jpjp-pKCi6|1{~-}x zl3il21zaDY3a%MvP*Sx(Ku!QGdKi-~QHk^+nMe<4f}T2Grqq5#dfgY&PCHqh8Gk`p z`3%h8P!=q`8afmuU?c-rF8lyY=&~ods#=S`Xbp<2$zHNd-E2~M^=kRrz%?uvrmq>R ziU9(Vse3Ly=$>TT4^wr@qR%FHYk7IVu*1&C;w*b&gztR$LXB;O%UntIdUkHhxhX_zwj~Oda!QBxHbrtKivTF}Kf4ODF5zzKr=ey_D7FxaHs^*~XI2o72H6EF_ zSqvbYO4B&{`c6{+3zUE)VE*KN!T{nhH9 zh_6&aXCGeIJ)J&}jLp6Up7Cr%yQ;%NxfFns81YuiD&{jH7y<9n)ULoBzTfKs*{>?oSj?**ss9Sz(i~Pnm^yiVI7ZmXjipzRB zbUha;)WlTO)5F$RmsgvdFP9_~**lhvW1W2>Ex_x2#xabEJPYO-u19e62EyKKBn-)_ zzF7?sZ~;j90(<;w65xeHK>y#15hW;>U%GfIaT^sQZH0gl(?X^E^y8hhMVUEeO}2lQ zN~*KPl=hmrf2Ye`fU2+jTjV)c*(%{UBZtyDWrzM^RXsPuo7V!0WfIVyJ4xXQYoBr> zy4a0H`X{P#%ceAo*dghvia~(fHRb$Y6IzhT(?( zZY)O)#d(azWLzE+vN_XWB#djz{UH#J~9d5!R!?lI} zstwX)@if%!qm723bN{4N$Fh#$`N-wS7`l{)y4TG{Vmy>0`IB1yj`0g+CqrBat2R`u zTks2ZW%W>W+)I&8PQc4w_K26^ZXguIe>xwdijLE8hwCTU`zXn7G_+2T3_eN#%Nxd8 z71#N>;;_FWNv6$}34b+Zthodh7sFP*Y6wV-L}=^iMt?z7l%@jvh}}Gc?ArGO*&oUCrbZ2n68=teNfW#wsnNUN5Ep-jkac=I=;k)^In337 z2qeB=FM*tju}81>ZyH-5|4{Jf&q?zA)ECuU5I$crf;8A=gChS~$V1ZFS4Kd~%r<9f zV$;Lo(I+n6IazJzQg{x>9N=ThDaGJb_afKD&j z-v0Us<@6#+En!Q3WC3;dzIk(VOMc{%sWnF_pIq%=l(r0|k7+x7Qj?|6t@uFgTpFNh zWgXi(;dO~99FC3$`h$GqM1%L!P<&zMHB{kIz}UuY(H9F0cg-$Br>_=9`yA~t=ZEqP zBtQ1$yX05#p5Fp?WudqUIt88k?8Ll+Q6dA;LgfXDBu}PIc#R%CNqBw4!5I^Z4tDrF zF1)2_P;A)3eN+M%Jq$v<0I7&@k=J7&PSxkZmMJGD1oKJtjW)8-C*{r3$Wu&Jx=6KM zGf^ipC%3C02DJbs9Yt|WrYY||iMC$4ZnZ_RNR{(xx~4wFZo}JqZgab~lzfNbGw(bL zv&GJs)%Uo2U1%I-)H4os_)(W3S`HCFT?4r+lAR)U`Xg`IN`cUs7Y% zu~eFPLx=T6T0&Y$oTI%^D|9FE?T6w5!&XwUuAD?-cGtAMeLrZgBPF;jgCg5VMz!iB z5$*cw25qk9aGW8oYNl*b=goy5@{jkYc5mS4?=DGN60q=_*#roV^> z$H*ycPOu6Czf)ltsYlHy0T6N<$v+nf?+6%Gq zsXZiqZA!rmyV5_ju~g6IxQJ=v^D!#0R4vO^aBSjYyv4VD0J>t6$EQgQfVp%Vl3Kew zO)N^GBGw|JET=O`F&qC#7lsMirVJ5~AAgYR@?U9*m#R*Fg(ts{Prz@OZ72^hR#NuT z!?mxrP!7Y{20AOROj2mHAj=y zKOiNn5Z=J9)4q9uBh&X)?SJqln;nhe&4u0vkXj@jDLwOk_yXon?PCt#-V!2R_9AUr zYG!Ui>0J5Hx`W{vb99@u zYzaEs7n&dAe^1NhH@?}_P00V*0qnR}^q43el|9KuTZ3%Wn}#jX%duIE}Tyo4&tX_aFeB@;=ykow6Nwz8DBsih~zgRqm%{T;9hrK9su|q z4eP^H$#aeyUi4RHzkBF+L~1%f<6Edkk2xHV?(-aZKrV@uwhXK^Y5faQwz1&zs zGT5}n=OjQ-wBDUF&j5L?!x~S`(A&1743yZZ!PnATK7nS?u?4nUWw!pjOGq_x&uYFA ziJr*ENz`;Q#dhjP4k#mfocYL4ISH-%@-M}#v8}lXsK|M z`v3|=+T)z@@-S9D}%*w=j@YO_dI7TH) z2A6mQ%BgYthnaup(*gO@ZN4dG;Tk)E>m_rJI2&yaKgEbsZL2A+Y_$Gk{d zZy(<*h5BxO+xaYP73WgDd^Mji`#Xua^)?E_$^rX$2DYkz8y}*=li+9+z5s>f3*zl3 z4$zt;_EW_^O@JKSE@0;a?3Rf~@wI3%P-v?b9<}33e-L;nsQ*2ezK6QjL463#Io-U% zx0RnsHjUe_KU|`$a!?Z>^CRsl`beYYC(Sq1csTLIu z{&HtwsW{<+Bh>GVX@OTv*?x&qLn~(nkY1I%XHtSe71-TfdQ4R>cSnHai262nl)BNP zHvB4K_I9*7wRs){E zzaCQHx`nxcI1@~7zojL&10e;wj*VZWVidUO_)Kqy2n9B=um<|eyhL84AQjEUQ#q-Z#giEV)pWg`+i z9t3>oWZ3}x^?89T6*Y2+S7E0ek5j1xnV0~-T0bssNa6a;3iAQaWj9CoE4!58mIbyQ zbg<*!5<9pk%hZESBe8VGe}kgkXk3FepB=W~J`$!H{Ez!jz65?x{Ful~B@?PMx$FKW z{y*da;vP#^;fRo^CZTNm?^;YiZ^o})(o+w3G8R@TJoHAIvdUFB$cu}7hx)JPv)t9O zrq(*_egV3QQ=zK#^|-MYIK++CVYIQBV} zA`9S1R&1KIRIUTQ-I)R0pzHK@i$+hjJ%y#twoL7yey0r1Go^*`p>CuHII0RZ#!qt- zrVL-NQ)LYeEf?&>b@9?xQTaOgSHIr3+fu+uu27%qDX;cALztTFZ6`}XvJ^R%2)Cfw z6)8wrM%Wb=f5)+Hkt}oNXae|PO=JX*NX$$0y`x)%LK^_$X8XRKs#`LN;#^N=qOAEy z01!9z+djvQ%==rVBD+uK<-t}p4|TRUrX5yE3fetQS>@zgxb!E@r@bI5o+MqtP~m4@ z2q#El_mrei$%i{+j7k+R?xmb>6xP5L9Crn|TYJLHbBpbzfdi%p{_V!pTfQP6ssG7g zDwTi|ab80e=qdI{3Uwcv05w=j>Zaw{ElNs(^SoJt(r^UK)P_9hBC1ccE#F_pVew5; zNPJC5G7rBZBI(7|MOQEkIZpL$?VkjS$?@(xom78(Jz0BV`4u9&q2Jjo(0Wsw>UXB2 zCDC*=XpBDKZGsAIo->Ck*+JwgJXxaR^g|k)NkQzZGbJ_?a2EuRWj)~*j$BLyj=J(c z6xbym+2%7gmZWeqRp!wJ4i(K4Vd(2DPsgT#jAgDA9-xxZbK~t=nm@kclQ&b^S2#}7U@2A7WYV{mnf=*a2${*CXj%l&Y=j)if0*5pV<_Ym4fb-n5 z{=>X8tT2wawAD3Xw=*z4K|(@mDx%$g_(URKQ6+&eEr^3@J*>PqfjiYO>!+l{WC*xsNN2&-t z`5uU5ZrfWAD$!_|0ZhdTR!{?s8quxElLi#;V50ptpT)85Pf!K+Ruhg!KzQ~yC#a#3 zZ98lMt34my@@$FeO}w1NsQ2L0?|!sws@Y7_mD@#28f)Kmu+ef?*VJ1ioLko;ouXC% zdecC8Flc?=u|>Pv$@h8*VA~h9dD1NjQbblV;swO01|tn^5y19;@vN&TJo%>rB`U*0 zf<**9FW|yyf|dyNj3+h7mq4(vZM_R*96CSzKO2H<`~N9Od3-k;hT8WRS3+kdn*s{y zAM+!BLtA z(SQ)#CH%zsGSJ}hom3L{mpch0C->m8HdTNpvcTuA(EqPV*LnuE0E51Vph2EtWR#e~ zI`s57e#u?9z_+iKZ={rMT2#7+%xykoOKNp@hcCQLKcS-`RVLHI)4!8-Z!AS$p`j(m zld-|z{sg0c0}@zeq1#B|cL);Xl#a#@2TQ39`K&+(Ttg&>&tf}_jd6njvvtaPmj1h2 z-5E$F*-oF8dy(^#_*}di@irshP~l^!w$8-YQbHuT!0ler-lBu>|Nmh#$(gSJWScp* z8*aVKT7}l5d!p3M)&qUyO}A^ePlRxRBI%6XtMMH&1gr})IF!Ird3|bYBE=>iGzO-h9%larh z*xo_(V-sH>7_A3^lH0K^U_KMdH7X&-A?x?sO4hVmdaX{#_j}R1&HJ|50pXY);0SUH zfEG>Fe{4p*9p~5o{_M}+0>|!?ZFwJ_{?%KWn*431`?U`0DxMV;8j$lQFayaB)VJj` zIp<&unQyZv=|?lU*=ySyd8{WyX8K8y{Ce2s`0jPf){Db?y(Xah%+jeeBU83{X-rN7 z88yNbD{suYT;%pn^uTRGX_4Q8tjdz#1w%oq<^$?uAy3|NQe*YnY=MyN#V(mntUb{6r=4s((~jzQ!Z7QgF8*eW8Df z+hxlKc^xtjSMHvLUc@`}liG0x&MBu%>$i|#8<(pqnVX5c-5Cho zxex8zy5JkfYC2rQVOF zo7}92$#?=$5R*FfrpmOF9^%&QhT0_+$Se5<*3IN}3UeBe0^5&WEjt;IA!05hEwzI8 z>ln~6Rc4ma5ZKZv+86@-mSt53-AiX0ZMzrB%7_R(8Q?IdxVO|g;e)SuAxk6iS0lE{ zwex;6^vtmZrx=Te$v2Od^9}j|sqvmltvv#cj8I%=?m*`QWZs+q~?W ze~$U7z29nm16-Z+0l(4xGj7`HbyCm8TujmuH)0PYwOw^f+fJ3weDq|d_Q>a*bM#%n zM46a+j6{O!Ie)4RMI=vvRwu{AeA)WTMv!l~F#Nh%b4>;myXp}Jjc173lUc#LgKyHt}x_297L8h;Dr25Y`wJbB1Cy$zHg2kbVs^xK8z(ZOpHWR(7ji|}qAcNPm7f<@A!fV=Mx#fvB$mMrt5bzFAa-T7 zPf?u#};6gib0y>AQ znk3tl1QCzUgL2?W?ABvw$kSbcwlS*^wO(w3A~eNKZ{{L6stQ7qu45Kxe_MT+x3}DC zumXH!O5NfThuxOiAVvnQv?79ke;{{S38*i3hp+Fk^BC22X>QGWNukblFz2T5?(Kax z9Yu+uNCuVM zX09k8qPAe-Ldns~4nat567v)2jmP4Kwl-iVW6i}=o@&b_zu;xzH@_$V zICpt7`E2xXeS*m*BCnF0C0#gNc>6s0=g)~D*;Mn1Cg9y}%I*?T5AOR?uzv6gC3o$~ znfuo_&5Lv7(2>N!EsCjW9IB!x9FVQ0JPizZZF=; zxb%;A7uBiMw6RY}fNJP9P;$S#ypOj#lW`Pq4OA^=M%3ytGUeR5*iOHNxs!m4O~Ps4 z6;8GHg?T7&e;}DZwi$SllLpnq(=;NO@1Wn#a~a2{|2eR3?V-Sp#cs@+O5fY53Al5& zgJw1_9sv())cBmS-wvP}VnbLIm8)oIn~y&6Gi8R9$(ZeRRTNwtivyyC9*ynnK9q3x zw-BH@ze4l$R`SV7S<3zYkT$d9M*ZTm@d$*ipDw~~P?p@m4;|1B1{ON&gT22Gd z`%&>Ys-tMz4#OjqoePRI6K_#L@tFZ+EWCg@psGyw6_J~EVWSsxv#7oe=}p`D28TRJ z*w#^K0bj9>qiB(5{*JE|y&JhhW{Q3Ic&Z-`@Tf|q3IgXPmn~1qUNUF_n$Q!-R>=cK zQbT`}V7B;8v{@UKK(AT*8GWVEh9wcG5V&sf`=3sUjaY{EU8UX%t||JbzrK*<{jBRO_d zB%F0;quQ3ol_sE`j)Eh!aRNORapq;@Wskx0IovWD4XR; zMtNkm{R^CuxH}yCk7AU!DCX(#ryMH9Bv6-Qr!6~@h>eECFZ{?u#OfxE9!uSHk{wy( zF7&m%nm>N@Tm(JHK7M-{^?z&bUSp=pE(vN|P`N~TGMYleaQhYfO(@W4+XYyYd7ODB5X=Md4t!hHfjhj zijqU>v}d?gkg`h}HG~1UId*1?@5w$@ipbR6_Mf!Eg>rM7yzLK7Y)DhJtk|5yDF zoWH}LsJVvcAdK!*y0K0mAACc{)D>u{@*l->J*qiq-2}3Eb(lmgMg=bug(z=wz}CI|057lFG$&WwD$lLiIV0m zmr4<~QtTeC)Nm6!(2AttZ9z7_o#>V%`JsQC=k zu#oBqjweeU&`DO_U0v9`tJz4wl%x_tx38=|PFjF6C&V`pZIkiGXfWM*e(C!G|L z2q7bTlf5O`JA0G8v)94zKAppp=kYwB-}m3|>-!hyocH@4*M05#8X&KdW4z5Q%K{)< z=seva-#c}t5)UZX&**R?e~E0Ylw>wI(5tOiCmo9Qm>gf=GS4J~kkb zb#=Pc{*cO|4i$?|Rmh>Pf`!Bvjb|)LNY9gd9ysxSe-gD)0Jc|mq}+~HZif=?riSMe zktIP-qhTU9j_-?uFjoPHUXuZZPzQIX^%dB-i%hJ~;gNq`LCGDBeaki!SqUTSd|N-P zA<_I@Ic_=(Ak46^sp5HdApkbPYY8X;3@C*Q#h>z+(K-&zDj0I(SR$_jk&8SErWXVo z=cW4b7h~N%SfeD33-#ZH&IxWxq9Pfh=m3%D>3J$y0I+?2(~i0|0^Uhr z&iuBW`U%e%AJ#M!KyDmH@g0>3ato-Y!q;KEIpkpDq_|Hg4v#$38TEQFpGFRftgaw! zw;OFwMU;Qz&<@Z6dh@TSD>(s-p%Z_#cO(_|uA+nrR8ZUmg>Qujl}fyfnPVPnrhJfH zD?K1F;%xhGv_}M^WUhCvG4EhExUH98jMlY@)J)viKfwn4u?C9b_aR1 zRwXE|%QITZQb8BM}MVTZx_@0=L-ClX)0X}8#)(5vht92|Lq zLOdfu?cJ~d?}MU|_HF1vo!;;eo86;u7XqMmblL0l9p;gpx1m6gMpvoFLRgZPfI-!XA;X&Ef-GB_ZMZm!(<337o@{61ifzNcbOmKBs62bUCBdqRk2HA%) z5C_pF06FZdcZY89)=>Te>*q95VVrM_uiPz1K}@^w40MkmU@tPq&Rn2s4-$Z*1E~hef|aOTn(Kv#fzIyUy$k31CW+ebdFo#lV4G9w6nrkxyOap#Cq~qAm~#ef zrr(4#kQ(2Qbc%y-%rGBvWAyCq)@|lyW2KprAU4|Tl!YbGI}IGt;Y;NYY4HPk5yoQIPh434kJ z-*w`>zWwIfU?c=_xCR7NNQ)fdVQFSvu5^87(U)OAh^~P zPrrE;Zvz!2V06wVd~!DX&cQG-U8KLv9zO!JFQJ8+A1N1f2pQ|mozlB9B7?e(kHLTK zeEC-mbD{^vo&JMWz18@-$>6QSEB)~EraOY6N$THMJ@`r7`69|Yd%d!d3=1)6+wUz$ zbiQQ^J&fHUzwnS3~$pUtBqSlkr6k(yDa-jEQ=)Bj$xG z3a3!MHsp+NRmt{CG{#k%?vZk!xS@OAxm%HEzw>z*JK6k>m^->-Iy#m`U@12UMRd-) zmh_;w*w_`YXPkqJ#@sa*c#Z_Z;TzzHuD|1lI~3MthmLVbO+%1pIf#W^UgzGkFCvGW z9LWGthuQ}am4U!PDSM&qvn%q6YFeD_8)W;09UtG_*f|XO`3rvF4n)CyYt`z#A*-jO zLMtl3=+=4UOn5qIPY$`k5#WBm+yuvjX=Ou&Zv4G0COa0tqV`U}&4|A){eexL4a^N}u5gxXVQXZ&G?(Vr8YIQG9ZEfS$Z?(`NNc$z3V$;J{lvqK24(^R z%*!^ho54%aPgi2O9Bz1r&(Dy>Tj(56gW64TY~38i|MTmVdr+%3U8d zQDLgAZaH%OZy%7YaPY&K6Yt$1|I0D+2dZnEo>`unxYc6pyI}Mg^^y$)1@T~ts-w#z=C=aj=2!UQEHxPI8UT{>7~$jwDTs(-@|Q{@9oMmvQ$V-j_4MXam%09_Pc1I#Ei1ic4Tg5~k%(p+%WtHe zpVe6H%p@RMcU?dQBOk-0V=|5m=jt!!+OiAdhW2;-B4yDE!CS@WSKcx=pI5A8JCRWS zq-`=hDlziC^Ky{!8s3e2aejQey)7W+gJ=HFhmHxY!ynvVKacZs6{(d#E9^HfF7)+7 z4&*-;by3~@5#|i1fWN@y{!R3|Cn+O;>>R{I%hej2&mYFs{xQ(;8HCOR3Ah?e zMPE@hlY3$-kW>8C5G2Yyd>^utILH3ZSp|AU)ataSje(IVBg={}qC$ zAEK_gOJF9AstmqPEnZS#xk2MX&NZt4*Tvz<3wrhW6l9#SWcm}THt4@@RuhjZM8I|A zEa|i{bK+4P;@{Zy(Z^SB!LWXXA3mK&nRLe?_tz{uxA-x8DF|K|{M!SlM;Z0d9+7)C z#*IJxA8?qy0?xXy-&Y^RN)3Xxq8|yeMA8YGj#hT^>rtEM9+5j85VZ;X4?cq)3^q1W zq=j*q1&*rxVBW_E-=$IaTzke$FU+nx+# zDEajb6gvRD$ZbKy5onA0bhaZ2QFY|SIV=!82J+4F>RGdoTo-?!_n(+{#l6B`Ai{Ar zi9)aeL~#&s6gNX1f4;dZ;7L*l;%}v>mCsyHwF4X`T)|`)(P(|F=G}ot%0y1PloRMf zVU{tPouyOx3Y25G?{)zcV_^S|lV>sP3L+JzT5LeF*t?q6`ixblCk<;e?PCu?k9MrZ zXW(3()QnP)DpRi-2fJ>4;9$`|`AiUWKdqrH6YFQvmHKA7!AvhMow{>kYdd z>@m1)H&g_;%zvr5b=L_bo1$&XITQAR9$fsEHC5gObDjTZOPh7YYh)d$hRDa;EXpVd z2ETYtALo~C*y77y1q}u5REC~F4;Z=lMogU>6Pq%(=F zU<@?onzJKRS`iYX%(tKMl&*`BM~2&b(cxJ?3UzkWGpa2W7NFp!h=u#S|l7+dL`mXfzEsD z-QFy(8aJY8O~FmG;N*?HYuiJcWm5-EU)CwbgSM(nU8>&_D%~l!`g$5erLM6a?|6NfpKPDGoFPX*pYdC{5t8=}(vkOlqr@W0JDaJ+X2opeb zE^xrfp8x?mS~*7Vg>zY7f?6cupaSLl4lB^$R_R${I@Oa>_WZ^$c8kP_r4YjiLT67E zf~v+hb7t3QEr?kUbPWi220_*?-Y?~iJ`)BS?t$@z?=-14;Fw@Rng5|w;h{Nl!#X5j zuAi=`me0h8QGar6&@B4BEJTh2)hfu#OgE}-jHZ8*EHc=umL;QYd0_x#3|H9Kf_`-$ z`QC$28!nZe;rnj;aJa9K1+}eVGl7A)xI6vcuO|)~nE%ePWxV;{CHRTlbcl^~dDw-` zIURNR!4Rm5=X@y~VWPq090uuk zOq4Vq-fO{1cgzUFpUF0>r(JSPu^Ic&dG_A3v^_1at`QIibqldN{d!y)3mRs_=Cy{N zok9CyGkr%($}NVRAiHF2tGbllI8xr+Pv-*vW3`e|!Sc1Bf|IJBtbvfOPaZ4^6hpE? zbe!3L`9_UzA*UUmR%Il*5)%>|jWKZ_;Be8m2gF+nPg@F}j!ia)r`$8>taX%s#3d#S zx#^b%E$5XVil~mcR+^k{Y53{sE4LTMm7uqf8|VQiN_$hxyf;foQ-j&NsI50wUe9iO zA{gY6kAm4eLt{fhH}5V92cA;URK_$I?)7Q4*r=h}hgeaC?qb<&x>OlkfAk>es8;Sv z%IEt1^_A*$RRY$RNy0pxrxJ>?lnNTXh76?n_SS?5G+B&0sO-xYKPVo2yr*kkyukJ} zFeyQwP;%0!yBjobH;^=dj_#g&F=uRMz`&#E;QjPd=!XIK1Mt?e7M3%IV5m9QZnZk{v^BHbH;qJq#r=pVG@>j)edibpX(Y81NhvPwMEGAy$xp#>^ z^&eM?X>6<~3>!bbcU#Y_yxd^Xfv>1&gFDesp@}FNtclN{b;FsJv`%#oGuW^tf@Gj! zIMKR$mo1W0m1n=6CZi-evvZiIz!h+*Dr=qc=IC`S`T{Z)2i|O&_S=eqb?SHtvjeN1 zGHeU%NpGFo_~6bHLh5jez8%mK>#k%duBl#T1R@CT;~1h`&-RKJiER|9u}b~Ar9Aoj zaOmeiKtapc6`cRB8=a+Z&4$DZ_-OaVy_I+)!6+t^Jk80Nr$FbGxUyUk7pxUJoJa@i zqzUg+5uH{XOnNcFOzd5qt}q>vkvD-a^ekj&P#!V}#2uYIgDU_0>>Ye=8Os>QYO|pd z4yc1HX3}?&QW+0L1ShIk0CKsZdnpgCl5DO*nZ-j#7PAnfKefe(VD!u6e3@X3#WeP? z7}JJ1_KV!=zJ=yiT(pU9Vz(R;*wGqQ@p#(@5%jwd7YO+2c!QDm59x9h!fo@|-&T}^ z8P^rfHMmm&6-*&my0pR+>Q*I9Uc6RWBVur(yX3dTao56`?!}e!xV4kNY^HeuHgHGk zlaV?+P=e_yZrOZR16(sXZA$*px@9VkwVtmQ9qa~|ooW2dgQRLnMB~U{dQGAZszTY* zCbC^Kq%_&-`?1^RfkKejq6mGv`~)Jj&aOSY^@)pl&=_;{jH*=CY>jOu)sLMT+x7Gs z1+CP}$}F|UaR2QoqdCl8){X?IwEA2Tj#6N`|4fR?+EVjyRT?O&0&_k4qW6=S0Zx5w zcZ7iP%khz2ayPkcI>&xP&S>-&g|oOOEZy)$bMH7m&7ILP==rXH*93vm8X-w84GM>h z(76zNr|LUjhET(*b@1fT?M>}<@4!0Fe_H6mp@q`HI%sVx`k09EbVU_m-i~m*Hb_Sb z&V!;D8uJDPVhM6)$tG^a@^9M1!g%V%c4qOeJak?^=D#=S>7=Wzzi4EMhs#q!TR=6X zGKWy+9A`>b(e=yC_Dv1l&>SIV@}1n?q634=m+iR`Y}J$oa}a)(1eC$r@Gp;U4s`v( zw6Y%WN3_vpAq`sxJiC_>7vqecr6rQT*M5_)ply_eQa_%5GRapxo?R&yS8i~iPx1n*RjQ)}WU!sl{|f!) z51>mjCyeoRCRMf2GgF&_hnV&snC*ee3yt~E_5;<^>arARdLgupc=TeGnz=+&irG~7 zrFe?M%^|Wcye}_ME7Wv8gMmB?-*BN~8lmpCr3a~tDR&HLK-fBFL`?jH95ytp8z9|6 z>1!Hpc-gt{)JhhZN#uqL58-&MDlWRLaw(TfO?$J5%?myo-Jf_mR=sT~%K_`)YPH}u z&|FB`sI0Ldp8E&GJ5s^deEBWn{SZy@Ch<$192auCi8N&k{ALRDCUURNc*ib20b#|K zu+H_Rq=X2sDMwcX4~*9~$(1eiczjtuK6xADpD6b{1##DScwHODkXUME&kQDr;jh-& zs(8`8fx@YS6g29B;9JC$Vv8L)!C7qjoE84F6`LFV7V1qC@{hvbH{`KI#~YuIh%#6N z?W`&9$4mo@)bL!ro(?q}MAxFVJR%0{Q)iITc9Dpgt%o~sE#pb7a?(!XPE7Y~DG@i8 zVxs-~@JH%2LJ2W-(waPCc&k$>2GgC|$~8dje_v2pyVEA~H3UECJ+#cZu=SRB8~5k* zWv{1u9_Sg~Y&(SsG#PXi-@IP!VKyZAIc_Ek(1Dt22EZKHx}^Ibd?DiMbQ3O954rdi z56O%>f|T;x-yf>ySiHBCg~rwe+@-nIIG+m=SK_cM1dU=M1rRksDjtgoe&E$hRy0k} z(e;?fm8}zxciv7A+Lmhj!qi{$mM|MM=7o>quIh`;Cf)WB^Z<3a*r6)w@KxgwfqhWy z254pxu!lj~h=Otz^?N3tH)v!*bJtcGeXrE{DC(H}hi$~w(v^~_R0g4pIxhlc?LsWV z#g zIto%g2rRQA+p?)W7=wj7t24^2YTLj>ocE?}F3@ z{_=d6>xK6pbSD{w0a)crOmvKpHj>WMX^Gh%!?>6P({QL&(tfy0h&mP7%={Rk@0-30 z6*|kIc2Nhf2oLrk-5XkF#h2DFuQ-`>=00R1g|1L2O)i1$fM3XEU>~%&-Kv7y3aM>Z zLI%DtE#q2EHijrMWoKzuyl`&nvufel+@b_J^osKrL2RB52KhT!G_xFu_tf^1;kJ~8 z+k*X!->3Rz`Y6SrmF{UuHBu4 zDk^=nFeKcrg&2i4M}6ih(gW3W6y5AyEJmNV`o9wy)T(gfMWA-pI`(>4TPZ2;zO2eo z7~4M#%%}u3M@ATJsIKf@8k3FCS!Oi?x^g%$(ju_MawY(sYVJ6*U^XS~+ z8DX-WO;Wo)kWz{JgX9rUA8rZSoJFCmyEnRhE;@$OSN*FzH1kS8&B_~*P0fFD8>TQF zl+@&=))ClZ+m-EKVcnhTro>O9pOp!Ub5TFLMU>W^Q4j6r4m+cOKgkfv^&{{Cx&!A+ zKAaS2SeJK^pz+6Hi8_Dj2^?++0S(3!Wk>!Qg+lsBDeCeJlal6p2mYiy>Z|p&>Z6Tb zj`$#tcYphAA)VMMinCaT$@P4S+K!--6gmwUS6v`X4bG`BDyfl7Bml}(b4?eX$e~C+# z<+5I8lX(mi0kN6lX=qt2rjcbwG-XeiL?CD%+{a4mFJSzt>3pabrWAVemZ)lGzpP;L zBylR;zDgOj>K!bQyQ5J??TtIOaIs8nO^e#!rhnC}HOToEDfeSnU0D%bpL@l?W`ckD zz^z7E(I!SfuR+xh_w9@3?%T)>WLi<`fwoc7JDL>6q)@tU2hHus#@KuBhf^CPGU;AC zNKPr#!%W&Lo{kaqr-7#f+X~O;Rp`6h@e(0F zna&-QhYHcFq?sv9-6=0gQ^(|R+js7t&d__zkkXn)b$S`A-_ZvWO9I4!4^j3SV4}2H zSuy=yO`!xgv^u?Vb8k^~$42AIeGbblG>S7~l98;Ovb(Ucp6!60*2ko(k}0jPcV8)` z#E9~~f~oo|B^8Uw=$M>=oAW;s3-7NFW$&0(9oXBC`LOrdsTYp>i!YWRxH`>d=%I_) zUZSCh0VIpXjv9I*e4z*yUa`5%^yR~o~w)k z%HIA{WzbX>S6DC^z2B6W&W@17D%Nb?vWh7*8h2%9E@Lx{NF&&|RnnzbV_Vyv!`7~$ zL+K9IHG%yPtQwZ_$t%*J1q9y|O=_OdDq|3zdHgt#eW84b3U{R=J`FFv(<6rGM{WLy z6%AP7!nY-&qlkQx8G-NfW8RlbxB4xdejgvUZ#~uQ^L8@SqP)CnpQC_tU3eXuYg_td z&tqo}p@5OT_%H25GAco?wj1g;Xg*_K=1*8*+xYR`HYe+>0o)5MfS^+sGS1qq5s;yz z{&Zq5#fiw|Y{Rl|#_IKJ#B|0V=^t);?9{2Xb@)}d!ic!od!93` zgphdbsW0GqUDgYvdU1otF()u1Q;rGN@s5GZ$N)jiOd_MG+xe3;4Iau?dC2%D4r^*X zG@)X>_&BVMhs3PL3UDi2NuDPVy*A4aAo$*=`-Z7qF=QOHqL0s{El*rhgikvv?qMcK zhT)sOVk_Db4VU)D%qiXj4AEXvuBpOBXfUgZI_d$dM=>AB`UK7b`~5g=`)&mOwod@7 zT{-@J1rsPUXkO;fnJ4j)k^I?fCSnjMuDlZE2-PoIGk)IXzl*P5uRp$*EyMGk0%2r_ z;P31ft!odrIET+dl5XZ!l=V=`*TO%Wi35E=WiR!JI{&fJR*y40fdXA09U%IxD(`u> zfDRMpjqb1MzQo{pOCg%s zK_wQ6xUeVpdfrHcQw37y7Mu2DRyI<9Sn8lc(&$G%t~UMNg3i(zr}|ckW|Hn zhHa;!-85WGwxHJ>Gbz+Vq+7U(XM1ciAI(~LVSm87B?-jpL`Gzk>tpe$-vbP4b8p-ddBcffA^>5|}+D(Pd@T8CNWiFwej+ zv}(+klz(lYwgSrFO?Xe_->_+Y5JIx>gwR1*MbzZii=s-1;6*J}EY|I2=fX`(K(SUyr|p%AiexE{30#^0RavPkH-hBX zRWOq@U+8y0TR2tplRM>oJ`$Nqz;Lq?q5bbgZf=8&?Q!e{3t8nXZRQ+V-O^;e+!2ZG z_l{q-y{NM>Cu;C~nE z_r64$B@n(Y@w3ILy9ms?$zhpzi9 zKjuuTgh8ADrbC+$fj;$@goJRrP+}~WXLkOO&?FhiJ8Svtdb!&)?`N)b3eqj5CU*3- zIsjzoz-zXUZYz=$ej;LzfJ$_#{IYo_arn?x$MOmFXm^!`=?a4si=g5K#Q5%pK!UoqP0+amM z+`srUGyA6#{DdmJ#1KWjLX7bqpzlf*8Pml234M-8u^g{`Z!8!uANwN?WRoTv(JC!G z1NSNF*?9Uz$xH{~K=%*)f~Waf+clQEyB-(tgEWZ-ymO)<8)%Ur%bY_+v5j(d3Pvk- z%9d>XW5aD5%*+?ra>D1pRSKkNE2RB=)rwMrdS|=nG{0`_;OcfbRe)ap;fq%ke?8NOqab8Q{XBRn&`Zs zpc?DwpKKBl-`&xBPMrGtvTl-^+#`|>HT9h6NxV`AH3J>@o?`Urn`xlzWMOQgih4gYB-T+QFV{(S z6d2f!Wawk%(J|lir9)Zj3I^(%*oSWnQ|>4K;4J?ZeOXA`rl2Fy_?FsJdG zY0abiczi`pwxi!*N`dN-fJk9urBlI4nSJdDx3v{+>A3X745>gk|CmKy+aO18al`I9C6sdx0U=4^a|%aA8xK zCC|`-%C>iPbG>_Sl%Lf^r-VJaTvv}LKp;7MS?4x|;x>b#IJxaw(PJ4QW>8_MA(Q4A z@Yj35?Sh~L-oiBNG^sSOPk`x_5W4Qnv-8iSR0Oj~#zNner$3h#yo9moW!j`RElLP1+Q^v8!^QO&iBo$6$8V}1$^RQ;c zXcJABs4Zlt-_~UvyM6fa9Jp)AS)BzYF6H3vs4kt}VKvTwSlOq5Blr3* z+$^K}O6v_K-6Cgisu}yK)=6JUlobZ7Zo;c6F(VTq>D7cyhIme3d$q+F9eNu=>GGYkcm$eAUmp;Wl!a%jPjE9d-9yjmiZy zAx+}fojrPGt9?mQM(y@WXVsK18jIAW8g5!?ZEvzM2K#dQf+~sryzKz3xW@fxx67}P8ZF`rEcJ+i_Gkcg2}HRapLRd zve*xGsMQp6>)#^;f?e@hWv(5@Uh#ulYx@c}Aco`!H#n2>P!YH;*c^5%Ke$q~S($Lq z0;Bl%dmlf7>yr?@n=k49U9~|f{H-7|MbGp3{Qrp7Idz5*#K&Ab{~y`{lJcSi!E={S z_x`(k5ygs_KtKi|&VO88A$2g284>G8awh9P+zH4nA|h9q$+%Anw*HeRfP3{v^4*&^oN|R_{5OyBRv=7N(n#5Wjgy!{@;C^NX9cQ z0|I0O`R)GQ>xYD>JBUrvLw}j@Ck+0W?g*^^4*`#%-Cwx+e-8r6@WKYZ9+u+zcYOGO zLk6RDe|Y78VW5#KX;;YJ5yyh zAbLK*ZJ>3;|+f2%g7ocUlBBXgDKem+KCaAQTSjiwd+a9 z`l9YS^!%6{jl-ut{Z8~d!uLZOIL=R_UI1!2$6|Nlz}n}r`~)tKdN(8?BT#`xRkm= zq)cLz0XsNL+4A&69yl!tz&un!pM&{(0QU+?0Ox{q0#st_-}l!K#u~(rN2SZW@&JK_ zmId{nPB!r-!0sE=We+pTyukjv^?FW}jbDEua3i8SJR4P`L<=m-ktgY}r@=A!xmO^N zp(lfE0hR4d07iBirEWbr6~s{lK+8V;94ZAt`xlTgmxoM_3Mn7o$OhJ*M4B+}%Z?B#l8+e@;z^-yi%{m?>N~FQVPI(>3Ql5Xq z=Ì)V5iVM9uqRuk}YshyA!DwpMNbbpoxtTJ=|22xLO|4a@(vZPZO9`-jn_Bg*L z7|T{v4mDJz>T_JH`%stW6O|125&({&Bh9BZ9egZ){NY4-rEP?utcAITUKC~Vfr<9U z8fA~2-XWr1rv<-yOo+8fp(qQiBP%%Lvf0U$b%3C(;ZGMwWsG3&SvtD*eu4d>HxkDwlhe)dFg-l|PLrGI+}`SE%(ShC=Hy_FUcplb;L z8g9ncv?#hfPLHVI*DG{Ld+L9L>+Z;dCG+KS;3ETU1Qo%lBx`d=vPaysvDP;L#$7ui`_#VAVN?uXq@Xi zgwL_H^DhCdb)z{qiW2$tMFdl-a9S|pI57xn5bSCG6}2tWj$j->U}!(yl=F#v_cb_( zYL{n+#S7}eOhK~Ihn*~rL%I?$Fsm&MlV+s4>%IsWyj@FB=!pYge~Tci<>WR46j@0E zz;~w~RY?f zK1jNukL>;8>{zpsB<*4h!jXr~YO+8S;$l_`L(td859^=5Pp%ShR;!O1MpB1qfyWl&9&r`WKvKg4UF6q^ngC(!A|XU}(BGs2u9AQTTu-bOgVvJ`tT5A0aSx1{Twc#GnsAK+;|ix_2V$#YvO}6AMcu zBh#UxFu=P`zoJ`jpZN4Ez;4F<%D;aiLjdn9BS736lNP6>khz72$dBTxKs^ROGm{`?nh@r69Jqpa^2RkZ=a%>r7t%_i*^c6W2 zmphA4wnl=VqaSBV$a%>aF9OhdjtWhE_>DcU0s$y9iv~U&RiQe@S^t(f(~1x{^c8YP z^)-(@eEuOoQ;Pc~sQwl>2$6Ay;Jy6c#2fOXb}K^bEfN>@Kw_~Gf>qJcW1fgh&-v-V zN#ruhq4NHDfXwL44u19OWTEjAvBh6i6CKE7Jf@2OZ86hsU`PAxQ?HTW`ELaQ#}cff z{&^l$3p5?%5IM!Ru0K*vR(}j*ui>Du?F% zoQf3d%7CaBNEDi>%SW@b1w)1e^@p4oR0~-`>_wOsK8;ryIwtBI^*=&x_cz?AfL^BB zeAqtaq7s0eRK`HHtq)@&Jj=+{sx7I`CI5B`bTK<%B{ySVen*B|xnX7OAWZ4mH}c#L z<OEdwNVFrSEiW4h;0v^>LJohb^Zp*~vAX zns;h=duL>N4?XJijYGraj}=Wz0JLG|+dx?%-QeW8iUcy!q+FK+g{k6Azl<|IwIR9* zxwRI2T9iDPC$cfJe}-bndbQ3oppEZQ*+xrUZ~3}(ew#}M73{`p%=ldK?Q6bb9l`C; zAqTGQ9gH8P2GF17axOd(0?7deQ@l!*@#~%aA;a=!#G3)}qkH)i-bW(y&voeo*l3k; zqME$9H-m+x5K}Ak9q_eZH+qLnY2Rq9*;%}Fdw5r`sA2o=I)sl565FROt344o6)|mn zE99<;GUf12aJ4Kh8TDFD92LyjMyoIOY6+AZ)bu%+}@#Hc6--2 zcglkk)&ZHLGh9libl-!^CrN%cS9VV{LtW3TmF`HcmjKuEx7DSXwCFz%SFzyFbGG#9{6- zp#*tGDHWaFcl$H1tm&npd({}%e45o7+x}{Bx#!hIw+f+O|&XpwAhEt`+}?!~H?Zvd+$T@nK?h&h*lo9qJ+jlgl+*+#mCT zm00>_Q?ag9y*IB*@C(IVvB(a%hAlbdQZvk;+3miupTAZ>WN$NGp6|-NsTUQ|7(XOI z-kD6CG5xM)V2=6bNq+Gigq}T|$bo|tz-K{@sOsI~7qyYD_ay6m5A-8h76z^vza|UZ zvxUGueB)tfRljd7_fdaBypA_g!u!dZg3o6)2e6#wWSdmGYyDMNV~T8A3r)ByZcFBa zR-K* zi0TL8{a@Klf{kUjR~<)YVAnz?yw$GuE=qH5cboL5Q%4F6x#;$anHLv56?sbY6)hQ=YlYf;Y7^YkQE=Tf|Ih*NQy-oe zu4>}UFApqPGu@%$(|dV`u)C&&h~Sd-XKWwGk}rkTYP&JiE-hk9ArAJp+nb+U9BKJM zafYZ z>0Yp5@2bmU-VFMO>ifBjCi?@wuGntY)lW|6_4prQp52Xqt{w1{BmA@a? zcH1SW)~!ETZ55Zqe5l8;yVeMsq5NY>QI@oh(@*sFR`%XC!uTLc-abJ(&uKZz`gil! z+NSr4Y;cmzq(?$ThzW!aoY(Gt-N{YcbaB*SveV0&5+QOb{be(^vL2+8oxF|zgAeU@ zGbHy2MB4AR_`|bE)oJ<>BDXA1@NqArOV%Dayh~@8o-=_Jd&sOxYi}6C%S$NFVz-l5 zNw6VMk%5~ub+*%U(V=B9YIl~e^q|wN4?Z=V;XG5-WJtPM<|wjQLa{cv1x?rZa;w0} zU~H0a;q7JnMU%h}st$907HY755tq(|Ed8JN^4gS?_%<>X3#?|u^LVne_KO-)Rh;!J z1LcW!>D7rCy1G&_2A{gb^X#{GFMiU^Ew$^<-4}OXpJ*7`)8b@V3@azbqxyOW$)#rT zpMk7d%kt!*;nm6%EtRVo`+Sukqv#LGklv%vocG8U5z!@sT|=v zzh^Q3)6QY+rJ#CHM*i1$@n##(u3;v9HT9*o-%ir&{%1%A?P$L+L-a1(_CktC#@B#j z7VlkoI+jznV$du~ktO z{jW@JmNaz6pJ(kyUn(?zqjw|EN2)lfY%hRehapWXC-Vz5Bi=7#khg#B1=)~jHp+hD zIkWT@j=kA$IM)-ImnsX@vx2P?Yn1P)2Q7J?nac<_F6ZI0_(3DhyEDX2tXT@Qm*JcZ zf;c}Dd)L9~g|lU;j1BcB`s?4eyE1L_HO9&n97=^Sd*4M`Qh$0;rnyn(IJmfVIgIyS z5PU@)&L0*JZ}Uq@OO3RxOQvj5>akxawg289uRP4X?}y2%+(WtdxJB8jRo!28)NzlIU{ zuYqe#P@Wyin@|??F?QaMK?haJMAvcx9a1&%(${~q2MBJMj(x@^u9$cs|KrZWo<&7> z|IoAT$RC(Bq&)-d_iv@3d+_|JwFQ*Lww!7R)`O;Pl)&%a*u5z4K*>AloVx+FF>W8P z;7aLiGt3|i2*p$4l6Se-bvq1iqp3wA~ulfeH3dOr}aEy5yt!-%3`i^4E!Um;BCyx42oZ^2R7% zYcRsYv1B}ruAsMkppe1FpjWnn2^lY5x>M8{({De7lR7iJG7JwMZ@gER?Y&Y1DW33e zZ%<~4`{4Gjc~Af->i+F+0ccD7H|_aXU0?k83V$3DzsQ!8`=K0^3Gff$c-d!a?~cF6 zv--FL6#YhvNZ%jO)6$X+C=CS!QvN%A62K zG0he)-vR#|;u#-kzeM^)Ps>VrAu{1kObfxYSPn7E(Z*Op`h%I+<(ani?gwx#+kGvy z?gYv1dH6)_2h+KkmIJ6-T1rXv!T#8C9+7|!V6EEs5&E`gN+So-G1rQ4+?ivs&vrL) z4!-i@TVs8)OQAM+=_kzAEu-;=5@w;JkX5g0-D{ikF$TJ+ig`@4<<}~Qf!q!_3@ zTmO(Fl`FO#>h^B2`pFxg-Q;4L_5**@;jV$*Cc3)jH*rL)AlA2YB(&SSKxzLb2}Ezz zKl67PWyVIdHPPyyd3>b(KgaKqlJ4T_kltOP5rT{-iR3HmFv39J-|sJ&*L6}&zPw|1 zBMMfUar^cZGQL7^^%B+RRmIu@y{99!Q;{DcM%>6{F7IP>RAepXFZ8Rs*^@_H`qE!)`JGMkz_9Zh=Hsk)UG z{>E9I6%8vu0A2NWSrr?^Z53BKs=(^kU%;W3`m8ik&Clq5RCJA|^^QWyY1z5b8aXSL z*dR!(c`eYS+KNZNW}H(?w5V$N^eI&>NaH?vZ_2$~{f*B{aVhK-c!qbe2O_2~tIM(_ zF9aO;Ev9@?8BF`w$iqo8p*(EXm9gu&IUGtvy}yc`mZ4+eP>P)qA;G~x$2_Xs8w@I< z4aXF327L=H>L8F)77=VSxLL1G7r?U?)~n16qmZ5&R(!@;g7jAK3y7Y?JLVd%V_A@llolvcxB}|YE!y{3wKv|Qfi-KXkrp_6V#-a1?5qd*%0?j z&ZVN~M}Ohb&=~b z3>n{&_36FE>D)tbp`&99J3r{s@1jDCxIkcf{#2a!gfxSuEdX|V{bZN47*cN~O$+f9 z#$OME%TPH~>c5$!%+Vh!dPss{qu#<0q>4Sn>11YWeTpTp`=hRT z-EfS&io+;r*m|VZL$#HjNLKbDisoVpm%9Cg43*+wmH{Poarn*fvK{lO#MoS;wvtGt z)aom$9-hVe!mBE`-#oa*_M6x`XrlI^cKuLMs9vF~@oMk~SLW*C+N6kelYsZoNA|Mp zU!yaag?y*WSr$I!Bx_?Pp|!fI=I2JQF_%%rhuG>w(}TsheQWEvf+;4&&EovxXD4@H zy6BE;FwyYJ@<6~cKC70G+qY%Ivc(o$xwN8Xul;9@rDd$|#qMvA1MNA9A$DK1uUw{- zj`gVrz6(}*n%e_2Jo;5h@!hAfUsO4vYeHfsH>Vqo$nH`QHvy_tOY} z`dlb~)nOZer9i*9AsRG2+_HJnPbJI6nyYR5`A>surJ;u8SB;kxA|YY#jYww)^K z1i$?cJZjkVvm_jkYx3)1A*3O83p73%mUEvugjJD;n}HRl$JsL6>t$cn;@ZTp5A3oh;R)wyRIMmMjJ<%ciHbT6oqVmtsu%C5o>#9t z9~z`LlgVZ7XQU-MH4HvnyikIrjN6y)TKKEe5o$RCDx0v@dulc@b({R0&MA3C+Ez2) zSjoGe)Z8e(nB>EjYH@uMtAO`SI*-}?V9Ej zDU~TC_Y(Eh4syzjOfyLH4txxe6%?9D_r`?Ji`T_2`xL)kByLu+nj?N=&Rb9ue<_bc zv+dF?pO*N#&o&rxvN6=M!ZQp+IbG%&MpcWN1x)XqY(h(_AKxDww+W;Q*4Q+TKCQW~ z5TLx{y{P1}3jOi8A$ThzM+?Yl`BR}S{H#encWg2Fcn7%U-@j^@?_p=U5=fu^6mKvs z)I~H~!u6$!3;9526g?g-(zflOwCvdYG0d<}pzYhkHJBJ-Lj=)WW$L%MC#h*TN7PDo z-#drRc%O?V{(V0e^cEzR&pDAKXIzXC)}g&lX^)NHPPLY2^sGlaq_!G9g~mtc1%e?F z=IOYW6!|)#`a*|iX#n<|pmQ-|Zfcx>-D!VjgRZ&!vZx%J$zXe6ICQ1&HpaEWOA5 zVcg1?DYo{>|AiWzu1LIeFm0aPn=Vbl!H357jQOBWR+PNF#H{+XG~5D6dYT*)g$A*4 z^EcJT*@9?GU#BNq%8C>8ZFVW$-Nc(e9}96bY%L^00MwzRdnv zouMxNHHZ1MwC(N`i#81L(ra)}laFQiM#Kx*L9ftrebm~NG5gi}c5u@|ZO^+<(H&$m zjLa0Dp4gPDYa(ul@H{xjvB^;~AE8Na2#LM3+<|tDgx8c;wUqGs``m_L-`Okt+Kh;3 zj_MTQFqH9%D1^qD@p_OA5+I;v7hmqcY#(4OO9DDU;dw#Ld_p3Z6BzJ~fx%7Bd{+ch$xYOPt?Fm>`nUGZlw=oaaq=_=+cZCc zF4r%Kpc*TBIk+YmDS!0|Rh<1S18^iwu^Q39oTn`STiH32=cX=wDT~Y1b{*DuU^c`D z(Yvx(c$bBfH?gK7yc-WsG@t8-x3qX9b-L{o)5>rdS07L1$U?C}VQXldAK|TFf#kVF zvu-_ahVtvK1)nQ5L~WnYQpYctM|H~x`l}MKz01{?&wD+Lr#~Q_ol!Iu>-YcB_0>^P zcHP?s76=%Gf`B94N=hT$F?5SGNC*raq7Ert(lvCqw17x=w{&;sd}oa3dDnWr-#;v8 z33K1)oL$%6`|L{>zwDRmQ^b|Ce}2Z3p%!z|w1u%!W)^lGW?dp&m{LomUln*RkYbsJ@m8$pUwDaR;Rj>QCmhUS zA5iNpm@nTnDAGAw+D}|{iqMI7W^z>`x=!l1=sC0>cPXehQ(25xR5$6`FHZj=v`fWQ z3c|XRQA&CH!7JmFz94bP#eL5~pCZEc4)IQmB-&)GuA1V1ng*WV1u(>`UU(7bk8oCk zv{T%7^VA@==4ouc7;`}{5W$&K3n@?-d}JNHc#>;8@(i|9EPV9BmXGFGj=U_z0_{e- zu^yRPamq1mv<1|@PqTA1>9G!r&^QT}#JOVE510PxP>bH5mU(3-Hgn;uI!dJkBvQJ| zZV9IGDcM5DZ6cE!^%d-hd_#*GHrlbmGDD@}2jn4x5wu+v$|2&Zuv1drgxajA7m++9 zyw|ApN8q`?%y2jw3%ZBj;KWvQM}C6~dK90SVXfF&%>vB;RU!eT_y;H;3z>XnOijFSgaergLtnxaJVmM*2(#H-nv8kWG0c?KujrQ;C z+Kv{B$DQdB0;cPJjkSXgs`h*bVlWL&s+?}E%}m}@E)RwSxMKR)@d}y_k^5@}@st3x z_L7OO#%>A}#fM|7i$*Z}Q!r8mst04)+)i5S8v>Ew2_@_0-IeW~{RRpOSIBtfJsgoM z9y6#0D(*aHqu<;V{PrL&T-yh3Vt^`hbL(_^Gr?RC%<%Y_G-;<~-2iEwBC)(=-M~sh z(gbs#mHSj)*Tsdk{v6cecW8~pin=8xtV2c zSgT0G@Ds_<+Nm5tI-#+ z?7xb4WFgP^y1Zj#l)ONadj6`;>NDY3cBWZhOy)PNIH^I0nNt$VswU*t91y#+hkZim2yPWj=3d+?cN6JD5@wdiFB$11q?nYJM($}VQaNTKm zKJJ0)x+!{QK9$+>Ku4pDOCYv^&aNbl0tUlsbP^G)NMg+ve{Gj9l`@Y3cSrar@lJE584(zw$=&z4OdCBd$jy?s^i{EvNW_-gGafTcPdT^3ExLw(FzSU0@+;v zh7VCJ|mGqi0?OtlOa-`EV?~xKHY%Xrr@3iGqui200WNfL^QzNu) z`(KGrJD(YA>nWuZM31_^`0IA-d&b0RvojZ_3;#?vm(5`bHu$+rCC_xI#X_I{;{2j& zvTOJTUX9Q`&@b7Aq@IWR>W#CQE5D>w2r#N8utuR=2Zx zzV%OIu1L^pg9J8%>As^ET^Pb46{-v}g%16!k_7@m!PI!=tvdJ80?S zBV-|4HK_$z>OW$;kKhxR%E{eqQC_>s?fTdkhKGYvGcElm_PFsfnK;(XIae=+@S(aF z&ReAyJq5&@T96F%sw%AyQkAn2O=p%}V7ftQyl9?qYa~m$sRjnH0JF)@QDJu!Fx5tG z>wozUF;QW~^es?41BIe3;$^>`Rs1M55PjB`pSQSryy~!hsF$Ec2wSaNRZ2IpA2^Vm zZnbB>_l$a8Lxb>o)J>90bC$_h=n=glNR5lAhlO8z>H_7hc|rtukMy0i?Gl9gpEoU$ zS!3D|QHr`jDwYTITW5ci9rIMW+RI`i2xEd%Bb;s7n1d_*^&?awDCgY9PIXzuNVTc;NJ zZkDw2Dl?gxWlY~Z-?R+7@$}f(p&UdvJ%Qg8fgXvkt@qE}Z5|WdRF6}Ke(T-|$3x6# zLrrhy~&S>IX3hF`6-1ueIqe==lO3V z@sTq5)m*N4FxiW*maYCLIDP))$=EVwUn|-{n3$Wb<(4|%`b!JYG}nYdm$@jTPIzjN>6bjM^uRcs5;^nFE$Wvva-fLs>5pnUsThd; z!%nXST7z;PmV>+CJI(v{dCtq1s%}1rpv|ec+K~igL@k(1=I) zbJNVI^o){#M`=2^%&Yd?64x}IbbK>EV_28Q=54KTC;g@LgK$O;_Jy{{2M62-MsA-c zQWV_L%r-zB539yWSWh|UyfMK_qR_hbeHrKpcY9z6 zM+vbe&Wpff7j-%$4+~5|i5t--F{kkHhG9 z&olo8%Bn|$Y9V``N5_v`a%mTz{Q%F+{oP!452xqo z!#evUt`WPhszanSq~^XFE~Mh>Wrd0v#RE>Y_q;8Rl3RXs z3$63l<3Wn&lOQ{jET^tPN2)@N?!#nm5;b)JRLGFFo z2~ILq68`3HrZL?Y>`@ULx9rng&-!4un z5^hb3DhW1-FA(; zMK*nZa5n1p0i|&}4)nSWQQ=-*+~vSvPi8$Tuw!AUM9x|8F7xWPr=~;22m>YAs|F}v ze2ukLb+T*ze7kmPBEy-KZ|o$_aq?}LR4I7~m>McQ_UM2c=9hq5=8QoVFRHY|ztVFRPK_AGhrY#^F+P8$ka*Zk8d||lW@&Lp z!IbP%W8I|GtR}?Z13{PVdBzX+V60K*>Xfq6DDR8DyLn#$jCGvMX3MF>#7Q8TRGcF_ z-!3+BB=j#Xpcc0VsJh|wQxLX2FwaUEXhXcc(mW}Jj_WP^%NE}qRG@2{IAK>rb~Uqy zbXtK9^km*QgS~6Y5H^tv%G(vW0T*tcHe?S>b?n&lyY`0gx{k-|OF^w8XbXtE#?pRv zM|j9W+v&j~%vU%mgxV&twee zwN$c_iQo}+J!q70x-AjVnDvMabsUROo8;feVTk zbAr@>PtjONh3JyX8`ozn&B+?)IJ=|5KJjHP%WgSbnGWcVd;5zOW+R2Bl3GX0SwwNa zJKjA#-}ik>aOZH;vvFdBQ_puzpdB#DA0A{mQ5F#G{qVDdc{V%th1w z^m#IxvQMD%{B>3p?LFekz+1kS6V$)Z_AOcz+HN69Z^AqKl+q%yhUZj?(^pGqNK(V8 zxf2s-GAcdg6`MFRKNpkm;X`BN(7_$bX?$pAxr|HSv0+|~&emZc z}}ln&IOe;8yNQOz%1W+dVeSXC=Yj#SN{@9FidR{)XFkh@pxPV6&Y zB)GOtYkmcj9&pDyV+UrFzV>WV#=5@sYsivjf8>W(<&_yjSg+f&S^Gl1WECr2uT=A` zl}dSgt5^91sphxM-jliT%#6*HvEkhf8md#4ppc4mPhaaQP?M;7kzExWtnqcmW~8Dt z&)Jr1ZSFT2S)=^VJPscwUX68j2Yn%B+9#Uwl(nE)WsiFKs7{Q{@@|B``95>->o4Kc zYLG-e@>I9RfOCOU=h4pNw^busaukqWTI~Dpl4&gYfT-;o`isrm!3YY z)_3!IqwHl$MM_$WEn#?nY1IfQ?=8j;Px!$R=;j9zow7zQla!il{?qX)f$+1FI9DVD2lagSAWAnXGzNt@o5c#xM z=a^)U8hTU$S;_Y)SKkar3LW66z3Y8Ho9`f-f&~aLXu*Medk%7NPYm&6`~P%&(V~1v z(M<)h3t{A$4vXcrWs7g#-ISMB8Yg%oQXlW`?*&Ut zNZJK5fH4HjX&$PblVAoxdoPg;5$%Na0I$!3jxyrS@w2@6U9K#4v_F#^gs;URf>v=I z+B@d)bxbZL1|x&>c&6q;QFUjfh$VPXz3va^?EtX95cc!R&-AQ@wWy@fX z?B($8N3SEx;gRX4OW!mln&N#i^vswbmAJ@tWP_eOjgz9LJExEa`#IRjjI$jJ7_2i$9Nwqq(cMH*^^U+4K zxz~&xS)Jz-uTX_Yj^|0x4!7JCX4Vs-Dd+u&g6{@9;dzInd{GCin`ANPsa8bB}t^^TpwihfDuQDSqWED%hf@sP>NXElSPDUxbC$&hyRM_^TXT zxVuJH8Hn<+iWz`Nr%WJQhv?45nON{cZ$wab1nVNRFW))alQI=xv1phgLkTk37{Bsy z%mfKd!^&-TRBVfd^*%J=x3biwTn7K>lp-1cEs#^h`A^SG<%4wqeQop@5G#%DC1Lc~mvTnwx}clb$Rd zME^|Q5pGP6&-XAhR$!Itp<6iHmiHEdB!lIBmRZ5+Lio9Ns{9&jh5q8sMqui}GO5lY zIWm)}^k;G)%a}a9yf<}FJ2Gr2t?S*6Fp1mSqV>C~T@2&2)rv>Dnmz#^;J6TT_vX6~mG?`P4ZZ)1F z^3VHF*||V0@!Ya5?ZvX(35b{?GQV#z7C7ie@koUCO0!89rF8Tnvvt;1=^VG{3#;6j zdLz%x^UW9~u#MQo4no#=<+6yl&~E@ZeUjmJYm=X;JXMk!u~={046T!#Y=HTF8x&2+ z^_o;3{8GMPKcP7LBmYjpFT??^BCSWhHC&c~<=fO%(`gUnuYmN~IqQZt5f_u=d~HtD z&2J_59}MU~N9CeY^KhUKZN~AFS-23q8zTkV_?LxJf&>PfKHW@lTjzr6rK{P-AHSRO z`$V+oA8nB4%UhWrb$r>&qoE~1{+!1imMdyEt<5~;UZ1+_k|#5ZLVLSU919*2lbT+Jv+pU39A^8I1Qw zHJ#N9b>*4K_hi>3AWFq)p*D4KtP!=IU#!+vJsyz^)?4;{?_JS8tWP|b02%&5-CVxR zYyC_LMwJ1fq;)}>wx*mX4%-g{;u=5|XCuKbz#Jn7fU?Vdzm>v)`sc;VXIlSY{>xSB zDAIk$)g!oOUD=Cjk=|zJaq}w*EyX|7X=(B@+6!vvtAYi1FXi-{;OfNYt}4aPZdLtC zq$z^CD>=zOcz=hSPkT_|OulGLH*rPz$%Uh9jQaG3eS1GfOyj4`lbIjCO;rhwDE*Jb_{fWbT(aCW&?wiwqiRsxcG=@J}ST)qy{J|NuItUC)k?X1gI zgVr6f+8XKqJS9*N^S@;#o^gSv?55%-ubr76HQICAA|pCIwKz0 zgIB3z&|%6ad~ij{w^&DF**9V6neC>L`GnL6e^WEj%<3Zjck|B_UBtJfF7`kO-84D+ zs|fen3w3)N?Y9gAeh{$hl`}tJo!W1G2@;%>WtA8&QeL!$BAdV7BvG)1wSJ%XY$v!s ze!ZL$GdT(DO zu>G8l%@r$w#R7e_H#|~F70f)veahhSlw;}Zcc&Zb*#YMkr!d!2={?2OwAe=*O1Z}K z43pJ;k%N;NYMbORjUM*BJ-^zm3y>eOS22wer`fAoR!(=Va+{Ft8zz5i-jl27zM~*U zt6+@+iY`*hAMbIhtyYQ-|Bse{B6^gZZ)H*Jy(Ckc6;OW-{1EHK(ShD~B(^Cr%A}P2lBgv{sun z2)-vQBW{#|;HCj1>f*tH>lMtDEU)PpsB0eUt6aSSmLrYn+X`nI2lPVzaOgfJdL7x? z{}pErs=-oPFC4RxQx3w3#FGxKvO)S)wqHZ)I#VCtJ-T#Yn%(oExX$@gva@W3pPBMT z^su!8sPJs!(Oi^3n$Gu>jDEbPNv{fvPG~t5UhPV^2s_cuyXmU_(D&(N1)Y16?2u#6 z@Ix5pHvVK-&$5QsM!0b_%~jK%OlINM(DID;;~69S8ylBX{#d!nwUgmLPiSOdq{EqP zotOPH&|#iRteE=WlT!r66Q9F7Ix!w|n_a9bx`duY^dfdT9fL|Ed7qE;Ts<9;+xM#l zbQYv}8gv=c4RLox6D>6=WN$<(a_-ePO3852(f3>L!(5T8q6xAXf#*lx9vvtfxd5-V z`Y^?88#Ju1Wn4TAD3zWrvT5#fKftX5Ffzf44qieLt^?e*wu^c)gx3~*XMa?m4o){+ z#zbBfCosj7vP0BfM$AmfK22zFDv)K3sl{}1tiO4XHt4V_qr=>F=C>GK-aT^4rgxt0 zb2^%AhH%<93ad5v-Rk!f6a}0f<@~u4$}fy+&aql8H+J96=F-r2-0sTL4?d|4V&v_A z8&WDwOJF1A_y(TZh{LpJ;8rSRe?5RJv8hGNG`+SvhxbPGiM-#BvQiuQ%B(L;wHbPF zSmnSB9%BdIQS*o5=+fkO9VKt4MkaXp6hhnsWu_yCI+W>2?7W!22&|fovr!`mAJC6a z*u@XS0`ubZpuZ^ZUwdQVqy^m|E#eH2CVT*M>1T8$^zXmu8oY-_aJxY#jCYw!@sN;X zFH5H=7wo;_glkHjy`5HY&@{V(ynJbsYkg~Jow4C6==cOf&fPvUo)*j)6_GC`bqzlj zhnR;scNQ+=As+@%91R>TtMjg3mjVu>RFu?Vq-l`<>tIAB@625gM0ZS1og2tZLtDV* zo>==gqS!HeikQoG=oJPEQK4%JKalOIpqZdNJ5=#VL<0nrxL*wn_iB##_`2N6drW(L zot}(WDU$E?_n*PlQ1|>jK`MOydDYL@Kli}3P)?-FX*9~eP8EhB$!@C7K)@&#DyHq~ zmXe=Pi=AKVAnKCdYa$?QQ?iXgGL*On=FMxp@)Ppg?9J@|WYM~sY(|!W6F`+Y(L;mP&j)Q9*5GQ<`C3V39?y z7fBU5%(dYitL&4wp&S!(*zzy^xJd?Qx!NftXIKbgET2pF!>R9Ud9&617lN^fEv?Dy zv%GLkuDFSuTGy@A@$YF>D>B|hw+L_P(VD8)qunin+;0C=|0W3MTZgy&n;>$u+q&PF z_Q>=<87B2c>q$SQ^Z0Zxy6eK#sVO`8(CRSiki$8_DBsdGOQFiMJbp}6N9p2FV{j(5 zAH7<`IZ41Xoo`L%C$kV z_h2Z3TcauJ!@c+5vNS(@J%g|0{#uSRCrwgS6f7r1xp7^z7B<$!;(k5(d<+)kkCk7N zaEK+@=hxY9+0Ffvy3VeS7Ww_LeQB9>>UENI&GDYCd2*A%`zA)|tt%|0JrEw&?$iwQqY;K`B7{X6RAW`o1KkQxzKZaM?hx52Z4{Wk3ZFanEv| zKdmG@jM=bgekwDFvuiwSFK&tPI6K&tQvLK!jk2H7z}20YPZi)DBf(0HPoq>5 z9}-)hHwTxyhKv(R?DfU_=227iD1N}vjH6jro1b5Zw8Q=C{>ImGZiJaIgd#7OM+s%FK~K$8xT<(uR)kK}s(3Q=GldO5x>dQW z+1gc5E;~J4xmco^w?`^?lQEue(^E$Oq|qO;`J&{zc5upxQ(^0!l+^PvYnX`Zq*riR zF_a~m;9;3!GdWp4lTypThN=Nn0T+tABX-P&6@t&6%hQW>HoNvzY?@2B60t=?I=$ga zLZKmb7f6T)1c)uir#P8V$nVY4qI{1&3Pa%B{%7k<1)fU6UFIVFb-1~Ai_L8&`a~_w z#Nx$h-bZZ0DS7ShAxfT~JjbhOAgxFhkhtn%=QP)lA2*;TW!%%1!LB(W#$za>K5j3| z5v9Ew%RwnYY+WdF?M-$FxCH0ARQiMOB0rQ%#K{D&hxm-slL;9@z7%iQaQ8*9KDBI( zTv%AXvZP7`BZ34BN!f~{%NGT{tBd!Ja!7=hk_u$c)moP6f^MJP-TK;xT6|%KBlzd| zWGucipt|(ZN)XBMU?rVaup>>yndq4;RrYLT3+){0NVAiJpL_|+_;&)_R*=t@=;{m^ zeo2ZkiyXue9i=!Zxmk|ALZiQQm-;Hu=ih=g5Xv8cJ}P3ifKQFjr!%j@U+ZNSfk(fx z^36oPN^>=~9+ae0nT#VFjgM(xbTF4iRNK~DKTEHbSsV0WZ_pjqP`zXr`)kp5CFP$Z^H!I1 z>Z%HoYM>gYCo3nEZzHnRcRo+mO?B-8-nugLyA;xfH8$=aC-$Qf&MK#JyGhy~j1LE|EGOUDEUu~lkLxAd+F zyFli+oB66VY`?1ogu3sz$3B;`2uVQ1F(s%!4MuI7sRXb&X+)C6f21e%0Y97$mp%NK zeL_i(2m1YGA$V9H`&zu`;Y(O<{k6V_D)biq^`EO|1r!H#E_CON_$+eN!&C;w4E*c3 z4s0A3M5D6Y+SKpBEwUz#N*vz?ueAjKXtyCiuGPhMWDAsdAcO=>ycrTzTl;w9-K&%6 z?1(rfzF5__ex@OV@{}m7eL*@g)yJve7aEgwelMi7KB)-W6*)h}mlYDZ)Ja08&X+Pb z8Q&Lm$q`ngnK$c>Wf{kY)0ti@^~1E_``Qe|R=Ci>yrqzwSQ~Mj+WDM^KFG5=>HCQa zdzFm(h=5dzuE2>MMoQ_8QuGz6F^u_)pwkBRs8bqp105AeDXz`x5hrf2kW#tAxP<8N z`YqY1!Q$o{K(}6hy^|<_WsmH4`fZ>2?QbUv*m;qr{;}3zvrg|D3&O*uldZK*%L}ih z8@N-`?P8hDoo@UHt^DSGGA)Xw8+IHH+ed^W83{$ucsaBxClcR>y;8XnYG^Du*h30k zA^+ieT&3R-{OFE1hK{B2Tz}nV2W<~+-(_B2*7AguxfD&zMcy@c~P{P+xpCE-VY!2)5A=dWwI;^&w^PjDwe8F*+%Ce%^ybTppynO6_T%kXt&%vCQWzA^}y~R~2o=g+TxpB6dIR7Gx zhrT$EevL03CLtJ1N2on^csS87Vi%3OM0;r@$R+O7+9$pNz2+D8HcQL2F?)>$f#4i= zcR%hd$g*pgghE_JmAw?Wf6^exvI^$VBc5-agn8;5#)VEQ7Q}8+g#L)M!Tf-pY3b`Y zxyr4;liDh2(GvH(SB3gOy*xRIC*z<3=C@@Mf{Pk~>%;f-_kU%w`v z_!XVHked3Im^~rVo_?z?S^|10eFJnb#9dnQ8hsWLbTr1RMlAC@U$Gf&c0ICDYpfB; zZG}$!h-0=sH@GBEQf>&9aT_RQdWo~;jI4k}F(JpQ^B_C}+5J&Yy{X0cq35{8vvI+n zLM$eYY{oj}hwi&J@3mE|7gBe+mo#M^rTxRBGW*45uW zcqqa~XiQJb1haUA1W=}$=&EAN7>pY8PdvDfGUgd zJQ&Qn4WBX<(rdx^eK0||H&yu zWLl4z?Pr=`Zn%#`N{Vtu-`z>EWG!JdAdGxc-^Gz<(X#aJsO#%{5|<016HWLgO0NlD_fX(98o()_o(l}nYO=eZlm1D!x`A2)GRMZbBqi1jcz z_&{x(FvBp5f|6MHQ%mU$^z33T7_otu3Zr7n2MQjS_B&Zg=$X}Head4cPC6e9R|CTq zFuERb#3_s)8frONyPem#UB1@i`e{F+de6>_}uyvh-2H`vC zca_74*R*{$8_7G#KDvQ@{d9VzgOBLl;|Rkxl1_>CJapM&s8l<-Wsxd^Io0Ps4KQ#mXF6Yqt=Y(5b!q~rkl-v_lZ*!(*7No zP;b~koXoZ}ADRbs{x$`5$(Z9y$Aw4ehS-t*H z@Ff&ECOgFOMV^g_-DJha?qHxkvYVS=!E7m|WZ)t8%*uxT5%&WTN*$(MQk~%s!{_+2 z`uF>EtCZ;pMxBS;pd_%k;+4S8X41xrAqJS2QDmduQslsS4G9afl@GJnhM5W#Tk+R` zEgj0{Gr|00xcGqKQmAkTbi={Q&}|ftY=Pt6;sBHLeQFgZjS=%r|fW+MBS zXm|1}Oe)m8--nsq8I>tD`PqQMs9Edcn3Q~rE|{!kuZxSZ)Z;X-bp#9>Kbf@Y;xPKw z(fs7sK2nJ!+T@qNIS4}kC&J@TL<3NQoIefy_xbW~09Tb}_img9K}`*YubXqY1zQK( zY%BuePg!%x@@MjpDq-jXAS#sN%6FPB#W^0#cN-LzY@?RwYEUy5g43mAWZd}o>rrS0 z*y^4KKrbV#f*<|)9}h+F)J#=QMF{!VmgZcvbTP%2flb2g7C`JP6VJ_|6i&$JcFuD& zvG@ITj z%G4g*^~FWq)-ZYHT`;**2bm~E|JtY@6l+~H?)=dhpFvv~A~PdMoZ|aGp8OwsdnyJ3 zq`>V=g5R$sM~GUn*5k||>oEywJP`)SC^C!*)nvL5!a>No{_9^0M?A6z;jixJq4Z=Uq<;O(Hb>c&l zYk%Z=3nkZ8iC~DiF!-`{U)sFetdY!JF9pXGOzGo9lLhR$3OXKW18&jn(IDqJFA_J^ ziT5l0>0YBKCa_t@vo8<+cMJc8w@-VBfjk|44gRyRD;_~$p<<1V=Gj3sy6qA4Ns5SV zLd)tIa?0ytiBPZENx%vo!r0BVRSoz|k${XK;pMy8jwogr#cQUT!(7eNQCJtpl7rSm z3aMi{r_8boDZ`RSj>#*x1&E|G_VG2XJY-5xEG_wfbV6Jm?008ds)O1x%%70G3%W|x z2sC8_u;4*I`&Mo@VB}XUMyCmro0;GW6n#R`*aD(cXjNYAsj04FzisrAdHG}n{DsN znw+m~*M+aA4DwaU`Dhc9JRO0l+c9ogxA8dTeJtm~EIx${X11_7moSR8UN_3rSzoX| za8<1iszuPGY3B(ufG>QGQvWcvtY&MSy1&*h!_}!*bZu7{GbSjU?z1Hbq8_Lt@|6s~ z+{I;D$7%R&1pl#7AaMKuULNDav&X+x_R2L5)M93AGX%0aP%c?fgNzW4%X7M%GBcT` zYr>z0SDGZyww7yxEB^t}uahAl;?;X#FUf1vqaBP)kWRJ>D0eEMVQp8A2$G*MkjVno!B(4PTQv+X;S ztARfSRlPNCLZV1Sl}XQOzcrOxBNz%z@l$SdTQX8n4ZvzSqjQ>{2BufTA7%8o=St=l z0A~cbn&{Hq&Cz1f8@$`!uFDg|RW04r07;>Nv?9P8)BnPyNN385ecHtaN-tq?ISN5z zre66;-DI=S7H%5J;E0%V-l_?H!72XSG?Mpn+l%k6IN$~BgJa-l4G%EH(LHXpdJ|rX zYg%L#7_6YHDoHis7>#wEvzIAHKsSo!w(qc8foc2a$IJqJuOwsl+wLm06!7!t$?~2c zoNR{kv^bp=$-V4x-9T_veMccps_+){mYHX(t!D~fMy<5PWkAl^;f7zltm{DNAqWkO z_f$vsdIYikiFr74ni?~Lrbz!D_5W;Z6Y%WX(20`@akD#TFqPX{K+vY zWaL!Uss5Z7%8$Hdv|^c!WE_SY@U?7%@xvFaGd5Uw{IBh{fUQ?8qVi!)>LEQ#VacRH z>^De0>DN!`o282A=3w))$H4>c&v5J1fVyGdh*@Eq3M<$ur zmoFsx_w~G01dRIkb18x*o;cb{uqf&XEr?x&?LgS!2T1-ebV_hV9ns}5VRd~~#ns>d z;FwH!0oir|1IAKzQ1EyTR%zHg5pMEg`evpd`eTpT|E@amHt-?O!o;rsQ3rlr@L7>J z4*WS7!lsiY6CcRXhT|%lamJO0EVM1bThbY+$ttcUbw}^h1(x#9vY{PbE#j^qR$zeb2GZ)2G>=5I;>SD=3hASM1BvyI$CyVME&y za4tm%u+2D}C0TXz%h5T)FG7a%YYxW>w9r!o(u$rv_>Z;($zH)_O2D@NlkvPqB>+B$ zLziguGcLrzEiU(3C9)*4eIMt=C?hClL5rMzSb_?Z!I!7)_|R*iYfLA}OMw4qojO%T zWDx!)G$hrj1`dW-9BDC1Up1Fsv#+`VvA!6ubFWjmf0K+(LR;~T)&qux+9mr1Zvujq z@hPVd&!1A(&bs0oDylkfc;ad~E`-Z}z~amDPcqMJ?pu)=rgA~*2H%$Q#}Ef{|1T?L4l zkN2a$Cgtxs-t#MfC{+4>@ZY6+q@i5SeOR7=r;J^Q00>Hh)q|fc>>UHPM#+~|o0V%K zfy6}URJva^Xb>`^p(M%sRUo#LC(uN|ocvE{^k3aK?~G>EPQ1p^r|Nvqc2gJYg?Q>% zp|W*BM@tAsaWqME+{3A2BU$Bgin&H>eGR5plrMt0v;4*V!qVtXC$o0unw=xW;1iXw z>}iRcpcdmBP8e0XVPzcNb+NyXo+#jHbzAW7+x{bc=%|F}ZD0MrQa6tR7V!QG^vC8R z(Gw!j%5gv$Y&)c81ost}`6`GD{Sa3+@j0h0lR*bg2N<_)AI$=?oM&Tkf7`y&q5fHYe@rHy|Wg<_v=$ic&!l`}pHT}#%y)>wrXVNKS| z;#jgG)B>FGSU@!3650)uhZvUGJvl&Ai%}9GRCIDW5e6P;r&p8&MVd!^eM9XO@IRhR zy5K?&(yaZS!R{gOGx(QVhJ!m|G1XZtP7~n(!j!jkvfB_RoC4qF6?(VTZi4*5Mnm-KwUw2|IN@;xNNHWR%CKT^@ZBY!n5-ad zp8cTlDBK{!`5QDp{Ws7*0YoQhN#K4i0e`4|0HBs zIxwA0&_HyC)okdK(x*ENfU|j98ArR^LQ7RKt7m8cp9yN^a=kl)!8OcCGb4db36h$) z@Kq`68z5|iAEJo0v`ti=3R%PhthT=!dUX=h-x26Ny+2zz=A57wqnh^{fobu$xu|Fg zN+wP!1tl}DL2TaRr<8uvPMZ}=0t&cO6*lS9N>BS-mbw#2ru-YAYh)k1Zd9gcL9xQn z!Nfd9FFP6Fbxb9XcK}Spcu4yh{qZj3zeh-gO2|B=3i(^1`lU#?GTAf;mu0fIaGH2= z$kBw2tfCtgZtLKS$~BVIJD#F=StV9y(z1A2LH3id$Y$rKoR3}@5e^!YNcw!%_P(+C zbuK_`T0Vjf0u52tuPeKIaINavs}4bDi^>%WM!v`10n&GID{Z=YhTG+{%ioTMbXdG- zk7PUwL$3xQf*Q5W?1CTxUUC6(jLRLRRFmQfF_r;~a(VjTWSYy`ows4o8VdZrp1UxM zFNTSVVii4}GWARv8ta}zPO7&GJpGHF(;)W##&!P#lBiG(0WWExf3Z`LFgVa~DTF{< z%C_7rr`6B*io`2wV}s1Ln>Iv378YLEBmr%ak|~sOKMR|Q{T?i*U>!)}WlX*dr52=L ziM8nxx7d1|X*q_@TCtKrRseD+d?jrSwH{f6F^Z6IsR!E7QQ$qz$XZMv_V*lHyt*x{ z`k0i6O$k@4!rDMzma1ELEV=D|Nsl@Pqk0Kd-__8P(29^3v-$B%4VN0^W}zeV{mJWN zrm%^x@8bKeS<^|w%#>WdK$|GAy~8>Pe4Hf8K)ShSzyk^~_)=@2TZL1LygU;m>6@&w zU(;--@^=-2-*t}ouU|-f3PSo1iDEbZ=rcbSD#U*oj2@L`F)u2C9m1zqwB1bN&fP`@ zk##eFkc1FJ!4GPAM<+*RUcfG@8wHs9s-{cdPux}dz|cQ8l&?cV9#nhs^CNaGNc4=k zDpC@P2xxyfUd_%QDh$b!P0b35a03ZTkf%(qeg!K+WMN?xT9we7mp^S0e4ZuwVGMA8 z8+ONEfImQANeSeQew&CG2})|1;muY~d@ex~$6=F}6?F7wnbG8qEiDb=xEYa>5l%gK za3w?kAYkHjAgDJ*R1|YSlpI58pkPXjr=*%mn<$r3v@LG_r@qJPLcu+FiZwJrzxx4E zQw9=x9&v-k{0=SVpdgrS@;<$nbbg2UB}jvDd)%7eqV{3itvsm0XN{>NZnx%Pjc zLf};&McQ(d_|A{voh%3^iyzAqe)1%te?su2g45HdQ$hCfGiUAd&)s*@vp6{2dQCgz zAvSFT^miPSI??U#H-MC4oQ|`tO6An$#TltH!Y|e7qln{$-sNV#{7fc^lHOPP+{g)* zr*}X^ofcsdoL_fY*~dcP{7@|VbF8*f*Ojawr~o-h)hNeRj@w>&-vQ`ssc-t#mGn0l z`xDbZQDMLv$Uina0tIVlO*P=L&vhFPH^vp(_r*Aj7L2LfBzJ?#*bV}Tx2-7QlTi*8fRNA~T zY8I~0*`l)j&*)Lyj@Hn!Jv&_NjE^OemWmFSq6e_P{^x2b+~j`VSRq`f-t+u41-b+v3(i3C_jUhuLjQjV*H{1XmLPVnSA~t0> zPmOpi0&>I~AZIQ`uN?4p3|)zoN0PC;jY>^5paGM|7##!* zRL*&A@L5VNl3 z!<4T7m=2?KBde|CI)j2nu#x~iM9g9G>${Wzkx|A$rG}wu$35kIZ7l?DlA}Z&;PWj7 z&Af22g}X|V&)=yuJixpmeUY5PXZdyJW}E_mzbdBFDkqIuhfgXBOg5sKGSG<7sh`?u zroG2eVUCHI&PUtz!-?pOVP6Z5*nC_jCYzESG!|W|g9l~&kCrC(f%Vc(x<2}Yiyz$s zamn?*`(Hri7j(mMlWW1ljKI}n#J&Fjv2O*SXYWZFaeB>fwE--rpoi~m;;m@K%zx;!+|3U_E2>ReVI-Ut~|9!E4ZAPNYwK4chZfC`%P{Ywg z9xvDxV=6U?`VI-He}4IY)D@L?wE1cEPeT9SykrmKwcs*`cKUKMY_V|s`7bZ@1nxka z%?{gNufqQm$|%NxRN$cVy|X`F@gx5gTftp3~7H1riBCU0Q4IkXD}9%M6Yg{`lYYO(M0&7E)MmG{5S_&$Q34P>pNeH6Nt^ zoK+Us_Z;;=T}Cv|uy1DhHFjer=?i{=mD#Rp?3Pp9h|k7$fw#{WCx4V*-o1+T@3#UR zD@M7!P0#cH*n7{QsF&_tR1s8?A~@tAAUR4D5QITs81gVfj!MoF1w;fLR1hR1ISw%7 zC`p21AW8;7KtVu)h$2CdaQZi}?|$D?_g}Z}xAWmtxwkb1_H?gay?TXbJ z$E~F6wa3vdc*F-*#k!9W08e-jM={IfOc_H-sLq&)D9r3vpD}Z3@Om$((49L)HetjX zGNQp)aG1LBHZk8lEbhPJV`cc4#9#B219&{-CELd2vmVD~>EH`T8~hxZh?h0K{G4fT z@?D*%ux@V&`n&uIX)@|g1R!e%3p&x4{>!NeSs-4xHoP_?x9OJ*wl^sR#eAf0;9WZq z_u1ABIS|I@RkV6d(o1I=HaG8^Li!)JBIGX7;i{$d1}&RmBvhj+6n7*$8X?OaY}eng zjBvsiT1AsCUhNpRO;o?`RAMsuXHHT1)8YC*2|`HOLb5qL<~(kFb7#`g()uhum*0Kb z|Nb-AQjiRANuXc)XW9txSyPPt^pV{A>;272;P-&Kl*^xD_^)%ARcY_{b#iF-WVJT> z`p+vKP9VCgk}_9UJ~n7UVB5#l{-7xdt9hXDrR0!Gq)4;D*Q4~U*nf@`ulc2piOF)` z@YnO5F_8opl9A0QAgi^HD6wN%-2p2QL>MXIJK-#n%Vh;dVSMN56Ko zk6|Hg3Abjgf9KXwDmOjKi!6**9vqOMIbIHwz>$BeF+)&AU3H?vDs~I)ozYB86+8XW zM30qp`oi;jf!ADG_PeG>R6Drd3JJ?gRqDs>CVek?qdnBw{jr-^~~7=cGmw z`ya>P1konv&*9-aAMgI2-*leue{mM(>CluBcC53FkXIie%6C-Kc7k#m+$`$i|IdAJ zf;d3^k_wi2tb3QY3935dX7i5)r>F&KW6do`TEf!3fvKoEb;ssEj0!z1_y@H=9MK&n zejRpcc9SL_6PT;lUA7e3k;=y+uh`z|;jvB}NEyR>T68F(|ASh5c(kvMiwkqtX%|pvSK9N|Wn&ry*2GiT|0QYlLY! zJWv#eRi*9RVG4^o0t1{*JP|B+`h~M71re1)CivpN-}$4f(^}1f*T*;4o>>Op-P;{Z zYT0-za_`sSwR`&mY~at{E4ICZwZnTP(bgRQh=~*|i}dmh!u;C$!M8H+s+~9i&7x5| zE3J02ELAH6`m_$FzA%8_YxX?w4C00#kn-EiXVNnsyA23VkbF?ZUM0+y-VKb|K6O|F zC0i6l&c;_Kej@MYgS1_d8^~Kz!lz#EgQYOKL-v|L%`rMQS__0!sg!;Mxmg)OcF4JJ z$PF5>nN#Csv)LGovW%^r>LUl76OOV9`+ zf(4q;c*$S{@qytPRhi%hUG|lqCH=dRBc}Pic{sMDy&4W~RDx!WgV|M&dz>Mt2zcv>f39Nd+G-d$EB$gr^Ck0J zAMPQcPtj1>fdmkw;tAqGuvPhb>_?YWSSlF7WBYpPBWMJ2!U(TD zA(!M*G?YQ2(PD9n|D8k*t2)<5=6W3S+5kQ1v1?~J+D-##J12hEFhLu*M%YfdzkHBM zQ6vKbyU4rwg#Vie9luRb2E7T<(KYdZ72^0H|DUB0h3p}>%&2&^t=fy;;VsVwM@7m? z_P7Six6{8S%vYB?gg*-1W!zoL!QcY4j9(Mj*w^V}lN$t0ms%r59V=N8^pmdUM*s(; z=tX2(_84Ut1oUpS0L0kj_7{_TFQM zW21fo7HbbK+QQ2r7Q#j-DdH#=W$YXuq|CZ z@sFVf{U4{mQWDM%yD7z=rsM~Tt+dMU9l|~K%Y@vc3<5$~HC!pABch@Qsbz4B{W&`P zyX-l)kM{Z`8j%6CKZ}uA*39!-7fDFs*%HeU;ONfuRq7Rlk+{g;!(Mp^oqolvXT%fq zJn`30@Y@U3?fL3~YO3fkVP^aP!_3;UXn0%7KE8jeQ&h$TCfB&hy9(@7Sd`+cU&_ht zSwKkLE*E?tUjRsSbeEm?e;2eOc8&pvB8DGeQAC3jTA=I>4 zt(pMYpmp)-J@O+1Qpgh$EOU?2?o$`TDuvyrO5l%l81rkITlH9)$oCLr8z{X*&OlqS z^9S~h3bKeNLSjkh4wuHJhEFPgIFUP+D`F?z%&oZtwv`R2=GMsoe@r65)V*ZIaUYxf z>(Eag$oc>WK zA3E%1Ju~Kv27Om*P#dmC(@zm4yMaE2ZWH_-gRh zFrNIuz^-*oBfSuU)|g~Jo}6f$_*|(uD?VCHEl5VQW$E8#Wl;03zc+WxVONr{dK^ZB zSB{#F^n&y;eLo(`%U;3)DU!wS-XI9ENW4je$KkU6APy>?+mIY*dA z#7xnxdV_Iqp$!`0vo=&$>t$826hm%T|NUM+;*(y^ z2N+K~h-R^eoIcIS`o}@(Bpdus76-HA<@fC7Y$w5~_~cW~M+M!35Pa^MY(lFcaBJ?e z3mj(kAH(?+g~3nEu1$jb776CHOHOw>sJ4Ife2~h(#(R(&;x^>W#r8`#TjgE)A^XQT1m>NuV$yL;!Fzbm^5UIiu8NEP_(hCGSF%d@N_2n;svqmZqa9 zpjIbB;NYK-TNgRy5X`W_=1v>ISp=JL1st=tASn$xut-DZ;Fk3i^vLM!gF8RJDD{T9 z=-mcL&#u{bd=aDc315I3dMkeUvTPe8GB>am)bL=v<0#Q!OVxjZAd0(ISxTW{WKwqh zUtUr3Y7O!Ys$J@Q_2lNuHf_k6^1aFK51I5D=svWy!F1^oJX?NwnvcgB3kN8LveMH5 zC6FQ&Yx=*D-zS?xL{B(sbw80ML#~9og%5HnBv!tbWA*`FuiEt?02|7BK`e3kY%PM4 zUc#mSi<0u=t%nFUcW-K5AORZkPRK}R*@ATnaTcS(jwG;@+-7iW)K-9OOzl2fn`lwC!gBP7ev=p_T_4a+Nr5r6g3#s>LFH_qj(Yf+zlMjFI=(fEGk%Y_BK0m(89Abb-s6U=zyIXPTcBygo0>J-ha=UTb0gNfX zg(>nyjA1c{-2|vcj!87R7~G^}OwAXJRTK&5DHM#R-T+W>zZP!&H~04>Omt)YXZS*P zmd?!l0EZ|JQGK&UOzNzZ-$0myZJ*(u%gLXAOujtk1UNC>(7iujpCt*K@9wUYM`SSE zIv2}Qq7@4xItrkupl*b$6k-W<#gzxA&ojauE@jU=B#)6A?2m4L$GQWzd2|Z0^}i!+ z^5eQ5C2R91e&BSC$`>)$)-$bh8=-KRd5x8Ete5NuQa;ldC@Wb;@XFG+Xm$uL4tPcZ z3hBnL1Ek$trkziZ|i8GL#XtW&Es%EDa>y zkayRbx9zwb_$>50DEK)C>@$P5zCOGDqdcV!UBxvT+tHOEfYD&`PnZYjhZu%i@1tBr z0xdnK?tG1uLFAf(jQnauOmndWn!WlmLZ4mBe%L8{l%ke=WpC$q?Zd(|sxOveiWfq8 z4bIB~>#VfK{vC?f`pHf*!V~h0uS90)WiT zw45Y_dilB*ZX*-p!c0$r<$7i{73hB@7{hVL>oIKLZU7^Tegra#+m92M_OnKK;Xs9& z=Qr&WJdVuK=EJq$J$usTUVpA_eY|CW&`wD+#h}^U8|G<)b5c$>&WESASM-<;q*$P^ zHE$KSLx|cP z7;U3X%d{UV6b(Zu$2;^7cHY4QGk%vmc`0wb(~uqf1s9-x0svB|`5};_Xd(a2cx1?4 zDtuu*c3*a(RRk@1{Z@&f9naU?Z4TUa3yMeCoeN6|&+r*F-ZJs2dbsy|5ZwX53!9BA zWvSBc7A^85YrB(skPZv71#QGI)6YMbaT?|w8vu8bK}C<~I8{-!$(Uy>?enX*bF7CI z>IO=qF_f=oLM>3C@jRghM76Ai0-g_ttFlMW16VWWft6oz#om)h7!*%bo@+OQPJ08i z-k78H5~q4U8ZP9-Jf=7!MCK<(Az&{g{TXg2i@*DPDevaeHGqNl3WM>UgjErx54Sua z_}eb6k2;PMoP|U{jy57YlXu6fY&)AExVZ4kpYXVFR^Ub0gyvtt;iI*L9Qi`}dfD#& zd}X8BsY)$R*{won9TZ*Vys$!i`No|FKPnS23wF zsQKEorTES>VkD#1l`cn1TZauJ{LGj~cpTXfSLX39(7Vb*j zdwyNIZUbbtrG9&V(KZSl&9MN$g)^*otOoPcC&EBu4x4%Fy=ES&D-4`GaLeD3+CM~YQPLbw~ScAyd#Gtd5-|Q|rhFG6vHn}#*u?1sM@BK~2Yhe;} zA;j_M(Ar~G%ZCCP$w4V?2~;5Tn*qf0vDuB0RK3uAwmAMgMEE-Z*^l+#Ji)9cxjwg8 zMzxe{#cvjUA9)8nXz!C%u#92aEcC9bNURYR;5O;1=2BqD)#&%0M?^1cfpfOE-fi$l zQP}Gyngw_@quF$hryvjH{{2X1iHH4)@b)LVkJ0>*KO}laaY~QFIHUv_;4oDD^Cg=| ztK_AcD?^T^*x@9-p)z3M4fV}?4>5(2|F?o8A;reOIZUWkSV=>B%@jdf{a z4H}1p*|gST>SY@A%2f>KM}2YB204^dl3bgBH~tMU)wVPqTIcOS-5Y?Tu_IVf+l$_5c#CP!lnekbD3{5uNp!Qn@l+S^z z69PKd%>!sGns)oI>B~EnNm(<-^;hr$2`L{Q&T0e86VLAjYVaA|L{Qs=xhh&E<)snS zP2>f70Mwp{G<_G-Dw$;cbj_feDMF7HX9!F2S7CsfZ1S1vX!Y1Jj}uDpiK$ooewZb& zjTAluAh*)|K%_4ycwSy}qIKkQh+GUM7e;XFCg;~1iy5+GeBzDxBwu+43ic24Yvk{^ zpi>lz6t7T_U-UK9lW&f#4M5tVu<^I>u;;@lu8IkvPZ>2BA_%8md$G=#Cy+KKul+^c zg>x8$<>%^og}6(g>rb9E^wLT5l}C*w7BSyKL0S^TeWCn>IA)O(9KVme@ODxfj<~;7 zDP#*=5J^pxSHHoKhp%GeMHesoAA&*!Cc$XELeOJ!p->Cwq z#}$F^e3IFtLd~C=Ue8Q|GeEI3?M7q^70=VTKMDoCN2hlX*=1MN^vnc)w$6=vE!bA(r>vsEgnZ? z(NIbMt+`8h8){~i9!h z?F+^6MhpHUPBC}cU0GdY7D5VbOiI_ohu=AuHE@5AK})OFT_(GXK{P8mkL<)P7^7p{ zNFyk8Z8V5qxJ@m$sTisF(-60g!ENf}lZxQWGGn&(3d@^zrBur>TYR*M_++*HGg_*R ziHh5cUvF|_#z2nKrulEuD3oVFw5gsu_GV7Q@IRs%4VJ>u2uj1(0}Hn-2s%7MXp3jdHTKGC0b6(0W6}| z?IOO(xBH&+S4J&qK^O7ay=0$4*5z7@UiJNWT3D6ym~eF$8c~k;T~b`$GEq86p4rId zY@4xQ$-3rc4;%dKm{RR31gcMr<>5GCLUQIUD|ydxb)*JD9g>!GOqx}nN;PwMrYrF3 zt=Q6i6MO}t;mU-W@)OCS$_@Yxd=f;K`$pDp(CJqkZBSX>onD8UP#fE2thxqXJ@M_L zUlAGJCgo2(c>#0(Wcg46iwHZkr-8U{XL!16;!SuWKSm@B<=iY8X?fzTR4m-O=hA_msL@ zXJ{cI7Fp{uKmPlaIt;~PvtKyw_S{e>v{qxv{+G~&im@O}sPF#fO>>_Fqwg{1=O0Gg zAxkA&ii$Q7?jOw5h(^8B!g{MAHnFtlYZt7-C}%kMwS*smeUwSVLnRJHM`D+`V_)o| z_oJGS@Ox^fo^PE-0FQg&(>qO`gl9Uk-dV~{L=iRCGUDm8;^|W84xWrVuNMqv(zJ)1 zQvcRl6^vahm8)rraE~bRe`w)R`yJ|rDjCzn)*yPHO9&WROz|sJkB)tI8jH6)wB4b} z;V&`DRBIyR`?RNW)88T$k1X*sm@?7w>XDk-6#IKs%Zth6To(Mr(-I`ZQN)Y0Co(;D zcT2&At|=xUCOsDa%8uES2Tl5eQ3iBG7q(F{Nf{xVg&gZV0f4Zm`pv|P;i__rW|J!l zhOw`49!5z;FXCAplO)HJA~paVPpjY0|LKyU{s)gLJhjA5LT<73 zuu!$G{o@;(jqh!*K8PB;0bPpJi*px>;U?7u+gkTF?i<_OCX!`(OSZ97MYrNJJ~{cd ziT*6zue-K=n2rnE5Bfk82WOs?VC?vUia;^av@tKo8u(u_F7n!eAh$a~Q}OEQk+XTB zL3#Nt&xML1cFBta&U%e5cW<^Qc$~=f2t4wkU2hS>iKtskKQs#{VS~-*_PH#owcPJH zku7)$FTIt@9d8M0Vk*DiQX%!^x8GHB>@ZqD)A+94s{5Ym`@4Tq+5qsS#Kw?aIa6!O z(^W#YqTMw6&?CO2b*P3)+u0i3F?t6h=tFRc1h$NTd6E81Uwk9lV2cT zn)aAY7Sv76=dXMt*pz^SBUO+KJ{R;2mdg4PVjD9))JwF&Tkr&{IQf_aV(us#pa60JgPonFLwh*sfUr*uFbpm!&8=priJ5p~lwZ zY<#pBMuB|YtiLMzC4ENlfxmd}mPWor#ZiGVBLGsrp6lz7S8IZ9QJDyQr?U|Jpd! zFS0jCLclxj+9j=iWQD7j8FsjCx>}%ruxH}H^z3KFXbHTlOKDCX8N>MZ-$)GOMAno~ z32{RCp9Ae7c)JOCF8sy2;+yzxLj9`pJ(1V(K@x(B{Y2m(ZFk1_KGh@rQAw|@`O+r% z?ZAihJ`_9XWxdxod$QNk>`QbZCdH;P5l~=(THB<=4nx@_rAPMYP6NeecQ5h zi}deF+r`s4b#)j_%3AwZ$Jh{b(6|ukDbRmK{d|9-VvM>tqLlVC%;1*-&3W<-4}>C9 z8F%E^OLry^_Wm6HTgMG~(CHG9L&rGG=ie zQ0OvGQ}OHg>{x@_%F1EPPoG`#XEa9m#ze;rPSr^FRkHV@tC3M4;oDmm2?FsBy$-;cwC?;i!3B;nu0nTjU|micY{njC%L8;y z(;lM(OrVa`JdBOokKZIq&iqQ>x}A5U)7*?^e(V-3mENpEcL+|s=jMK~S=^T}w?l7i zp^Md&tO9)|tUT4D&${Eyn0l!;_3CZHPypC0IX+eAhld_*9uuw}dr{I>} zFXVSu5w`@|EW6&6`LZOp2alJ$sbN1vN_I9P_EvWpKGb=9o zLk?^DsW$^vXn#_g=KL9cHc_Hl`5Q`8`Eruaq3ZJl@4g-%fW!Y#Et1yusSlHh zDBUy`$M|b0$WM8-XHTwhOI~iDj&UgMnEyDwRho$InDZ^+z}qSdu`bKoRA*1G z;A+aA0iX2m*~}t>fe9rT7*QDNLAZndD4+#!jy-L)RW@9)0~Z@oXITONOP8DL^Db*W z2uQ8nxhC8>YVkP1;Q32jdA0@Hh;M#NaS^immY7Q%^A0ZG#5Ts$)gU=>%$Js|m{+0N zqRQq;Ej%9?6@PN~`m6A&KW_NwptO9fjmbG+O`vNTfW?MQ|4F^lLjaY|496mPBTvek zOWkGra95%n24t1u6U)Yxs5{2|c@_onsx-ElpldIoWGBiEmMOqjxnv-#)!y~;TSg#U z{B?NaEIHo3^P=pCv$%rcB0w}U85C6PG1zMA!v;qcCA@~Ib&W!amD;KC(c$q(SU%dQ zDwj)Yyetc2au7RQtl{;sLQ%5U&`#pi6zGV83CBoi+F8pKcG3904Tp7_;;(>NP-}!4 zsN7uXjrqYqJ6rxiiYv>$(`A&k0JQXW(KQR7|dI@D$M%$H$YN1gZJCD^P%^D^4U8J3s|LMM}w|i5)(n z-2T)--+)pj>b&|o*wC!z4g?_?qyh3H>yC|_0^;eH>o~+&I=!UUJlNh9GP5*WlX@>V zPXz#^r36XbANu@Z=?|a(j+jqX$<*5QTZ@sqPtO1)c-Ta?Ed?{M_deFL**(*!HA>m7EVai>1gYvH?AJ+^pyJX>6Y1>{|MpUkx zxs=1ND^E!#=r4dQcB(r)6pHNdUlvb)VJ{otI=H|RkR*0Z<8^?NC6|o22@&18OCR${ zVUx8zcx-6;!${8uF=Q=8+!DF7TWm_22`Ol=mFgRu6fHqmj~V%V&z9XTU8EN-NLs-N< zZi%e08k>d>L<(j&;kmY?uS@1$R7}Om{sb*!{K-gAbIzf* zxo>;kl}dQ&L5dHuvJtc*Bxok(XBrsZE_NX+dAY3Cyi^p94rOLq&C}4rl{y(R&2R93 z)^wf5B`tIDHVGE{2x*LO)J5DMoQi!SQvW6Ns}{o!uHXt%H|@SyfXbM;U%Fk>P26_$ zNPuR;r@!?)rne#COzEl;hV;HBy#DEi>ZSSMV^4Y$5h8d!X$t*ysUhqfczUliJGdkiDvYp#_GkkxB18+j5btY~7`H!u3nC~$2LhSTWM zCb%<^1ctEciU-b}#Rs&{%RvTB=trbOG${NVjBH^r3PMaz0Ue|=_-^!72CMuy85!~AgtgaSu`|0U6j$A^d(Iz%sq8yXfAQj8Io2QNg&R>n45 z`sN`@{horOcWa{w)9TN^2QZ`7s6>OTisg2?SrWW}nPihRxSj2b0%_3}+5 z-lC7D246?KHPTyOta)ba#W5vWF5lee$fqA{-`mS|*21e-aw5+gY)0qP)}3B=i8B)I z-|GgQ^*I|P0`iyy$(wh!;P^zpA0STgn8jDT)Ww-HsKRdArJj}sDJar62PP#L_9Rsy z3G5449WsAcO3AHaqiK#WdZK8|uHL$8aq(S$*l>l3>v%6r0-5UryIj+%zmCdWzVW;aJ%e?y zRfctHjNLM!bl4fz9H}WA}`EhH}zwzh)p^g5hPY4LY}=ciHaqgv8sJ;NO`?Z09WfF#)*vzfSQe zuQU%4#?;&?kZiSRAu&lYUh(O7NGVs(xv2cPPfh#=ELHMrhNN?%q)e64A&c?L{JML^ zB1018&Hre`8CaE=IX0VMQBQ=N1aQCchz~KE_-&ch{x**JPs7Ua3~J|_xJ?hC3q0vl z;TI*JJJx*Qq=biC6c#Q^8tQe$dbfu;L})i~bs39M)8PCbVb`cPYrRU~8Up5DVh+`)+b27{3d2 zlqUl`eQzg&2>*?gd7=?MbpzV_rL|FOcfZL6cqv9Cri*-9HYX{@t>$c8Z%7dJU!UW^ zF3uBceD;w<29gv|S9g@(+bo`_Hd!{h!1;S^E!#JK~CUs#{Edw4G#S%GW^V=s}6H`_TuS-SlUaDib<-ejjyer zVog~@pc-xwrZnF^pXP#qLcwFPiT5+fON$zL3}xUB%rtQS?3hpcfilj++t88$H@1Of z@bV6x8oZ&-SDPErQqh^yNX_{d)8h0be<$DsXckHToVZ27>y+KU%fwNPj9L?#u=>+fUzt1o51AWS}-$zWH&R1V!h~v8WR0&I=)Olsn zF|h5@O5H=xXEpbw`p;F9;XJCZ04 zeo6E(nDCiO!$vzAew#5}6xGo>z(XGZ>j3{IF3tAPg5HP;$}$?#ct5>0YEfRR$6ia7 zvVjHOZu(*#*Nv6(>C%+ls1jLjzFOLbb@i;8vY527&X%mQUYr5?a@7cnaYX#M1*0K5 zS5iS@mN;hrdq(h|N5Kd!{mY=($H(=pWc6hVH#gKnePl?WR25(wWqkNhu4=)(Yhrn^ z5>!JCN|${C>|_a$u|;vP0}6hM?!^#bu_d~BRRswlXrUYmlx=Pew930@Dv$xHP?zW* zP(Dy?!*qb^nt7K-qW{BtCG@=r@8b>ro~C2G&0*=0y$ z>r#rMyExRY(T#%ERuZju-6#C>FFWgLD#65B(}r1$ed?cmU>CY0oJZYlYP_T$Nx`{E zFMzxED8|@2h(zM^THYGkQv_IpM-*}X#))dbmX&;zE_SLxdK$!Tbt)T_haXVx+uOE8 z12V+sSv__+Hbt?ar8IUfMXM}m$Go?_otXJ;3Wz(8`=jnF=7x}U>I3V?lasau<1iX?1 znm!M@TlX{s^fNMOc(et1GXbLv<|}9kE&16og7%&E9?~ir!1H^6&gX+!x>ixSO;ZZn zW$Bh*lp48j>>4hs=;-G$bDhI3j2xb6AFf`{he>LZk13ae`|@J zU>pM!VNSly6t_s{s)~rAJiEg34mz&(LS=@$a|zNQZ)>!BP#v^li>$f0*9I8B_nTN4 z#BWkV)o8#aD1}tQy2zRk`u{^lLCuPgGW*}N2<?gDv1|NP&d$hGW!T$cC(iV0&7LTSkT)8Z_ z%Ta_DA99Xu<(j`yu+D|)ko!^JE{0UpOw4*-9<=HVAY{Lm&9G>_GI=Zmtif|0sdEUL zcPFO6U2T$QU@N4W2#z45dC$6MYzR>1CS_#dI{Yr(WWVS6&wihS64!aI?>tcp3n|D5 z=%T(TsP+W((`nMu;(kgWevil|##i8@mwzpy`$cnE!*m~;Ro+zL*X-ud>i<-BfAVy7 zqv_N{7}XLc@O`J(Q0Y{2pr26_cBDH|@jZ_3ovj*aZpIagjxPwRBn$t)t96eYuL)xP z0LN4(@(F~N(>-@GHJl5B*r{GQRKLr9#V%6+Y1;Et?1W*s^PsX@82coyTJ4+wy_`nNd*pgkX1 zBh>rOwm7(RfJ=emdEqlJ4)&6`GP1R(TWwIwS3ZE=C1st~2WQ^F^6@*qnCzl;&u zl9`1%F?INhgxq|rq!eWV;ACm58MIMGhhF!wh>4)I%f@8YTShiW5WwD6t{XP0D^2^OV|Z62?s4HB?z@zdv1Q4z{!pyu2~>c zMAi%bxyoCS&Mj=E?7sn@XCX&OuKnIj#ymN$#4)J@5J`3n>1sUQIy72K6b>@%0}105 zH}3L|`QBk#yh-+!O+K=zvI*re$s(qPjZcXayy27&%lSNLkMS&o@9I4}_g+U7EfxWK z(lNKZ+i*n{%ii7HJ!DrSA#x3Y$hS=|92gV#4oYY{{qA|+;YRnv5<3+Vx&(G+l4xTf^?k(vAr<4rLJNi8u4#_*sKH*~Ezge@g zGhpt2U-i6El?{Q}a|23L>aw>dUn0o#nBVzG1aU0_6idyz$I9C`Hvc1x)#jB~#_DO6$|c%p_p?z#KD->JL2EaI9A z*1_k%^|Ka3aBkvN+ZOCeFdYxhoS!8(PG=BG1j`PB_)HAaBWC+xj=V)tUP})DTk# zZDuZ9*y82Yo|PGZ3Th)8a4O<3$3A@_Ub`WyKS-TAT>urkO&^;>)&JA?QFa@^yZD;A)4ZwaBrQ9jrwMp$lWLF z2RO~K5-|f`<+FzPOL^r(eoz)Yx1pJ&pVA6?_fnG(N(9_!v-1TUYpqJV=Zz0_mV3$e zujgN9+q%xmKPRgB&=Ewf{VX8+JT-GQ3kMu0hxb-TZIOyuC`fJ=Tj_ZZ2d@4;bh>uW z{nppzcW0XZfIDgz{^jG3^aHPreU@FB^=aRI=IPqi{tvF1KmM$~>TTWG7_oo0cdN_r z&qu=ptL;I*(;jLDhEAGay|lXOYM&bD1LkMP*OBR*_$C?txqPkEobOu$+d&D;c#wny zJZov1pJU0ECLD~k$1sS`6X@A^!@OqS^bGd!dvs{_h@`LlbAUu^l7bu*u>y#=HdVux zaw8yO?3}q7L?m(+RkG&xg&1Aj=RG7Adb!aPXro)^In>qytP@9vOjRpzU5-S zBUO~fcKGMLUG|H_1URE{>gKQPN32K_4Z@ot7Bm%njQlpZ{33b)kVQk3;;;a`E0Z4J z8BkM&EF5I{(-bK&9iQEhF_(gWZoX@2o_2z+2+Om~!pN-1EILo>H!@J2_HOz%x?E5_mq{6;8awB46i&1pwAWNjzEp!SqOiaUP&?1HV z*R@}7X!x@|`SavRmp3$X^W1y}mYY8Bmq~@e*d#55`JHm7_(9nQ4U}tUI?qn0h>Y(S z`gH1*!9RKP>iUTo>Mv#;VDSQ4EpdI1Gezo4}@s+2}!hY|*AB_ZN-UMXzOE7-#B9*XDpZxr%tv6(pbXXUAclN;O<<+m} znF52yWSKmH9L>3@9!I}l#~>VKA8{Y>?h3Oh{5>40$}VL~v7ItW)=aDhEMjqJjg=B^}*My_DCf%&N7ifnjf_T^d-A)O@Q{_wp}rq<2Ma zAI*RR4sQ*K&CGobc*9Ul@fKpvM)=}ryxc4PT()t?GXWE73Cy%`sO_fhK2=3bdv%{e z1ljof49i;x=L2>Bj-Cg)Q`(gDPZJW7LsuEL9u~nlpL2WrVXOlRFoH*GWO2bf55pt} z{B}O#EkCw6<^sscLx;@`H|y%hWPBQ$af9xDYbk)kL&XAw0P4TJ7c=aC{igClBk<;( z@Hy5+?BbyZmj{FMPd}RkZ?4kb?_}`3!abn2c=zg||j54i#3n-VQDX zF{kTB)iYzQ2eq#6Q-&4Iu`f4AP2+r;_qRUjl`q9>Mi>KUSgg558!NFUnl6@E+AU3H z!&na#(VJRS6^gnR(-#AKKBP+b2@bz2jR`2^dF8ef6LHVCAg}T9eP8L?H`jMMbzbhM z5FIdc_hHe#@`Zpn&HJG(x&*kPR6ASC0$);xv@;c^!~*R3z5-?~w#0?17Ia5ph_%>j z=oAI0wW)#@VP(nnyh>m?7(gPTJ8QB|_CE*;P7p~FRA#pIiA@9`ua{iDl0p4WrK`Bc4bt=RHYK2LY=JFYy+@rn3%g}}8gTvGnEwc|S>67#RoA*|bW!)| zLp%4XtDnr(8z+hl0_Fkt#DYbfug3hJ(X443+u;{w6SX7PzMJs#fB~^J;;Ux_>I^XZ zU#^?e$_J{oZqJWb+oy7YKz&GtTqb#r@#%GBmg=)8@aGM-&D24fK-EsMM9 zYs3PF26U9~pEBe|B{GP-{29t?9J$M~bU#X1;0p=hN$TP<9wI*kRL#kP;Oi1lk9@+P zQv`x19D;PrPa58mUps&A=P3bwqT*Qf@puXUPp)-Kxo@GkF5@|r90?I4{rHCzbc}vQ zXo76XX+W1gXSJn~R079-r$&SyH>FWXhA{FE{rCh7BmLn z7jDT*YH#yyCB3Nyu;eR}%^bLikJeEw8!5T)38g^13icaM<7Mfqa#i^gE-@TbKT4gL zC%=j$UqR0TUQ5>{>=cPw_f16;{`UZh{muPgoG$k|7m6Ln(ggZg_+UB#HEJ#3U{&cm z<-fmtYEXFO_5nS@=Bdd=yMfb18C6zUc(NtaqN1GPb#tud`jt;N+Mh{F7DKqi2KZkg zL7k6t+c+yP0tHrmtg#87uwzc4m5aS599w%ac8(N0hV>Aoswgr~zh&+si zx-goI?@@VPSPuQt4G2>3Mq!n^5pa*D!Swiaf2`Z7@Rh&pgNq=#Gjicw6gyE3X*Tm6 zhT6ZZM?j2KVYD4AX)3O z%Q1Jyus~NjXq*z`MiuvgzS3oF#v>L8szn+o{}xhUa+ZQKgbW`&WkQr-nYX2t@UdXY z^p_ZT<;BsWi);G|%LCrSS0lxS8nb*B+VdzR4i!fyV)63k+i;Ie9myM4Jeve~Z`GPd z`D!NIzgmS`9wbq>M0~?P5VXzvAWMEeo*fTukTQpKIvF32L@`Ajv^%*!3whPSx_@MNs=pwZ-N2RA2L*dTi^MfV{N9mbPa)5aIk;<;MX&?XRPvo zCRdFu2+ihI->HS_n*r@c8K9>Jx<-oNZK`shCp~hv@GOwd{|ALh1*bEH)~WJ$oX&C) ze6S80AT06!ryRbJ%hWixCOBUO92<=k46(}yeh=M2{P@?(}$wtWf*^i18u-KD)&<9~4`|o1? zulUcwJS%ojIUn;^jDV(K=poBxYj!Hx^9EO#H}yY0{7;X45#qtW^D`@v^E~s_pTFL- zt$nIn>yRms(B6mt`8^#fof-H;Mu^>{bH3CZH53gTUgoNE1r1;U`=lQ`PQt_(Qwi^q zC>DUl$#Grn-_e5_&_s_(nRG9JJ%=sv_At{DWosL@RyH!V2O?3%bx~#1LEuz(3;Zdy#yCN8?XH({p(n2 zZPRqXicnC&Mr$PBvwW2tcz#JVwWKH{?|cDKg1r6CetgabLorKy2JeoH$uKWI?P~&E+*rppmbom#py4BI$GD z2CG-06HlUfm}-F6?pnKSl1KEtV%+nXxGW9&eWYncf49<5c{~(nfcD7jn-x6{agU486F~{OcVPH3#lm%E4o@@oyx? zMSY+Wn%o;E2Wb_*dfU4_?IuW>QvJ5A^a*eNS^pjhTEnv(em9G<=Ys@G=UrB)R}YM#XVA81Fs`y2K`DMeg6v*(sR9sKd;wMVlSHZ_1%I= zYlT{+tV3dn6QgvNmccX}O%0L){5C$cCqDhZ7-vVto(rDuqgYZ{9n#HA;~gFt0KHOI z)Je~f@1@kj#^2$A=~R84{hDNU)@RB4ZIPK>1w*+(p^#I|H4Yq{e;qJ=ao;fTRx?<7 zs~=?v(#1DEIae6G-dGy?By0`dG%)A@v_BCX*q5MlDG_)Vn<|5T0+ES#OD&e}uO{$) z=_=NE?HtI(J2u(i)l0EI<80FLr8pWiV@q>~4%Nt9CK!X0Oux@plQ~93$Fe?OinZCF2hN_>ani;ZVdG+r#FcKG0m2`=7jy%O9Ud*& zOA*w zOgp$=gs<2Taa%!uTsf`C8HC>XT&4N(K)}oG{KV+fGWQkIH(Gp;G6ik^sNGh3;FEG%?R;b;`6SC>0{a=)QbySpX5bq~~iiWcPjN znP=uVGryT}Ru>zpL$A<1gNP6obC2RGWwqlj!$+YbM-Mf8`j*l&y>MLJpOd~rrJY_i z`{$oJ^wm~pZ89DpU#y7l;yS>tDjB}EkfpgXy@J@?-X&$#N3bCbfoQ%anyeZ5@;W-6 zk^jma9HU_f4P)dcB?;AcEJ7iJraYq5U55_&={XpBv7PA@^B;KAtlT&mXUkY~kBt!- zjthg#WY zlzy??vaLfTex%$hd%uwFZM>l0pE(d9SrJp6=ADOwMIwV08T`zU9`VEKSH*W4^%M+l zr(M2DFmVWnhyh}8C;TX7Aum<38vprZk9;e%YPxK!bY>bZR3c`0=kI&luv{^~*a#b7 zUfgY&FLE1(FK>s?6X1o8Nj(BnOKyxf^iYARN9PfvB)t7n;?Kh)0 zl9&_|=0I?7MsU*f=M=lB&%_M&LAeoZi17P8yV%7};P16RAY1`<%WX{3OWDCFIdvsZ z-p@Hhl0o|ia7tdsw`~wvZtI(Pj!`U*$E~RcO5yEg?1GD>u zZ+n|Smvc@VdC4Qp)4J4n7f{yYH}OQ|LSuB|M!5Cy#x!HBwEkqH=8h&!LaFz9-n7bZ z5p(RlvTW>?4NSRj)KH_z)Si$NHXnM1el4-vRxDM4`UvQFq-xJ|5Ybk~ktrgr&a8cl#~yuEvIM|TjgxWh=L?-Hxj3a^Y+X%9y*npZmFIE7 zaT>RQ*g~e$e_!7T#V1f{g*rdC8vPi|wOVuU%7WkD1ERtM2b5(45qXo-V@aTzWdkF* z8AP0|#)G%8=IKNnUr{IxH!Tb-~tDKtDxB&218 zl=%F@M&unSY$3ul6nLQD+BmrZltcWfUh`aczuYD4y4g(cd^B z_+2CY+ng;$7!yaBgrfV(fsk6 zw@;D$jXwpiYq_NqbuDy12R+iq81$NxMsO`jr1_O)MoBuv8+O0<&xO_P^j=Tn1{k4v zSbaG&j~$S24)e_Uo&&6Lv1>uzwwO#v& zM*lTGb%RiZxrl|Kc4;?)a}_iyPzrQUCWjj13%;_Ge?8jq?sLHHu;VzC^nLnQ=Hj$)z^ccWgEY$2^m9e%-xenw8YL>-ZjN>Zj(e%I7capW` zH#OOoHw7|rne|_JtCQ&=Ucg@5(@0(`z#+{#cIQ%WW#Bg8muEd7T1p+fF7sts<6*o$ zqK<>J6hYS4C_`>G4EH@vZy94oN?6!vto1!+D2?Y*;GU39(Iw^Aemq?gKRTfV(M;+8 zL_ArP%t(&t!)Hua4T!!MLA9>|>G`oD8=tA8wX9x!#;H|vAI^U(#?q+&BaH038w2u}odM^!Y z(A1Gy0q8UR4|uKIf_`rd;)$bKH8~kYDOmVOA)(wNV2pV7dDmD;Xl4vSpq~@R&Ff%Y zFm-U?TdxsgeYKG{<`jgtPZYV3YirqQ*-#}eMmdxaC4SpA3o>fKnl|{dJc+6;etoBI zy3FR8IrTo3$Y{NyrJ`0MC*!b7e)|K0U-?;gl>* zjT)2q^pPC|zXZdXQ*hOPNxfQ(J}X|kOD9-2H_zP2)hr>lB8a|A-P^~>R3ZmP$`-GLS>3(z#+LPA3sT3sS5(W>{; zq5{2FqvZm>CpGX!TQ1q(*kPZ?edQ>7O%HLa;DhaG{)iU&+iPeHF;kpAM*UalNz9!H zQY7qa+lX@Cfo&KCG^T&EA(}9%M>Z^MM1-0r;m9|CKVF-)kdrYOa)$#}l19VGXL7EK zNP2=;Q#~m@GQ#;_19M85F;DsIpGQ+p0VPeY_t;&w5szj`;amIJ!}AW19t2*U=A$^u zUR_@*OjdX@%X$;7$>q9|4EDI7Wns^ct=>llFhR{ zaYAtAL?Y4^xEMv<7@^La758HUb02q)C9^$@jVLb~xFRvi`=*JbE3Q-D^<@`3*f9|< zvRgX_x0BL?v-7vzv4kmFuuy0s9@xccn8B(^lo|ZC$4a53WR$;S2se`_3{yy-ZhGU8 zitrGRHjSd;tGYQgzMROo;C*`CsK>6ir8TdX$ilpPL)f5IRQ|WW@)$oYMRjoMV~#%l zVwn=XJ&s|6$m3I6Yo&IXuW*Q@jeJ(T_C)L2iw_SvM@A?g<{x->-?r>6&corJdp^fC#(U|aoD2`*s{V|VK#+`JZ+ojScF!+ zbo?g9_ZKmq0{3&x#Q<{}CH9P$i3g2xDD~6qgiGgFqI0Uy6`Te!&UZBKwzuo z*IKvhya3mEqCR4D zlwKi_mA$4ZIH1~UTVdCIwrs$lm4B%oJZ%5}!kLAVM$d3VLS=59v_bwTJXG}jWQjzh z#A4#B_t z6rDT{2MZBdoaukR`Q#pSrE4S`wM}taz$(+_4)jU2Wk1nYi-R+_iln&5fNtFGfd+Il zJ^pN7Z4gE}4pq-ce6MhFsXXh6OH2eJ?Teh;cO%+3x-6DfI zcgaa4?ZcWKGZ>86rG{aXT|J;$9|LHiaw1LId|UwnuhDZR!*Eicv8L~JvvG3x1P^>;i|RFc$?)Ub!NtJU?XNlvX*H*R!hKkB}8S1NSnpRm$LLMq;Q8aQ5uN~80Bb}4Ldkwm3hp?oP-_aHs<8}DPVL-xjB ze^R{P9EZGsiq(vt*c&Tc_t16@&rxQqtD4Gmgf}@qrWB* zf0+Js>K!bNFtZAcNFx^L?>C>rE(bJcw}=-A;arQ;M?fuk;|ZL4c#cax84AEaE%WVW5RA1*8Lj}v!35pO1o ztcppl9-IxlKwuANjW6*Cwvo1YpXOvs5jobL*dg4ukJq^C=O?%cj zMq8q0SkcEa5mOpnieBP;Q+37a^_H&DXq}iDWcm>5dR_wLt-ZfHLJ#gXJ=qZNzn<|5 z85LI1iSM+*Gdz|Obnvoo%K05x$5x>a;^<=-4UxL^ zZ->P-Ao3wJr)+&i734B=PBT~Tgeh?4_D+6_rYq#~xsv@z(~+U-kk8!aK_{t|+gb@v zFFe0Xs4w%BGs6ANU|*GtT9KF%|2vsx%9xX5Lrfq1uuV}#z_3Pb82;n;R)r1O)VM$w zS$bkZg@bM2b)5%r*MG7ONAmPUecG+=uLaTHbSN$Fwdc6mz-Zm@3quEVDkw?r8mo2y zBRYKg#c1zJ$Y{SxDuCybE;l;d61hO$$qUa|>EbAPb6w)X(OKA+{WQE91^E`q?KQ#U{SUJ^TE zVL`_-8FrAKbc-?N`_JQgbJZc7^9?D>@W_23emVx#<@*eKs&jLVVI?er; zOvlaqUCb{(4b|ynET8cxVA6GizGdUe3V-)&?Bv_ZY=6mv&ycs~<%Fydl?-ipys_pu zTj%sqp_q7;8C05y}285!Lg^gY^(^|#_$ zID~OI*;5_;Z}$r2>H240XuhJB6_hM!lZ!!LM3`As^JzeChdmg?m&%v1a%{v95#KZu z4vP_+O=!V}hv3yaIy1O{bBJkqvMXFZ>9!}yP_?JMVJ>o(SN_p#m_5_?l+Dk+{?p0N zb|S%Q(Z@?losSING?M@Hu_gQj>vn!NkxiC&%5;io{sx}9^CZIp*jRvaNw&)t3b~La zsYCI?e^G8QF^EHhsW5ObV{sbHVZ8Y?nisTiN_==sxQ`gTIFY!pwu23ctfyUATV<=( zdk}n@nxK>AS^~L3jG38)hlczJ@xq zj6YeL1(?PhGsWFDfu_!UFJr-p6!R%aMwMFUEGV!AWTVaw9{TB}R9Et!?2|0L39?Sm z1_oTQ+A|-k3$Cvu4fUtsgLW`a`ma@e+!lr-^*di;ABI7=>%UY#X18L#`zQ1cbl$0B zen8JUo#J-=Z8Ji(eebV$)0AMg96n2$q)26)YM1PTi?jV-rt3gJ-eA1gPpWV})=!Ik zy=sGnr0%kYV3e8}fj_aEX)fN~++^WrcP(YtVpT*#H~z3b2Ymu_>9&)FS_JZM2cnM? z{Z1&~EE5%4(HkVUvtBVQUj)LnS42pP;{tWn ze%xYn1beG{w}bG< zIwjX*Mjdk5aB1ZOUpkhVL)gfb5n~pxO`Q{y(+rH#z2WC%=Q>Jr@16&n9Vn(fd}|g| zJlxW;^)spdByT5#rQI{>k@$&1Fqlh`4?A=aA51$^?G zPJRFu8x-iuxgB5tc~M$8hfUwYGpY12T^%W}XKfx1$paI!cwbEwh~(fCqnd?2m5tux z#v!7GyIVz>=jWw^$!2eO8LjxCP9>-pO4TwcckX2@aSmFm9NxrWq?U^1=VYBSf0dY= z7uVnhn!zVQEQQf1N8r zg^_tyWy<_aDiVIon9s-O!XbTv5hq}F)?i}wR)_LID=Bdm_Z;p}XsjW4z7YKG%r-e& zrqu7yrJ1qm=hR17u3d{r`MIvuNR;5}J$djaY^Ewo+VEQfPj4lP{5gvq2#EFge-b(j zLaVMuY#3py4&W0I3M_RvGu0Tp3OY2#mnejZ}1!{q#=qB1I$fGQKAA* zawisSLyle90!h4BFzsM*s&b21Ui#Y~h`1U~ zvc?DbwJ`1mgkU8~^Qwp%Ke(zX0q&+Qb!b`}>7Bq!x8;wolm@Voz$ap;deoj3Lylc( z2AJup;V}$Xfb2bY@cQTHlc#gZ++a21n#&Qj%g7yLD2(ZYpq0AH)AtMoO-TK!YZsL$ zrAh&IqXJYZG}=W6{YoD}z6#uP>D}V{HGOe<7!WHu-PLV;;mFeHH(RUmy%v42wcx!! z&_t4ScMvpfXqvKEC9gz;enh=-Futr2&eIW7Qq`eB#_CRc{o5}IFrO{@je4CojYJuy z!+A3xOM@5sNz=DT*8EUV17GA*Ju@;)?RHwuo^)V~Yh{RkM-l_1=W25JO3K<|y^{s9 z%gU-w)7|B4c#J(dIek2hDmwmT%!Hret(=tSx<&u6ZYhnep4GLU;em86BF`bm8f?%u6n$@G)3x4&qwVRovy6A!t<_JP=X&h7tXA+Jkf2a=#S12L_KS z>wExHh5!n-?{xLey&Wc)kbRicUdt=2pp-1SN7p0^P!sXmUZdc(Z)^ZJptP))mLi09odlbIc&``{Z@dt^Gl)?FOtubhoJ=eE$ zpngeSI8K&k5IslO+m5n;A0{J>L`_Rz2J~pjG630<`LhDm=xt=yxL&Z*gn}0{|Kt0YP>6#lkTdu9(N}yVTZ#b%ig#CHaA+a!#YY^InJj&yih1 z;$Y@8mQ-Wm(r`;+tF#&;S)ty)`D=>H)7*o}0P|(Kbx2+hLmZu>g$Btoa^#Tvu2>uZ_1N|n%CQzHJXF%q5&xASRn6F+sJCKg3H7qD9&49ov*hIZb7W6yxw>)3<1g&Z`6>)#aT*n57b%7sFeGKCQvgunUSy9=s{BIMh81fjrj7hJ{4i|kD4RQvx|=_NZ{%RX&Av_3>Ula`FMpdMe=Q;!&p^udHxNpDCICz zZVWYlLc4gGOKxRDr3L`74>B}}zW+5GJ7ND>`#0dc(f4NHKNedgelQI^ zSP2giUR~G+u-oplF_hGZYUs%M^NT9%gW67C5s@=8mw3bM*^?e~ndP1bE3T zu$-albja_|emzO#5Ex@B>iE$Z8uLiU$W5!X?OYsR$-IGH__#7oE)w5Li86Y^zg)U< zH~v)4>c^a?fn|y(*xb-R7F>gYsybHe?yT%x)E>h97*Q+~*CebS*kXr8eq0uuHT?h*~7-IzxWK_tl zK?S_B+sKcuI#41Rj$DBJXP-I2d4G|hgvbk+&pC8i=WDWa<}TQha+Di4i$0|CNLS#b z-@`U8!S6fY`RKGF>p3J)re#_~*t zk3dKnzyBW9|NF^#Ggjb&0f$!DOOwTo#{ESrbqzUlrGQGYP0hAq`%x+?@X0?h`8$*JCrKy>|xpwK0 zr5u)O0h{gy_EOL;G6cJ<-eVc;Jb(K6flwtl47q5s|9c4D-w7(Fn+0Cbty4xGn*R->$QVHw^%FR%M+7wQBS34l@dUCdjuGl)pUgF)%xWY&P5%+wbEv)g zczRiV?VXyK6^+QszO}^f+p2`tCC`aeL=OJR71_MfYSneM>l=-!OgG%>fcdvu3Kt_u zbO<8bM-fX>n*Hr_;h_aIetWT*x$b1Sp7~A-P1x(8 zct7bwP~&!0A0w_?4#^hC`s(_e?0scPcU7;$U$BGW-JZS+YMz{TQor~daWXYLDcMJ) z8XL&_j8Ml+eBt*B&E?~vmm`O3=38~89Vp-C72*_ByURqC!@LJ}eE!%_NUXucrX6sBCBJs&%end>CG>rAN11)7 zpL|_+s&xG;Uog43-p#UOa@n3fwkeb81KGNGV%i)|cXl?aUwe15hcw9eVxiFPO0MK zsa@Tq;dlOAM_ml(H~zj&U^2zQ4PLRB9glH@HO^Em-2cyH#sZW1IU@vxO*}7)#}U7C z5^msHt;Zj#^fo|nGR~WW3HBo+d5X8Qez|1Sq31VF9G8p%-{sw~p(XY^3h9Xxp>^mw zv`do>7+|A@T>0~(pMPjIRKwEBKSh!I10!_TSaHC;4fp>Zt`&~L2Qwuy6J>&-tJ((- z7f&L)lxY`t0YVV3V_=(x@_}a&BvAX+dX>nVo8A#ta;nkn5RN^z8P_Xv^cgUqLX72z>wSCw(7Bc|E<)Cnn)9&DuxaWK_byXgZPoFg^hXst_Oe3!Yu;1VsE@@aW4WEGzetpxN>E>ycCu^+y>=dzoz~-KQPQLu;|KYgGN8#do z7zU7^nJEIJC6-W>fcsXKQHNb@k4=EK@oY_}z2{>E4?hOK`{4N(FbCbF$zvCrzkVHg zK0^7&(T~k?wvteNN5utXe}xLT49T!(M1>zv?~ARoD^)Od$5~~i0=iT9OR;OMu$y}a zqpF!;IWVwIA~D@HGL-f>J|NcFCX1^M_2yFuO%^VZDb|0p#_gOAvoQ^0hm9#%+K-u!*^GKj^vO4pzxgMy%(he zOp$B#6Io?# z?#Gtlt~W@#Taj&0f#-}Aw7J!20nxjPeAf2&yF{_Nmof#d+JEiU?*48T>I@7oSW+`z zW!bksB++7Jm3^JyJJ|p93GAX_6n|ZfzV%F7r=k@puU`oD$`01p!i>uW>A~QQfGs#ZF^F;uynR^DnbL0G@ukZOHSg3Bs7)7dn zc!KpESzM!;LjT^jpr@6E}?k#{vqwo@bu3%sjR@ zb3@LgAq!E_<6l2+?Y}4jhqlfn|_^mv6|nR^=7iv(nZ zof$qOiEbaUc}KRx#h?Gm1Di7tlqw$=UI&9jkb}q74_=r0OO6!^<}D#j1rmoA);bRo z%yKhT<@3PFgw?#a`0Igc4ysx}=V!^~yh>F{^cLqd)`z>dC2yR)1`7n0T&g7lu<^DKk&&ytsAS1La0}~7KT$t1GQeCD2rT)x z1Oq}_7M`zAbz|F=t_Tmrwe}hmIT_10#?O6l`W%c<-#VB-u0tz3P)zpx)xsHBQ#&psb5h7i8@9M#a1V03 z4U@^t1IM{;&77gKZW*$>CLJ^nY~VZ4nb1s4n$sOLW8W#Q<;lQ!Qyn~~dj`OGprq(D z)c5<|hl{UyXx_>dZ)KC4Jq%X>4e;r1Gg=pMN`y7Ab?Np-i*?}EI2eR|_=r8C&qUY< zZ*rme1?WQJ1{hyLigjL)M*FfpB=4_rg(WEZJJv1!?|9`rC@9~4)UMmnk9l1THqU>J z&3x?!UThd&fF3?M*DC+eqxNQ)ZyIYG@cY^OXfGvx_A0!4nu6qv@llc(FedjYsG#a1 z)(9M7VDYfv`Rgz!m)-1#iC>O)z)h*EP1>@@YREZpk`?3x%!_;Nb-Ws@dTRFd5%|}6 zhSnj8yNEAAJXOurD4N^7%us+?I<%eA|Fc4lxcLL{1a?f)?Im=okmC$sV!uIZJFXI7 zWt#{1<=w3LB!rnl=hP{wqXC^+j#m8aXS=JP&(BZSrNzz9h+e~e z=lnRaIPMRortV`My3A2jXAXOW>^GFEQMigz*Toloe>;|2a7cOr=O)>G<7e#p>x$S` z)}ukFbDY2(9iZDQDIFC5afqO#Np~T7OZOP|NcKZ^KgE0S1)SnP66Y5K(S4NoDJw%2 zK%K{f+$!6Y2@;&h5X6~9Q6LgIr7yu>XZ?vdrV2ibK*x7Ye3aZMe(1Le$o8AE>d0LP>1$0v$JOgGOkP8x-7E;fhyDU zFhQ%}kFAy*K>uQ!%-&&x9TCOgzpBWJO)TlO^pehdTk-y^_>8iU$h2X z<)!C1YT@0DPo9!b*po$6?W!a0PKX^J`9*_Lr`aXDi8)=)D3KfM=*&yJGO>qb+RKHc z`6s+1B>0Qi{AiJB(`sa9$=t74TEe4Ypg*uF!z_}Kye zegBySL1?ni-U6;F^An;g!7NPd#P?o@$A(IExtX@H^HpjfWDiwt@@3PJM4uaAjJCAj z(ou37>K=U7mk2X$crTp-V^yDWcZ>ihi!QKQHy2W(2Bi$B1`ec&JB0J%vH?Zwq zJ!!nn<5-9*BIfvA4DhFuu+ik`*vtZ`vj*q`6FCv+seY#tl}7IQZ1S%mYLGUa;Y1b% za{h`+es+5H%k9q)ZkIgW9E~=22hCbJzs7c)xW)xWoZcWbk#Qk*$vQMp`Z4%D_Z?#c z4EOm5+v$&&rq|F2E3=I-AV-s=v`aI0$^uZhLKJ@-Q%+C*SG%R6xbYQvM z#qrdVtr4HtOn1#hEPi-gD?@t5M5$iRIYXH(7>xsFuIh{Mm+4lTpY+!_OjjLUY+X2A z-T-Ywzh5%^8Y?RKM#_OlhY==j1OBET~&3S?|vTiXJ0 zi90Ac*X)OH>A!ysUh?HkuOJXHPrx5=s)s|pB3RFvbsq@)|9Hz11H+Fb_>S#ZG0e5$ zTjn^lj6Z)P->1bZ)CVK2~;?nP?A zCEppRDG$d&kMI!e%Skp+VWs{+sTI1%6RPs(8DzT0>X!;u+rUS10Jp&!=67-4GEmc5 zaObYP&-Dq0Yv%ER9X`^y+5a#j{p&6d?gv+P($FF%K+0D6X9hoD?nS0GQ zypZG8MMxkL7qKAKJyx9WVyT?v14J-8X{BLcVSp)e^xCNhFIfQO|DwUO8Oytf#a?Or zZ?%r2*r0y_{r8JqlUn0+F?M*KUs?m?tiTG|8utO*6iM(#_+?*253uZ@`$~hCbrCKA zsHdy0xwrudqyVH?b>b-Sni@nJS2A?4$9?%ZPLoXJTwM>$A%=K9Z%WwKov+|TK_gy&*F>##I1Q2o%WW2i%0 zYCanJdnJ`jd2lprU=pVrIZBVQF-eJa{--IC_rQes0|8veUE7Yu;(n3Cg(~bDhDU)P zI-Y7k0&Qf&M!uWDAoc3H@3oG#w^+4g#?b(qM=6m_grs=4q3UXARt)IbA;QOY{|zX0tZOgRbu(U5oR!9T7H>J20tzk zBWcaKdT(2HI;8Y1odlTot&aBrbdiv+g06`>)6(T}_^z*8@A($AT$~s1$1Y6kpLyJSh2GLS^_8C<_H0>I<@&n zoquJ;EMv;fxf)x-fdf-R1Kh)X(-{oPkF~=C1};Ac{C%8&E9DEI4v?}9pUd~aR^wDU z7B4kC68zlyjd#-}vv8qCAH#?LY4!C0+iG+Z3E!PC3J7)!a26dH3B3?0;N-Q;I=dabvxeA(Drw>5W`rz)k&CmT zi?j0`KS0D7Y5~1;A{IpYDhVd*0bnR1gMk1F_{V~G!_9r*a<5>u(fA`7_dL|O?beq8 zY%T(8^DuFln3`Pa5-L_66nc{sQQ{S(- z<$ct+$Yu3qwSt8VsG!_6156w#kx&Q`;|Uur=vnjorQ^p`%;<4|+M04L`!gzNfWR@z zSpy^ajV|I#xeBU-taX9 zdo#y%Oa`3p$dRPp@v<@8z3sl-Nok^V6j&2P8_8zSp;NBfWt6wnTOfk-@D&QG_yCmh z4n~>P=E8JLd<6Me-*`TOh`4l}UbA>e`F>$2EjmrOixP5D_RNRlIExzaylnqycd0y9 zf4hbJM_=NhL4VG^q*x_L3wAPpPV+i-t`%XqH2H+U$GSbqm=gMS()Vk;ee~Xtl1XxM zq&RRYk)q$CtnwMKdoLLxkHJWnN4&%f@BZ4Uq*j_6N>M0HI;1Y*T4?smv`XDdG{L>s zfna^<_4R*7N`Q&$SILTiR_T~hlHMvKJMVIh8nT@GaLTp1tt{w37<`L>InQVQ(JgqI zrcAJ}&_mKxp5MWuln}l4=tugH&@eVfW-qw86b*u5awE@x=6gR-E+$``Jl>~|$`GBPAyqxln;tNPUD z9^ltd+&-rHF%H;ef%0vYXtrAgmlhruSZld|DnH18EZuuYL;n9PJP{UW1Et^#S<=N1 zTGe`3hyE^NWi$#U+lWEs-hOUJ8chLXN0#_%7-vFb;9kwa>OL!8_Zf}-T zR4lTtayOiAvaeGFL?v?*7w1dMcTpUn@a2ZbiE+S zmN>|!stPCx;h?+SWPfj1j>@qa{A@Xk)nv6oN`XjH4B)zOn zltq5GgiVWHadILD(<62SAOz9$cqKa4!Qt8LU+>)| z3xW^Ed1hd>-8$g1bd2*+xbP8(fKWg86t2=2ejH$Wwub?b$gxfu>Nh9m^aMFj1r2%| z6_KyxUjAXuWi~KMF8u=uD`vyHLx^|Tfh{4K4WNEf^;z*(1R%FvWuXIAz_XvPB^W1| z#c(~jEaHYT{gZ}fxdgEU&X(iO`>SMXM3hFZB<*?yOmX?PCc_)!S}MAt7DS{fGG$fp zjZsRO?AJB>zfiiq^Far()P?F zhge*Xwk7R0cX2OwK}&vNyYA4P&8IOF$qDmHlujNlPYY&d@JP?}t5^7qn3Mqq`d+zg zTacbGs5(jmiyPAPrnQrBEZ&zK24lpZdjC0Dzc^pK5IcvtaOa&ItViwHGFxWNuhesC zKb&t&8I1&t;d)_fklFG9b}#)Yh!0eRE8GYF>X{w==PQt8uZW?M#ru9>@1Z(E^ylV8 z{!Zi|hv>nEb``30(|_M8KUMP7zH^voIDJt1Ix}-%YntmS^3)AH<{2<%7SBkycW))iI;tGfi`Z7;V zaZ8SYXuB}i_~oTfVGkAE-_(*PQ;cUl-Zk&Pcm~EIzu0#C-FFfH3MRw|VT)Ct(<9u* zgnt|P*qzo7H+T4SC#-4+i#YTXr8&j0G}oAqNwI4^BQ7E_Z=;)W1k3K;BB57*<(RK1 zZ_%E^*ScBz3i;|rm?SZ?D5wA1jo6d>7VQN&z_c!QT&=(3fvV@P=d4-tdZ4d6`OX!;I3zRW{E!)>Q4 z@e|b9+rbD#My+w!ypwA?7(DSeJ_II-`3V^|mvpbfB)<5TC|C88?xuDd-29eZ;E#c% z)*8}{3{)Req;;L5`pdO^mIM9Nt_ zw=MffLCU{)0$xh|BZyIfL3DFp=LUtXCG+Dn=Hy|H5neil-QFE^@^i$Co2bNZM)aYY z_Zl{GerBV*zk+$>v+-5Y@@fW!h2mU{uoJ%A4i%6eOc$WZP@8Q493(BP{O{H+9fj=u0Ud7yc$Yksie0=d<+8;^+iV{~c{3Lg z4N;ovf|D@0%9J8pCzm}#y)U|%wRxtuu`V^t+kWk*S zD`_&^8srkqg+H;BFC9I^cf9L|fq0M*d2%u6(&(t&#`Lw6s7$Jt%KP$}mA}UBL4r;IDcVI(ho*)scL2wcX(er)ck`1$6EcEMSH+tW)4t@gn)jN%NDo(t*b!=kJRKuNI+b zHs+Gvyd2aNU{dE3i|(ojCgW^z`tEzSn*7R^>&J2Knt!52%;*DE+)v5fBqzx(PnUl9 zcV|)wq9lY7T9J+^!E(9BWL+m|p^I!9UQAM6pPLH{?z?~17iqbSQbGA{HSMRM&sVQM z&Ysc}`S-4uXZ{6K{SmF?`Uo$oHu%W8qz)tOTDXj?`y5y=zScEj58{`8INkTgu!W|L zY5QlKj1M`RLvofl(i09EWiR-zdQ`m%giB_26}2bXe58q3D}0vh%oA!0ORE}FRISsM z@mX1w6LNZ^p*ANZYk90G0{DA&qoV8qgtM$>+7X@XrAfuAj?@~mj$o0qt{tP`2!qe$ zKV$iuw-TsE=+JF_*#dNN%6%&c)8T+p)<<>aG4eJoqI;PT>!ii`~GT6 z4pL7d`j0I~Y()sBsl%4vlZCAgKNxhPNg=*>dh!lx%pj(z`~G5v^nzk`z}N&6#Stm| zLRumqfSPr_(RfoGB5&pDGRFREP%Y#IS0wI>N6vflS`@=H$)gYSXcI_7&S=1-fD^HA zRI*=>imTVQSn>;S4)P~G2wri?&QrbDun#pWPOUZC+nS^;dH0ljZ+7vIHmW0S3?8}D z^ftl@KiyUYVjDfkW2j}vjNhh~Gx$g!J$PTapyfz6Ox1==Kje>Ms!|EjSiUF`2J3)U z(Hf%#eock_pdgcVGt7SxQe?%>{B5(3>*=Hm7H^yWTOoJflbPI4?1MV*aeb%8MyKG@ z`T$=k6~+0HJVqSw^qlURbI;=bk<5yPst>yizgCr2ckQ7MW#kKzWY}3ETz$gM$#_27 z34fRT?bofcoR3axF4~i8^luYSqq)`q45&*Ah3hdwEtwL1^V%Lw!nBHuWdmsoT#TKA zNNXOlk;QXnG+pJj>V=c`XFEjGDc!GY`=^>$uDH_X1xB*oa+N6kH6*Mevoqr9OIS7f z`p5Hv%D5plJw$|NsCnNZs-pOjFKZQ<|NEJ~s4d>cWBoZEA?n0BPXC*`% z#`v7!Di`L=K?*xd3OTrnSMmjAq4uo>3A0eEP{lbP$+5|rAuXrq%&x35TA|9qnbNwe zYd)NHD=uF9v=$5lsCe-E?XL4wAs3weCKyc6QoF2;+$AS0k}?W3wpXJzK$1~S=@GHV z=TN>Q-JuzeO(d{Zu)RqZw#fPV_1z%TttvFGb1`2u-6N4+MpNxD^GMEBfG%ci5}w@2 zgYxwYj`tzOhCZknFsqpBiR77xOmR2;Y?~TbW+`y;RoHxE7v?*oxqd|tk#2k{BK(ad zR?LI=S2W?ouni=+PBWpr;J)x|;lGVH9w50GnYa_ASh0RTk!v+5bvpIGJ{ew6^UPHZ zB}t|d?XN^zh))ObX4&$;)O2#L-gM zxY!d{DtepjG;j;IDOyM6hjlBVayY>%4@nI+3+f13S!2#l{qgPMW0|ixX1UHAZvxie z*~Mwa1vSOX-A>BdhV6I3gjT8y|AVvl0LS|M|Aq^dmF$pp*?X(3a9Nj?>`ftCWM+4% zi;%r3#C6#tvq(tBWn~kh>{+rhGM@9JzQ5o1zMtd&Joj-QhvTTMKIi9rpYQWMUa!~t z+DYWI>kE2t68sk71UWr}^VwTyf|o4S;hqlZqo0emGBz9J{XB&OJoN-`Ka^gkvXo8u z-HHAh{WOnS%|le}wXHI2A{X-b1AJ#5A44KhN;ciy$z$r2U4GQKKlvfNct@TjyRaI0 zwx3w!r>MhsdB_gHM3XKQ@rV!7Z(?OY5_`T_bVj6amqOr4)3nf){$J;``BP3wDE1W(m5nBMF61caENZM9~Ig=gjk|ad^sNUKz>1gSt%Sy#3 zT{?}V{-A9Y2Xdv6G& zSs`nOl|D;x*aruGwdiH@Anr|Z1}!79ubZS7mI|;Ro$!WoY+$-c^t6Y<`_?k9=v|E= z$)P;z%A2V?VK+{80QHbHxc)HPVVzM+@$NYvEqIYyK$u$KldI6+yHtXH6{VNSb1WK`(0% zuM66;_McBYNy6q5^5R6WM$Fh8Ek@p}k)VKFX8x{nWUjcKTS-MkC2+3kXhAUY;8JC* zd#;0oM|G@&wte*Hz+0^6VqY0plWei{j`K25$KTAh3e7zRI2ab_PxBE*n^+b(-Tl zN7hAq4a1RI!&zv03c-8wO1RJyfZHFJ}nzh{{k_LOhl?P2SG zHo&$?p3z~b08MM^o$yPIh?b>ym*>YVwMG0_Pvs~7c*h@=gm#v5Uka2@|9>|Ch>R6< znp6F)uF&8@)fJ!+nBN`QQTGupVn74r(&*P(tzmKKR-HDnwF3Kxe55xXK8mUt7xJ{e zX>^g7{I&LtsDAO!ho7dDaHgta-SKw5Y*FM5pxxbAFfIijuI&D30q2LLCpFm{$a<+* z3YV{0H6FHeI>A=Hwg(CYUe9sZ2)|ig5b#`}pnSxsuwnVi#t@vRAufen=e-35CvTVG zWZ{%StBVg9#Cb0(+++SyP85(z8Fw1tKQSZGo|IEz z_$;{J!Ps`PMQhD)WkRm%`XyR$tE*E5)&(8=HEw-sEYP7dS7CH?E`=y ztUm1=)tUx*X>8{C!9kYZS48wCgndS4MG|)+z~BM8C$%5|u69q9=&#fM_)*7HzmNss ziO=W%Pv_`=-$nrBv7rqt(tI7;ps!-C!3Gq2b*H{9hl>T%#IEYEnwijVReUP-H|=HX zmmvz*3D0^Jsx19`y~_Ib%A@#dBGz4{i?r>8K;z=EV%vFCi1ycg8-6x#Zttkn4-u?) zHYSMdK^w}oME{Vy2k2)CF#`^m0-^%f7(q)BlEhO)89D6Tck8!vA?o>_waz3U8wj>V z9;-1Z_6)rHE@s3oe80*dhPc9yICNYqXbXEw&lEcwS0Z#HlqagfV~9W0XuSizTIvfm zG;Z_o+W?$bg^Gd^_F3Qk-DjrsIkgU&h;6njDz^0IltiSqBolJRX&B35esoYbeN@RO zC*eYk#eAFW0v8A8^mfgWYf9Elq#+o6I{Q2eEJDnub?9C}6*=YRd!YVlZ*|+)nDO{J zK#PGBW}`o1SVJN$UL__^v3%nKa#oPbT>WCh#AaJ0Gxzr@is2d2ds(2k@t6<2zvuA) zikfp7)Mdo22iuL)7o(3WQ5uss)W?QWGF*nwV0(8PKpC^6+ouKdC%SCuUH_;3(VR*E z%C%@-DPh==JYG%(3VvjACWZt->pV{_9xfKU22@h(_@SaV`&tP6H9(51Ih^&%sxw3>q8u|5 z1z;$4e3y9@PkwS^(jr+at*%>pMFnRVL{Otov$)$s^(S9?jT!^(f`QFoMx{+|AY{UK z{bjO-$!5_?94R{%UR0j>xx(42(Y>2#dGC&D3OM7`iN$(JdnGgs;231 zG)W9yVgQwfQc6qOAUhCF9z46H@RJI2#kH9jTjBKN`ki{8_k)LNHam$~NR3<)5tTG_ zGK%fm2Okq{g3B!>tYJQW%4sY8gyd)HxZL)QE;Xdhw;0;Q+5yNsXH%lzfs-LGd`Ei>G9$eV%U<&S;qmaflEA|M2oKp9YMsAm6bSTBt{F2}qs(=hcm-NE@ zwp50FmI4rdq5ePkgMKn*#$@43_ipA^{75`>R%gX7UmxSeIXKTE zu19-Y@X*DUVPiezDeHMF6D3h`PJa3{%mLfCJUPT+kagUd$92#&Q+^RAbRWYBMvCq$ zzU9(nvWlas$_yYOuHVa*8&5{^p*^Xtv2O{mtM2Me#tL@zP-nf}UfgMBJ)St-ZBo&M zB;Ovi_pDzIJTe}5U|@t;%}s6W5vzE%c|G?hjZccirq@&I%2~wd6M;1AocErsRy4Zk zitaSCgfO)Z%KQ8_$@`W7J;`%P>Ds$--t<(-$mcdYay3ahWJWH5fa_4Pi?3vJB6k_> z&O=^hf@r@KE!q0v%E;^6JPGA-NUfi?em9Q8S61*t@~o*$E{){mAePNkJlyfrbN?WgL>*!U+x+?ks@l6o(N)zl5lI7@zhYm=@f`VcCw zBYO(WqLh!QaL8PCsonE+^s`ny_8}#|eZ@<@USBGaGa&4+M!CN z#m0-j#e!i)(5kCotsI=9+`#b2P#Tn((H?6b@_7d*l^IQH#9&RXtx~%bL{Z0Q`cj5- z*n~+Y&y97YhNWG23wp%6A8jbrrZ;MV9@a@mHX6NN#&>c1;knIgbj3BqMHWm ze0X;fCNz@`K#PwiDD8b7E|XN=jWun`MrOIT+CjpOqX?}e+2yzfzy$;(u(DS%c)EgGaL2ij9(j!G{HX!Bs#U8j32EjI7{T=G;xV%&Zu5)soaLz9bZA6>ei-o(Pc)IG``Qgsyn zd>+OoWZGqTzkjAwn)EJz-s;y+AnPpBNbgTG0QxOk!|NPPlq75sisz&FzcIe+-*Gq3^N0vdl7~J*TS>wt?g^gba@sJU1 z%di02;|k0{nrgZ8XIchgP`!L^>Y(i)S5P0o`2aS~eb}eY`{ENoekriw(9_UOi1kuC zlth#;6%#wcw68}n@>Ab-cv@b>(PiccTksh?Ty}mz74+f4RYm<`O~Q;(^_@!cStmMQ zN{`U0SPaU^b`xvU}#vW{_BF z$Z||TY%-~_W0xUE(CK$&Snk8CLk|O@&eh5pJ&n!hJO%dk2cVH+2J{KlaWHN0|rAwPWm3lLH|MmvOx^*&+Q{2j$GZ`75FZW^^9byMh8dFF#5|e5>IS`aQ2G zpJ=?279|INudL}rMwix^_~>@77>@b|iZ8lam4>^x;4^~xPDJkqMlx>iiQwa=DkCiJ z*Avn2l!BB)(W1}Y=;)L?vF`vKBqV6RVAakwO=V0&_}HEz0>#sq)K~X?swv%}ER?XDd;=>uEkQy7yVK zeUvre?WqV=0aXk||6XKWHfL6j+5-HPeJqW_u!So?FN0zN_lopm7L5h-^7Q_U-cRIp zAG7k~Hu&_do+?=FaY6hUX|17uTv?D}@|;+vSlaQDhR@>vSsYplUmgDqjVub7xVm&nAA zQ=6*1VMx7p#}s#EBzDHJDjf~9EADl~-da~0SiZm3Za6{0-!`L`QK^gZoe=i`E$ds_ zcb5*1;Yl70eOHKO;7ar#0bYmHn?|scn>UJnLqAQTcIDhsyF%D!@o1({k;VO){HbEK zsFxc?*!JsDytbfY2+?F*Tds9V1nR5Vz-1R9lIXAhBu$5dYl0nc9>%0L6Q94lZR$uBNj>-(6qEnpW_gzY zE?EfK2pr2VRAFfvKt4{mmjMvqSyfpKaxJ3#Y#q-mUMr!M{jj8ZR9%7=?TN++cM&5? z8)gv}^EgC8Y9EcUuqAGVEiR*omFaxL_9rP;&c0tA^r6#TXR;ggL}@ra^#~AoV9}n@ z;9f{Ju><_fh{9kqQ6*SzGkW9c zc{4}cx{x1jFZG9ft^P2N-zH9V$^%FlaNPPnsf%xOB-6_%r?&DLo+tL=I#M2bS!z88 zXD}3M@OV|{9QUk%@cNr9r+>tJdYJ)Ue+!6x1sv~=`2ZQfN4DwC8O-N%H{vRcaycL- zbo!(-F#$*3W<$d-2E~8e?>Fo|Mp$3R0 zT%x_N8)o+}Er^qJK>z4@ymMH8?_(&yI{gB~iqol$pfTQB z$4x6{xXZ3SnGTCmSGXSr-L>j@I!K)909vQ=z)o;c@TP-uiIUrnr(&yVNMcA37mC0MQ!VLK3|uWTY{axLU& zX?J`<|8j<@vjLzDQhf(iIWcQ_4%6bQj39mtAA4*2u33yonnWNM2bTz0>` z-*5(jr35dVI?dA_Sa2BK(7XSzd?7Ti4N>PkGVhpYN>7N#i|QSqYGHUf5^WePP+^^rQ>cy z5L{pAF|R?%TY!0FjS?Of9`1%at{X_20$|)#-mNW_($Bo-N#TO5vRNhSpJ?2Y%E=hg zO8o(V8S6@v7PmR}m(-V`4z{GuWiI_}PkussVz(rI_5_e5A9r$D4^aJ8Q}>gFPAB*{ zCcMmq`+e!-D>zj_&I6Pv-=lLjpP7bdgr&O`RvZ=$wvnZDfucof<&sBKwyM# z80H>}$02O4Jy#O6X8QFI5Z(rr(Bj!^z=Ye!!2Ea>?VER|4bp6-2rZ%XNZ6T3=K8KR^0V^b@l}Jz@|_-=XM6{hMY${cx~JK zYy;0`_Gc;9Qm%WWpn{)D{VaeX4N<9Pf#Q7}659brnCD!51?ez#`sYeCC*ii$$5c416|q1bqXM-vBjyeMe`}TX%V%vtJbX8T57ka%0s?-t zY&PKVLtq+!`~3Km6_7C*uPkonoM#-Fd%5zQcMBL`@Kz@$F&vC-=}EKzfsf}U(0yd6 zHu!YrOFV#CNpJ%CR~-l?EM=sRH+y!H(pG>d{jJr4+!K3SNx*lFxjFj`s2X)^E-f7X zMS?yiNQ8J{jVN@V!iP|Rk%f;w2kna&g_3AicGHwV!_LBTJwL#nGeJeCZv^eVGfCuN z1EgS7ceoCY!&md2MWsGQ^8orR&%J>al`$rU$DX)@w2yv1wI*TsjoJB{p;c0Y;=8|= z1|+zMB&@Op&Ja0t^5-seqFTX--mj15qGbGk0l%E1Z$<#FI|GPG=(XKz;A@H<$M>!K z>s3;lAxi?X1+>4#hNmIHotiHIXUP!x1HcWO<3ew@P1!{5cNOSX%mSH`2=%m&%8(Wa zHX-O_ze6?%bfv2FV--`(+*O}%6!kRtkcTd7F9Kq1>ybEc5ozZ)#}*+gd@%Xe_RLVE zfz`~6`)Weh2^YW}8%JPxMFCZ3Fn;>V-{3x#4Sdg{5%wYdB`^@(VHx7P86;|*n*i8s zTI-}4BihZH{dBCed1#p**t$+tW>Y>M6Ra~8m}~$X#i5Y>W$+6|#gcf>xUjgSSiX5M zOi7w$?cDy05$LQ&fF(z2NQ!k(S1LrUnExY@6b^Q5{WXfsQ@CA?KYC7b3l~t$<`GiH z+59$~88sPF_ld@Py>~6Z7Hdc$0YJoXKm$aV)l34=kO^cE6i_G>*7%;m*vkNg;!i*U z2qiXQ{PX3($7}dc&A|Aow=oiDc`Ef49P~;K?tey zPOp;RCI#TYep?ZZZ(jWM1A1<5Bq;Wl^SJTNCo(ck0|EdN207RBr832UZIoYXru%mn z>;=F}{9%lRgxWoz$xZM%ArsJfHCWP z;}AfDTi_y!xI;kvuVpSkBkPmKN)qvk{P@UFij;lni!;TMzn4sUqdNdFHu^m}oJ(8b z6&(V~P=UkRD#rsooA2wn5w#gW(mw^WD(zD~!~S)a2F6sDk;*ohf0ygBjNXAMUiG9>if!)(5#T*X zIa3>gX+Yo>M1Tn)5PA-(%z_DZU<+*4Mxdoa?mz$9G=%t3n;UV9!y@TjhNy9dZZ&u1mN zZjT>b1#1e0yc}h1Z3(DsCnvkhK-U~}*LH$~0)_Pdap zKMx4s1XEBTFK6KY_WGHn;@|-66;BGO8N%)ObTPvfIHDU?Ffhyk`_oSl&1b`fxHZH} zJNRT2YBMi#(^w2l4W+ZlKn`!bvK=?(Qs*XbYF01vej|`f#Q>(P#+-Ot2v$xnOukf`?23 z5SzfsA_GS|cscd6NZ%jcy32P8lF#??|pH16K(Ua2<(I}qI| z*x(Zg^nf4>bMt)#Gg&uS91vEY_En*d?~dt#N4woLUO{A$w#s}zkk!A zH16Wzy}kRb5*c|(05%H<%`ChSKjHaDu%=4*$CqP~ErfFb;Jlj(F5D95B06FmQ$L)?M92jE(Qp{kkP7p-7*Pw|QBzb7H@_^+5~KGy-N&0MZN=GJ7@34|JMNT8UfP)w0}k^ zi$1)I9DrWlo?Qa?P#B=mue-M2`(vWUX?wBPlM|@?Ekk3sjK+|~vs4<{>EMitGySRf zCzF1od!l|pfab09w^^h*z5pBkQ!t229ds*O;qB_M-!b5cHrD>9CjyBD{pB*u-`7tJ z!a`sJpRGIA(wdv`Y-?g+EWYd%Q;$agJ9W%r^7TF3mk|()7fd34lsnX1vq|{BME7#) z_{_A1WoDfioU6oU*WEVd03QJR-hYoAU-hB@OawG|RG1#hx3u~f`@g^He)Ak;n}%l- z!S3QHKzuiHGbw}OGuxDkzBo2`c2L0y1|9>D_SY-sb~Wza#VJIwX%e&Q?raPUbSTC! znFlik?Za3qhvAJin-#$0JP}ZoB(H-_R69@{M}X0(X0Pvl0CR+PL0q#HnHP(6TBOoo z{&Ruo(V!uhKNW;|{mfGHax1lX`Zf>99l9Q;-GEq*7Tjak{x zG>dlPOS#~b&#W%iKxgnS{Q8>zmQaq^kc|rSSGCMFewYJ2#D)d%6D9i2V10`)&Fdz#MsH6L? zk!aocqyZQuLr~h+&Hv_$-&*OLDer4pac|!{*MA%urZ93}DE|{aNd<3@Bz7f<>PmJv zoLzWGI{YujsgH=m9Zrr8eEKv12(}MIK>l9?_fR48EXY6h12_#I;kJUnZm7eo7iOQ@ zVE}8e@!J`bQx{@84)uas$Nx$Qf;^~`0k;|G7CMtQfsJm}v8dbO#Sseh zXP3~$C2jWGTEU2@-G}^?wLij@Z?efmyl7oE57-?sBnV?8+^R?mkM7{K%>(y;{GOZ) zfZANM%k^r{zfFp5OJf%x$qg!=4ylV$2**I86};sQmEe*GnqDqn8(((g3Xs3m`xf#_j5Iz4p2N%=3f40}Y+mFS z29|UYoCXQ53WEip2CMH;SNZoM>4)-Mkm<^ZnU_YBBlyu5?`!g-71=gy4|kZ<_}Dgy z2*ZZomMBVo0XQY%)`(ymlcpe<6Jh<>eHD%)Yc2%0%5KVFON%|J@LpOURFnn?$E!0f1AlE${Bz+)E-;U?%ia$05^74HxFiJ%4ii`y-qh z7iN9az!Y*#=yBu!fJ&1|w-?Epo2C6`j><|LH{B23EpC5>JYK7D9ZAz-OZ%0}FCyokGvhk65OA zKPYk?3NK`T?J^z@48x%$J#A3pOq(4rU^C=erFpjEzNv7`=N?(pUEsbzYhB?!-rR

    QNdyk8?oUiGjh}WAzNLduzWc4^ zN%s{F0i{Yggn(fsQR=-iBuM*n&m0&XP|B&Cxh(c(bsYXFDr@X-J^4JvJ}4NV`O(1y z9Ct*KB}{@uVd%A(@LSC~M0KZWoGTJ+5%P1DJS^lyzF?dlwE{Eii8d^DzeA@l)!VDc z|LnzEFM!D-8V=GT|1u$7Bx&>^+q!UGY|JJ#!UAtxUEYz;fRBx^GD{9|xg*^e`%3Ty zbdA>gS)2tZp^!cSam{Fn?w@OP@v)oIKi4SNncXWe+_dP#3*`h=nw77-p$gR2u&@5! zxYY3{3Q^r^_6kQ4Opm@HGal4C%*}AanYCkAu*EY>>Zn{laaH;sV>_oq>UWn`!=O8A zGdOehk>`I;>>b?mFLF70ta+G3PiZ!Q!-NGg*xRtI=Yz}|q&^bNX4-#q4;mo#2vo4{T#7anLCz4^nER3ur5R)K!=MpG zS%DGzC>^pofHlZ)ZIocMX}}Sov}rgXJH!kWGcRS zOSIf*uf_`f8F#H2ZxtrjY^`D6k?Q?PgDwO}!iruf9FwQ+ucXQJ?wW0mj5J5G@w)k9#*I$3PhcCWL#8n5h(PYAm9AP>C9Rh zf!}FeRI4DGMP!<+Q^@HGAWed51~`wiO0i@(v>U;#vRipPls42$se~OLWzw^bgWu{7 zooWdp&7D&0I;K@RPL(3&HvenUax!==5?#`v0q6|M1EW=@I=#F27v7Upjv1Bsv4Y_| zRgu(mkurT>8 zgMlC5R%y5Zq-mbYtU+dDF{?mq;YHo|yL&s)^+fgr~73L{p+EuW^}4COf_7bHgI5M_`dzZ_yu zH-t{r-vV!m{sG7bK+0Y8q$TN}k``1bpMaAD^gp^lkt&Kb=C=Vc0{{G_KG&QjA*wjK zmtH{s{ms`PsgpP^gz~+peHF-4h-~o?ir_F(70TNc`bmvs(VC~P80P24Hm|K2Q1RPz z&2;|v{WjARC<l>v95>~B#kf5|0cTB&2{xG zKhcCLkLXbvUvt0+y-y^u73zWf=fZ&w-i~3vPsJl!N4=r9A{_lLILFVhIVkC$Ekj^P z)ve<&jlxLulZQZz5KyASrg3N$Ugt2+^~SWyD$iR#b!aV&G3D{{V*K&xXg&B7!Jo&@ zh$$B^!dmN)X#LBk7ZKk)UVZ)=EIPXBv7Oqba?ml$Rr2t-8Py6Y%ky9@tSdwe6uj$N z#_G&6p@W!0=XWM8LIcTpG-D^K;UEDL?KANl5Qn^Cwm#CI1!6qH4|3tR291j{Z+qEF z-oqjNyxq#0hB^WSJ{^&3|27!T$6D(iEZ>AVs;xDM8;dp#sZOy}@CN+&?$EbEsfi); zv(SMgGsD#3LtbJz3&bp^B_^p=X9tpBTI`RQA5>@rbrH7r?DqH_a1yuvn;Mm>YT;x6G;S{9KI77&R0f*LG1h}yBn_%<)rL=Lchvh|Wl&&tDoiyxov zkbmBr&mFG_Luovemid==mOy?CshB@?i{4cIY^?!fqAg`O;5x;6!MU(MW6YFV-!ZFO zUh4F?<0r!N?oOIDGn=6nxE7Q?kI9Ks{~+ zW;|)_ceL3-YZrg1?zgjn-L;Trz!W`y0zzQoiZVmmbh*){{283uud7mvKY)q}Rdf&|R zupY!?r~otHVVS{-y9@!Gy&pA*4U2z=v6^YsT(7h6%ZFZvKyL9 zLrL#nvD9*KYya<+CG-I26VYQM-0^fCISr;c+kxX7r9Ol zFWTc7{xhk~1j^;jsYNu#K{<3)V(Yf-@y=%!tAb4a+ipOxI3Xk}90X(J80vWJ1x1@x zmK75l6F^-DLwCU=a$z5lp()v#JULJ@p#iPZMNMMg2layUD0J2Do(^eIf~hJ`Zp4cG zH?WU7S1bJTneTIgZJGp&^PN<1=~_xyEL1)C0L1#sxv+g@a)d(&fdnWJP}Mho23@Q> zQ0Kx@r?dw`xPg3n>zf zFW%}rI(@Sz@X3$TE_0D-wkDtTx|KCc`_OwS3Sr@QWYZ5uhsfrl7$;alcjPKE_ z-i_NY%csjl)(Y>?-@ku&;AXz~>w`*faN^1GY_8xGN4Z0;s7H)$ z!l{bNw<%7>vclNlbe%_jd&%vjq37_J&mkY-`0I6;pJr^?N0#T!=JLO~Gr%;*j*y{k z+jr*-!efO&-(*ZGhk9|L?ZYCOb=@7G*|X z7`>2ceIw0aXG$jB4u{o2HtE|qO~LLi`Y7pTEnUjK_31&H7E)*uJg6X!(ei?@fMtE$ zc`^5-e&0TZ@D~M#e*{7nuaN57@sieYZ%DODHjU+j$vNsx=!g7Zj|s<#Mqk;`LC+sH z_;nZ;=g%*@a$XPnTn}*>34}Czmegyb@K^ZUYpWSF=N7UC@PzB^2eD$(uL!wgqViHz z5o&$rA|-`(?w9oAJH^q5hAh{7yy^*)C<-_2*>(8MMJkXB?_#Muh2MEm=?K4*M%HY= z)lho*8AKI1N@}PEU|QltI^^gd&cFB(85bPL(YErA$NCA(E{nN|Dtjq;Cp|LJ8YfJG z$h}Oz>KynFXzlceIEA%hp+aXbyp_VOcK|0(dbON8=Fn+2_vG;7#|Acpj4s$MYsP7= z3t(BzEMQGJVesb*a1@^(>A+2VxpUB`ad>Pjc+hAHite2o42^?85M+^#+rH6p?d11H zFzASOs%COjy&g`h>bo_m%TwGJuqX;7mb<_|od~KW7hqxZ?m(e|cRE7XXcExJQx-W? z5=O&ifZhXtn#j?_SYw_NVNH4oW^B1~2k_nFtej8s@`_%9zOspzfrKp3FG!Y)&V@+j z4kd3v57}_H4s-#BX8B5X9rQ1&+Xl<-)%BT!(h524TDU$Es7LE_NgspyQFUsmiKF;1 zU1m@+MjqwK`~j&9orNEdf>H<_GTu@2egFy_SJY5Dee#W4UA(o_#eE@(&ELt@{f4@;Y=#AnE-11(I@r&p`QkmSr|89D$ZL`=17jFEL zW|%c=|&Nz9m~Bse`Y@CkhgIPVtor^$w@W&66Ti}cwH6>a--c} zm6FFwa_2{IjdGGhv)5es7=5%oYK+K)re2A6( zGTHP$Z^g@w135sxfA`qciDy0wBbNE~Jx2H`v|Ksn=W<2Y{XXZ*m^25M&i4@)o)mNt zj4@Fa$UfcKWTu?!{+_!yPXN9#JLw#c4R1O*Nvn~79+teF&;>f9;8#>r$f0`FQ4K&V zL4z-0-Tpn#RW-@64M=ckB`*Vl%zY%0>hXFrQf&B}^s{TzG2cnQ!j+NM3Sbi(tHtHN z`ZU$PBr++xPdmK3(HWg=BuYs25cKekZ;?MZip5Evgn{NKM78c3orTocT=O`j z{j8|ao$+12fM8JJFe%kw_+_>`yXQv@Y^Dxc{B&W*vL;rw4Ox0%n<_uQJ(4 z#9i7??lfb-|8@1iJ}0fbs0h!3Da*eA5grPGxl+oK4Ec+*PImG$V{nQ>ixYIBDLp_; zVx-nvqoG@*&7sKi-fk^HLF?uWUd^W=US8Ki_XM$(Fg}k(V0yk6evz|e=plA1Ya^tz z-yZ?-ibH^GsSc0LgHI?TRAb5CiQf^w6?l@n60?#RHT-a5ahfei8n0G%`;nKx{Dz0w~PU_b93Xq?Y)50^;HxtQTLUtMW-hG{qfBJ4rwjh+?V&Fz{ zeynp_@9f3dwf2+$@p5t$=NI85cTkD+9Ax}cgX7VXMFfF5TEkiH$@?13I}~BlBVDbn zHPBeMh#iapyK$H1d?)38uWJzN`P&)6#oBREJm?n^w=mkYd1+jHZyJ&cn!slk^A5Nh zn@bYqV|E_1!MoKWyLfaWT_(z&Drs3G0R|B+F1wg&jTYusF8*Yq_95;~QT)y0S`HqY z3!mh?_a2>>Eh@(r)y4jhsDA8!6-p;d$DMf4w+Oij207qrA;XG@-FLP1glEpme6UkB zWQ#!hx1tj2X!)D0x@l?Qmx+I_KT?fsmYaF+PC$jCYx{|}Y|u0U{{9vb@c;fUIBC!u zEFhOdm5_5g%C@_|f>Vpz+Y|q}tFx8QQMsq`pu_E>f-wlS`&F45OV>vay$xGIAp0O~ z-w|_iLHPpugG0K}TsZZ30eqEvcq_4W{85?UMN>mq)86|HOuoIUem9Rkt_(8KK&ptC zs?bZ^#V{D?irN>JakTmQU9mSQ#tgGmFs5Cseo)Ge=&qqDOmk3c*=BjYOj6Pd)`QAl zvrufqcG&Zv+w+HAG~JT%j89q6rENvwvDcx?F*2rbPRz@ioRJ%8msL_6%T1^Y8~S_q z3#s6gT{pmqin$!V`quFEMn@*N8tEx5qmS%F*3FjX^oT1aSn1!O;vb<0{j;B8EyNPopFbYb1!|S z3tLy}2yYa{0?1H}$8K7in%|^XRCyWPab$cak?D9@#o}^lzRB11f;@`}^NEq29;1fL{2xHLI~V8Bd|youOWA$y<8EE74s6QYXp7J`lFs3g1LuOP2fNlpl;%0nA+H+7@y6V9 zK#8g22Sa?dorAO=03(Gl`V|oQGTu?+cUkSEwAX_{8lzHe2ELj@rbbB>cwWGxM(c;K z73CP_k^j`utJhl67~;|~L95k5K<3BTrHNlpG$ZE!UB`7+PYZK9FJ!&D2{bc_FW|0M=J%cZZb{20T`d_GV1pN zV|17zhmB8+HF&jgilgg^C-zG<*xh!isVZ^2-QFb>#x5cfiNAZm)_Bl?WP_Zb@KTbF zzoN^mtM%XwXF_f+^Up8!6K2@J!O%4U^nZ3sf%Q%t-8COprXb+9;`np;pe-bVmJ|X7 z5ym^rIS>B?5g=SMtTKSr?4t`^T>x4j_{3NJH4sjWo+(4=)XWxR61i5mujwNsrlkmz z{5zI-M)eVW<2aza^X@*o@oRVc7ISWu!?*K*=<0NkO2^CHVAQ$u8s3d9~MK_b)+5D&QR#84f9p zI21^}`{PGsp4rp?@gwJ`L|mzK=w1{$90Dd@-{_?sFhl79!6(_%0EQ81l|XCl^eITl zOtqwM$=3iBcN!U7^kCwFJ-TX}h4XWkQF`g@`q<0Yt{WMEpxc?9>HF$KLtTcNLaK?8 z9u<8HmO{fmtvaAe$)BY_BB2;ZuJ2;F!doTirMGLCnS73Fkd{R3jxOw~ML=F}ioFNS z9QCE%t2mHy3ltOw-AjN*V_CBG$hU#3uwxI9TR$FYWN&$GM(3TARHkwrcE6U=187fa!ec)xS<$iK1Gpsgpy zoRVhqxU(eL(%OgLxCXA7x>I1nF`wSLFKvQ*^M#UH%_iak+QFouQLvC*;5ZhKIV-sU zPrWEsaueVa-ju8k!Nh}yNT%FAURGP-uQ3sfcIP5 zdKgKVW<_uHRu#45XJGcSAeM@Gz3kaJp81HM4@JP9AU}}(Tndx8_4q!Z@^Hfzvs+B|xWZmtEy0`%CRA zR(=X9^sL?wfD>~RJ}j;T<9i8Ii=jyryJMCc3(yQXgO4hVzO58YKtmCPHYwEkN&SVd zJ9r-xt^~^T%jFZUcwJZFCK_?eG_;eod~#Xq7(c4ed%sE|)&zIoU8a(=lz>RQtK z2kQxJW#|F(iK09y?}a;ohF%6%uu`UU7Uf9HaDns=XgWHff-kpli@f;GE!glgq==J9 zWYQVeeW*&2p~PyZGVg17Q3vj7tSgzHA6c=gK$yfEGAj2^4E?T~XP4Jk3pSrK`+VHo zK=3);eY;G_u}-}UC+DO$(M-A3Z!%$U>7^At!U=ZI+^)$5RksLNR8 zILUGS-PInioXKczk%2x$$%7SEKhs;bQad$!wRqhTDLrGMN}H7ohp$l&6uY}bHA}8) zO?Bz1(MkAzcIlYUCuUoSuT>x#TV@Yru3^v#go#QAa_Ck8&cNl4GV*HUJFR&%mT<_^Q+kqqh>BNEqAM@r8z+*q063u23`aM{ZtfToB%vutg8}ttRfnL2UyhUvK@<0k7 zr`r$)8qOt=CRX(qtCX{q3xghIXE9-$V6L^nlsa@zuSd}ydw!@LW|w+as~dhm{^(xd zp;NN?Ow$x+sdmrrMUuINmuce=KxpF%+2K`_YR`QFFdVWioPxmIlQ@?c3lO7Z) zWJ+htyQ@|C+TIs$%7k-9=pJ3LSo~g4i!S0K5=PPN?YZ}<(k2B#MLsz&^^id&MGyA- zH3P$L=W>4~bOMO}#}kkZCs9GY=roF5&oPeJnpmm9NGYg_rgy!%f4gR|J5MviVD`G6 zvB1ruJE!S3J{MM1*e@1x9GHugH>_jnP^67= zmQKM+fZ*_j$ED|Q_e=B;Q^a1mRhn;9zN>L>GsPlk248U>ifEvQ=_O6${3FyxwKfm< z4pn;?d8BN%Yy@|tlEHk8&o4R;i?e!%wJb$0Qt7NwOB(UlW^<~R=Kh%b4sbzCDKR}~X@|P3kpo~T%jpD34 zcOE}ct1(9opdlQwduX zp@hmlP8_{v6H|f|a%dOQTf9e&?aToMx1QI)VBTOB6c9my zU<%whl^KIu^NpjIsCRQI3*8}&p!+yMtCG6LU6xG0WagfWipgrHp~bb0wjU;fiC6D! zHn+U&4b>f;3`Y$1fef&re9?Ch`x!{QX`KA7TDt=ZThHPK-oy{li&bs>0wW9OcDsez6}K_>)@T<|bli74qZ8xqwA z;czMLH;P|F#8f}Zk~oupl$1+kf$#Zx{KKcsSjp+Kqip&`Fabn~A5rEWd^nV)HkIKf z{PclV<<=XjiJE4AljIgSTT#*H3s=puO&kWEsNs~p;*W1S=ym0@z?HkKZKXt*H!#XI zQWh1_wXzOpW<9eh<|!WZr#A2E^R3=b$D?WKZWU%7dN){@+^6r8*)vO{lHq)Gb*=ca zh`k|ru7koTkZOdgoueYBa*wDu041JO!#lwX zUWp$K8kdJ|OZ8s0849E0+;P0Eeb}WXk1~_^m`#5N_C!usBT9~p2W_&jeZmOPl&7*AXzXdW3Z6Wi)v7Zrw^(U^;JH8@LPbjh}q{QR3F@pIdJ58PygH^tLx2s&xkEO zvUm&+1~D^u`HPk_u|^KOo+{R4DGkCsbv5bacxwSIPYL3r^`{iK3z(AL^t<=xl!K7N zMh3Z+ST&&jkV;3%kenA|mY6w+qdwXd-`Ql+srjy}A0wmAv0j$r=f|Eg&zlBqjuRr%W1Es|Xar zdDv?|KE#}N4*MGQJ#Q-Ed+Mv1p^U~xyq5-(Eq){wKKg~R)T|U)ziKRj+tQ~iXX-Ta zGw-IBoe@n6%0N@-mj&VP)b#~^5*a8@N(>(@(&wh03%*Z*IhZI@?5rkHAOg(0gQL!A^G zk*iabxrG6aiFhhx+Y#I)y`y-sKRD=;pOBZB)To^(r&yw}h3SMV_qn5P`m42-8&|bc zXzB2>8|*gN*C2*u0?cvR)+9Pa60?Sgbgp5pjKq*72=QkHh+D;Bc6&Fc6t$1)S1+n#0 zoxvDQr>AuISG&=;^&d5dXaXxA2R|-XKNHZ6bpjhH-WR0Kop%H zL;obG-gojWA$;D8(i5q}3na`>mfEh9ATK@o;&&J9lAZCZ^iQr5sZH8(D-%w)wLU2b z7HqzYwI~=R^D2^D0J86Iem$6~A8~hkrCFMg^|fmE>o2b9G2e{W2A01k#)wm>@9z?8 zPUtoe^>2Bb+?Q4k7uOm}yd`1suJ_G5o3+krw--z!!`j79o|jPNFX6ey*}hzM_%Lf8 z{;P<`qIWD8<@Z6TOT*NPDS28BWM*-4&jsIP^sw=uzaGD4pi2Erf;$LU>91z|fmUm8 z$Ubf;kD$K|?^+RWDZhwhvNI@UP_v5KF}Q4=H0P?2(Q+$wH5+$zW)*6ns)=_=G50^a zum`-CGDRmp?hiSaT*{YiQOs`C{h6$f))J@aPFqzaHb6R|~W6XNB8 z?vcc0@lRC%Iy3>4^h}y*3;-?AWhCl5Rf1u8AWmsPzjO+1j%sM+s&TYvp?@S zv-g~tGrxQLABCp7-?gghsa31iQ?&-#C@+`Omqt_9lJ_Nu;yzDI5cbpbI(NB2D{Nrn zy1vN5^;B*2H2?3B<+XT5X%D|YF}=Mzo78c)viy|iER9d)@YTl$IpM|kwC)$)OMEw* z(8rnz&(EEWIr4d~HM*CrbgY>&>pp2*qwQ+o>17+{`v}|lv4N-_EK4UR>;)(!#G#Ug z{1l-R85Ebbl8k?`*(;^%47hjiLz>^F30p}OhBlfl?N-IL#+Z-9H@httFk6z;qx#H^+E++PBV=);n zIn;`|W*+CgZjFk>iWTZ#BpXE@%1 z8cuf@!l^t{{ib~pnUZ0A2sss-5z97Km=H5skNS5hpd2 z7_N|_aCXdmGu-D@=^?3?Im>zuhmG^9>I&`I4PX=89OM==Z|M8_*x3p#@$JEC7h%LPs z%Hpbit#3F6vJW6uEIO;V_onLh>!LJiRoaglnX~A=a95=_TNY~3{V3;YNq5q1AByr-@u!+$E`>87g1>xV=z17$ z?^)yQ8!A6$v4Z!wBNoITwT~RK`ZvA90DI1|2VUA9f#W#c;!qLiA%KGCHisycz#F}-$wYu$Ec zt$0Qr#7f9g+%>fGmPJ|vKs7vna?g>8|YepCv_@~%z`7*JKInj>BttjOUriO^dasU4Jv&MCwuc6rXIBuuECvaWS03rXse@ADdyT7G zXsj;$WNX86r5ifYZ=mmQ;Vb}B5Q%3nC$;4seo*pz8j zbxTCDk0H@T8GKJHz5jd$Ai$TXv*;0RSkd%F;4N^xF=jh5$l<8>^M0(y@j>Q?Xa#4! zowt9a%ZljJ-i+?b=BJ;E9*u(=!!M`#TN<&$84ceufNORA_4706pu69E2K1DKYT~Ft zhWYNqCtS51-+pHSR>)aAG7DarANTDZTFZKVQ8DB;nisgoWIi&+E_icTY9}{g?@Np| zoMO>W8GTgq(D38*R)wXnI{yn=qNE?dVW@?v4U22Cc&ZrMR=g^heR7nT(;PL#kkw}s ziUUH#7w7i!uRz<&<$~k(lZB3#0HaP&NmmJ+nq3pr#>deGLevj0WspzD0lt9l&~&P4 zcYBAX5@zDm9h5tjExX!ye6%&{5f|TWz(SSQNmGuO*G8{1KcKyTC4s}+Sh0e?xtps7 zj^{{!E0CxASyF0vmn#K@cHxZKrVVAx_8@h{YipLm#37l)Anm8m-a6iMN?F25X~`J; z#_6@I)o(+YBv|;Q6x!7t#0Sy2fFs>k4>Q%>)i{Dck=Ik1fyJ-%?hGjd-+jF!z*X=fq0DtI z9Q4op=APW{1QMgsQ?j)sz5tRnckP$Z3&PnmgB6Jc!Ft~t^+dfAI!~E#@sBh;@ep#% zrK+JG7rl5~Gj4QSczQBh zmR)ZDS=x>0@;J<%i1^Q0J8$x1QcUc-YV)hW?G5!pZwxppvmENpj%)?D6&8CF4a^<6 zwrubWA9~jcYS*Wi*p9-EC0?|_Q`Ii1TI83v2w%F^Y>eBm%exZ!%<)urPQJw8_mnzV ztge-MORo1JICZ)<>$@-ZI7`;4yr{&(zIvB~=L|0bSBaf`MAYZUo z{kgUV^4PcZ7uYD?F8cPQ<&EjKgPi{;duAe@g#D=s_OL=Y-D1`I1tTx~OxgE@fmhx@ ziB0|&JZvY-8(t}JHkERz=?3hZ2pZd2jOTzzbZOnAR1pI#KAumU*k1^%Gxe%fgZ=ve z>zj@sAJ7ajD?I?c3~UJ1$cJVk`(poJWdsMX4io!73|wZW)TkUt$?iyv_{dVE ziB{QH_m%F|E^FdTA~8y4{Zq@E6-3R<4B4NKpnmFe1gr12rc^m9%2`ShuV`N~ILv+b zO)thX#>HMdUBk_%QY+qENJK|ALsts7le^1F$e_SxddfC0rt=Mzt=`ebS0^P0olISZ zf}*{5c%JUAW?|k8C6uxQ)%95UWo}nw8L1F{n5J0P`jWQh9!11NN-3RJ1!UO`7i*0! zFS(tiNsD$b5ZU4F)g~4`y}1qAQOcK~&lr=4w{TRTBJCMHkf!3^{)QR7IIizTzj71^ zEu6h>C!K<~)ZEF#?_pXhep&imRzLsos?+Ve+ySha*qD!76F6PdyCNfo$8yAHs~Znp zQxXUGW4HOs#(>9Pi>J>!G?i2kslkdyXavIX6Pwu7mrOos3Os ze4h`L>0)&0!F+1#?ECL<1^{k|-w_=<+-+>3CXvUF<*R^$@OM`BoGyf-i%HA;%Pq*J0WYd!)FC5mFDF=xR@B zy(q?Le$0lEvdw@~+McP_yZX7!8I%3>=%8hMyOftpfGX7YT0kQ+I4@?QI)ZanBWO>j zNF}9q=AmS}#*q~1PU`?^s zNQSj_>LQnR?e%E|jb-$nmtG>7&tGgB-_g*axs0s?^NDbp@2U3DWL`V`j4kDHKjW&F z;Y-ae9qhNc)CE;}F&4X>1{X+j`neONb!a%e4uNUs`-3Y&{PeieM;W$Cq@D82O;QHQ zIgD&!qHS@fvo(R#S*mnX*xE`y+^IMS9D=LQsNw4wcD^z&G}4PI#}9dp)3W1P_e3y1 zW`xK^(do>b!zDZC9h)%Q?ZD2)O07(@lyt0?&cqilxZVKj(^LBLHZnS=1XI?4?DDkP z`H!Fh?%Tb)wtFv`6SH#oC%6kGb7P1ier><{VFTY_g0^x~?6P45=UmQUh1ilCKEccD z3IHWiWY)>}H7*;^VnXq1Hq845VQr6F+BQXw=|)pT)QB!;ePrAFGy|9bECOTKRm|^S zz<}=A(caXS*Xv8ZSgBm=h$(bkqn}L3sGNyKrSnXILc2+oi`HIMX439VrM8}42q@In zRnHAL%~6!otgSm-WRfE^wdgjY()naLB1gZvAo#g5xcCn#kA1s4E4d`82i#_2AAQT{ zd^qFJKL`7PqeUb6DgPv)$J=s|F9uXi{klC%BK^b56U)2&?Rqt878SAS-kP?{-1wlv zK?TLE24fn}h>@q_)WV2;lvT8*lZYGesMD**epR<=*=Ia)bHG0zm@gxC2tXP4ThEU3 zRnTWhk|gti{nlAbI(=WrCwm&!Uv)iM-(Iq4rsLHt9vhE*BVTA-X?$%a#W%?MWm89a zpBsLA_k(G>5i#>ds5CJa(N0>wv!B7u{vWRIB*2}v)t@}OK%!x*A4A=1$j07mu6AUr z%VtitO6Gh6L=hvWk?2s@eE^-O^0EO`uEHzBRqn4mU{V(&NhSy55etcC6NWb7t(z2JHVx#3Pd-=QjQjMDIa7qTbuxG=(nWi{ua-Zv`cn)?M^tiu;xt3JhR>JY} zo(8*}GG11~=X1ek!;;J%fpTbP`=2IYASIkk6@sDH4(=CZGFg_`{BXlDW`!Nz&ul++j0h!@6hKRX-O%m17Mn7T4&nuAi)wptlJP*C@ffPEJ9*a`iZnyo6S_3$s zNg*ObhPtXG57uY7wZhlljzoFPoWTh(1R5-R9jY1$aUb4HsGJZ4LM$<6^I|SB?Aq$p z0L?HQ&3`uw3;6c@MjwTdMc|fR2PLTH=3uv38_5&}RK=Zd->7~biVQmMjvp6IpUFGf zw_G)_8Dw_nD1RaA+dHiZDJ2nSACa*y)rJd4EZ%OyIEzFOf25!24nLiE<0_vB*4fw)CdGysn#Jmw-adq5> z$HWj2M1aTWZ~3OIs{|-Xr)u%P&R+ya8Bs5uI$i)G2+@SZ)XSoUAU^|wKpSEo{(FwC z>=X8xC^`nZtzA1%XpGSHphz4gKgDU2i^XX**K9yf@`;e4xh)~k&v#|jb!%}D1h$8Sb=Bas_tWcJb)bJ}(0ZhW0~0(i zX73{O4-F^#jQMt;FAJ1h!)5M693wP&Li2yDbG{E!y&>eMD9+P*o<{ho_G`hlC%h*8 ztLZiro^4UYQR6!!AHc%ZeD9V_81nMiaj!nWh?>Ls_VcY%LK)I%1m(e zyMh$jU1be`hpi3U(o*UP^QPDUM|gn4a8w$&{SzE!FXrHl`35Tce!Ds+Tl^^{AqkV+ z#__TWf=bq%284CvXa#;67*H-S49bZKXk_K{GFPTgtdco^O1nOp9VeSDxq&f5X8b2r z>g4sI>kKQ~-@SCLas{vylq|oc9e=!}crYtv;d`_PN_k5xjyG3VKpf`>^2{PGBWs}M z8oE&lFbA4J_*(&1^}_^NcLfmCbwWN?OG*81Avn18f`R(_Km*u~DQaUqB?3$=U%-H~ zp+!ov5En?&-pqLeGOS)@;Qh!oR%V9(6iyzxit zuv_DoK?~FEy7A(1&!xqlYSSN`J;Sd9`Uj6a`}K~?=Q@KLt`9e=iyZ1nzLIeP`UON~Mw`5U zkV^n4PZji1`5I=YrQF{8ke!XDPYCX& zG#vvDgAGz{W~mF53KFYoKXBDnaCsSSd=Q3)WhZa>p|a9>Bhyv#l(xYO?v4no1#*}V zZ2U_5o@ZfVO-!wrL;s`Zs?Em))TeQlp-52_L6Z(KA_ptL``V!yXhBGi2l!)6{Y2af zovnRp!`-@%JTvenzj%uqu6w%YCO<`Awp=Y<-dSyJShedo3x#%COh0>6ACKVU&bvyN z9coDO^cc({6G$A5;ew>Dx6Z`9J2n%Cyt7ZlDK~Q1)>}-oohsUbKq^2=1x;-!86c*qNd5P_wN z&SOLB3>1+3U|_2NfQ9DNE1xbwq3xC8;@MyPIyNK|chNMuO6OrHE z0SP8a6^ys0rCfyp64*5-t?-yyt#n!LanFZeuyot`Hu+ue0u_;fl-uKT<18w`H7IBR&oB=|q(UR< zuLwf15qOJcyQz|ojl5#1(bYIc^|0k+q9?^&#%ZLEY2kug+8P!tV=9DYJuRe1!085( zjCehFEnj+X9%LFcP?=|e4}Zz1en)NCx{^m$`0Cw(=WkZY5>2r6(J>-HjCB?d@dRFH zXfZV!PC0;d<_$Q@%PsNyZxHnr`=S{~h&Q2yxM9pqe!nVElG)`ymZya43Jicw8!yc6 zib~i(H`}o^96xpet~AV)73}abVxi>YG>BDtvC$iQmuveiOzsSBDVo8J<&B)#Kx(natazhqPfWx`8!V>yZQ~rP9I0Crg73~u*VK+uh zmAG0B4PaY7?L@G?N+pxb!5JlG*qG_r1)Nt8q2()=Wd4=ML5!L6UV*B{Q*Jrel;d1{ z9h0Y6Kt08>iyOoGv5Y>Exd*}M1!y7i08Rv_5O$L+gno-M#itnI6D*6Nur#9ado#jy zPaP`#CYfEb(z9wmisVcdx&p`Ya&hA|?ib?3?iD{v^4(O9)fQLVi20e(<4v;f+=&we zLR3i{_;89@pfB@c+(6RRkO85kmyG9_l~`n}ChWr9nStR(+`~kwhB;)mVM5J_AO~7* z9{@_^S>;Qw3lVYgWZ4-UWL%8Htoa)6$<_O6jYQC;Zfq1)$$!hZE^K1b@qGkXd$elP^$!HcccJ%$AB7osP~E|9}+RdCqaJi;?lkm|#O3`35IWV(w^ zJ2DR+ESea=UgU3hA|kcl)p*H{vcom*KfCf^bI{0D8}1lerBKj9L@v2pq6|xH%+r^p z24Q8C^lZ6`ZCk`n@?x~)DNiTdxbT?dx9mW;?4&;-p-Wq4L2D>S5%V`_>V}ZqZNGGK zU56gm?P{uNfiTqIrVX4k*|PcT6TqP^k7Vebt#lN(u~mQvvhWWwad^Tx;_ALx1|G0d zT*L+6!@;_?KA3pd8+|O>0>`0$*70dikll*V9Iqs95XL3xo6TFT{k`W2u@ zltIHNbA2HQhO`NQ5J`{m$ABvfoMuPsx13t3U1cO}u8JUeavOTO>!YE|#%;wV3fbfI z%6E@}?KR-Q4W2dM=3`Iq(s|m_R(q7oF?QR|(qJ!jsX_0~fEmU$aLV(SWRM_2rM;4N z^?}O(6SGJ1h*gjdDV)!gRDA@51}qnPZ^~rFE+oD(x#cp%5X<=39#d%oRAQ8a8!M4$ z6OTCYat-&?lvmbmOM~L0Y#l})M*)0K@wz6XI$+BDLB4QJgBMeMJ(uge}HEC3B-0;@Gs)|azqQ5GJ^)<;1&EPBS{OD36zEi@or zIgCX7o*VrIeLQ-@3_iHy?1khiF@^8OTp1Mt&ro;{OlV9)-?J$TC74wQJI;W0Mfw~N zuQ0!S3xZzFbQ=E=wA6riRghiO3)2l<>WAsT02hDLPn+AMq9e9XRC^^9)IIX@VB~F# zT$B`EjINzL=Z;f!LxD(|k(|P_D~Su-v1?N2yYkirgyc9l^VY-6CHQGiy|bYysy4sz zwF#`oX&~k`=BMMzn8pGWxN*1lxQH@|_t);$onWzD;j@3)lM24J-ssVN-L3F0STr^lyuf`n;71TYj}N7jzM^uanYt zKtFM0kKALI5N!q2kSxru_< zt$N~s7-3`wig;Wnitr}AXdb3JeNG5RULPJ25o1;7Qy&WLVR5i%ct$$qCwy0KOpuG6 zMnBK%0(MfF%Aql2u;USM5;_SrnHmMFC@D$ryJBbvxUrl)DbG7VnM#$_8{dTM@egW% z9y}Si_@-^v-2nH`cbtn!5|RKVpP>WRi3Adx025KDLH%K_b8NP?ARevb42drFB3cL^ ziBd^jtB1T_#YB{9!wJr_a1J{U z54Wd=OGpKG>Rf3Rf!E~Nvm9o4wx{5$yQI5YB=9jbL-ekz{95{I6gu$Qi` z^8&8uuy&&&R1Tob3d_4=sd;$TeLJMt*dV|QAincrxd;5>|Px-eYcS22k$h87}>A(k6s?5s+4+XVjs z*rKQ~r{h~ZmSVSwIy^Vs<$C_wwbOt}!dkM^@E!AE((6_^6W4uu*2vixvQ<9Pq)3MQ zgEzA05#9(5qXceD97m!vD$0-OE0)a~iNX2utV5Q>1_F#lkrwm+l)vh8{u> zUj2!d$O)|Y2}z6$U|{2R?5IK3v?6QZbUyxCz|=Onf@)Oe3-fOcZ5`I(xiLPQr`{n> znC{#oPzS<oj&ls$v^<>51s(rU*>>$%?)jEm)$h^@vrl@Dqc$9Mkxcwu-5geX9y z`x(sOIXTNFtUIqeF z(>XChLJ(b?OD8!6f*&-D^U%TW2DmLi-dV!@Vy)YO0GBhO?2A6DknujiGg7qBR6~S! z4(+{Xknxr@CRjjEUqnQp!AL z0a8$uM01eDI<3aoULawKv8&Go9A_)GS>7YHsCC3TCKeUp{~Q7|^9$Lbk5s?#SRlGV z$J0V!g4Yw{T^^KHy#*R?gVga`WpUrOC^}Ek`2=M$jvmmGC%r8k)F1~nZ+-`t2LLYr zB%THk*|683g3Xkn5R2&YX77!uK#>(2<-1Th zKSxKvjh+-CN9(ZJ*qFI_+`eG%R?t7DIL^HG8|aeIXG1T!)`a9(^GWa=7n31gJBfpo!3PKo{%7z8uKT z*_k&R_oC1F-ZUU2{D@CGXz6#B%}VmwI;bRD9&SAH=~AJY1lQ-Gew!H}?>({hE1@=!GjTv3oBucAFb zb?$N*1~^mBr;8I@SU=v>IJSw6Bdwe-Xyah8NC$aq_pO^;n@$UjYhgStQwCfL#2~1^|(vTN|KXLRn_x-eAdc zZjx?gviGRorA<)!KLTS5PfaX=8}Gh?lR3cjRZAK-0@)NA@FT?lB})RE_%)EL#`nq1 zFG|rDz!Kq2g0sPXV<1?rpA6*jss@Sv1#k1DAVz!h_F5=|>CG=;#^RNI@!GhqDewdt zi&#BRIGJ;S#xv-W5_P!)a18Ha#O4(#wp{v_{mJ<1rU|2+XJXFR~H7 zQ!(1ZTLXpl#hK?s+zM&?RsmP=5EQT+Zai*;F5_5ItixZC6i;ObRVd#v$)l1|+e>V6 zHJKSM;BA5^pbOLhWl&K?LC}0hNJ|5PTCgyYj6i2|=At7fYwH8)Id&TphR&iJpr`0l zmYbl9OC|GFc`5A(Bv7P205nm6D?iXH7tU$&)LV*)1)B{z>{vs4}%=rE*6%%_KP z-scHiP?g&4(3q*bs=*3k6mN1!7Y27Sb)cMTm7c4tSx4hc!MQEauV-E)$hM&DTbIL9 z?rpEHjdmZkdY%uL=q&#%ZiH@uHI8RZz1d6{5v7u+7VUc`--b?6Q7&phBQBgIh1X*VH0}&`iQ(BjnpUWG_7XV2|;d39BllT0c}%k#c4sTHcV?hAG9S7gfxbl0kqAnq5kY}T6pe?2F_cHG8sp02 zSh!L&y>i12Ux|B@0F?cF>ih`$q?wyk` zM?Y-00hx^30|wu!1o!kFDwNZ$a98Zl-^`!6z}34QXxZ_xkKC`)Z81YOkv;TE-Je*z zJ@u4)z&s?EqP}|DS(~ISosUa0_qvd12MXD$_8++iWUd3_OZuFbld64ZI_lDU-%0|P z%ZFe~znrV@RK&o18f3B|pD>KiV)qV8tGEuDY8#Z3gFf?{gC)21ZrJtQNx-TJ-2W*k z$F*K$TkE4M#aRRHDCv`$gJYQAv{&a2J);KD`IDR>DgLuL-! zAJ=u+W=Fv!_$6A38t6Nt#dAdS7tkkxMRaZqz?Xl+pdQR+p6>`G_(}ud%R6qh}Ipg$1pE>4MukY*~H?{9e4eVab zqAL~soQL4v^~vPPs7{`Vn+d=-q|1Vk*OV=B!U&>eR1JEQndPmRe!Z`d4sD`r9WG0n z1@3`SRmz4vE55}q#sTfn?D%POS&##oZil6l$Q>?^;dZ(@pI%_QWKSWqO}8`@x>C)h;P|g!XG#C3yRBI0K?vvxy;wB ze_$qMPsn*MI5sXcJ^?za4U{fp$1&S;OP^7k%dH|Yas3{Cs)Z`>c7i1Xk zL5$Q?b8tWiDpU-{wYakF`J9Ie^Wcpj)?DtLI-Hmqh3#^4f_67ic^8I?bB1rEFDizE z1!)oTty_1(jrgEMxrxxnzrOv7slm(3(p2B+p8}PM@aJqn2|?9b7A_TX;EX&w3u+~H zWA)RM?4U1UR1zG=?Zcq+gHvoDA0W88YLw)~f*U8&>H5e@=)dDPO@dZs^m)OQ4+qPO zlgv9E{U3mgVy^;F^(TFabq+>E*v9xTgl+r=u)$kUzq5k-Dcw6D7pELG&?0y2kQpTj zu3aAbU{dW?IT{;r0M>m@84rLu4L~sivGUg^a4(8#_o>_S8TZFvMeuxLm4=!7Bj5M~aw8f#D~LURZs2qyvV27i z|4LYA5jId>ph)MF;2gBo1X{+&`EG!M3_Xo$zBrmGkXsY1pOfGF=r29lC0N$XUYy!( zov;gvzO|J2>}v7-sD@T_qhlNbycj@4F=j6 zz=4C7#&I2}s|DQ_O5nq73|kIb8imyAw>P&8RtiIj#ODExBGGazITgZoEVQL4b73-2 zfD+RBivI$B&Sc2?Gc5K$0v#wUK-SN@RpIhS9rkpr4dM2~9Z)5gGZmoL54XyCoSjEu z0qziHE*eOo&%ZxQ*k92Gb()0uju(FT6SV+0A!BYUy^Scq(!c&M3<-ouQLhCYC9&|% z1DoJ}IL4bo4keg4{sp`KH)H~GW6m?62{K?{vVc+2^M(WzP=ZlThyL}wDaeP!5~HOM zhoD1C6mhb2gI*-?-Rm-_1l2#2twbfL7(s&S|F33gIl?{zcCNGZPzW$%O5>;aF#zXb z3?jW=i#xB++dOAKF#{zjnLvj9PbI2~G{q*>S20?aN*bd;TJ^fhk2Ym#ss`?$ z2`q08YSxed6DS!z9lGN1S98iurF!k48+0<-I$-ccz0L&rO9dtH2wbdMiif4UMj~Rs z=ikE_|M-_vU{(b&i?scWo7><{l}|NZ-`eXULB9C{mUr1w+;&&cg(B{}Ctw>!WG#6U zv@}ULef#1vI4?N6qcmS^J|a4*N2uGr`o8_Z0dasHkgOSqOz1X|frRU0v=%GGuhg)R zHw7_!?9Xbjh1lW%NSX7mt@rBJ_g=q)S2N=(@(l}dKKzYOcH-oGSQTWn zO;bqOkeOo_EPgFm+Yna;R8}U9IAJzA|6&9Dd>c&h$I?sN0R)|xJ5yB&51Gqoj{zqd z_ns>xBEB%)9^mjYCd(~I6z4BMR0o)03P?zF16JIO#KN|@FC@I>`txka2{Q5;fKPb6 z)A)i+G%JDMLYFDop%gwNf?QNcs^tfMh6;^#w?_G$XrX9nMwweRvNzc^HTRfMbfdA)Y>>2P`NXp3^;1e}p zb3BmRy8wQZ@x@Ps68JQhTV=a#_ImxLX@C>~I6hBS&@#Rvw1XBB4h;J0-S8_&Z3|-rZMEq| zcM3^)pXU2MZBC0+$;Du_s?6Z4a9kilbF4}KYhQSde72tKP72-1vSPJIut)g+Uyb-K z)=}g6eELMt*358Z4IcZzkAVT^0~w)ox+TU))SXL@vt=wFVo82#SlbHCC+Zc@2Oy4s{(+0F3EOUdgV+GzNH^gI*fyRX zj;LstI?g2CK8_wIKCqx7+T}b(z!D9EoGj)`ldoEPgbk302kya&B>w)<)4BUIA8vbo z3ClUuN|8nf%ECwL0AbaaAgjKCPJz@=2-(fs6U{+K-=79@_A%{3Jk=BY0iB74{AdvB z&GF9{;sPIX?6^*kFeLB}2H+hIl25@-#1YmsA@V}_#OzOorb;4ecN|7{i9Nb+c@7-n z@o&?^e3PZFfSJHA#(Gl7QvjsqtBA;^wi_C23mdLk$OZ6wL$((Bq3VlI~vDM zUm}kIUn-LF_VLxz(d)X`jlswJ zfdT~x;2|A2?H-nwN}mER`V-6CuyL)kHH;Wkkq@Y){I3V-M+!{$LnLk`!qb5x^pPkx zNm}GD8Ki(2d+he%r_{>iQQO3L+CfL!0d#|1<3xQRXh?K4kR!R(p>f;OAt0CdH+9AX zeLTBM=6IqywvgoQ^@=Oah;hbTLU&PIy{v>F4n2GXoWa<6*`rIKpo5Ti3=kH8aLz*{ z|G22^Cx{OK@(_TmKq0RY4#3W=d4(xNAgeH}K6A1NdUGizjs4pVh zyl}dV^wJ`YK^nAqu{nhUj$aU(WX5fI9nE7zNQST!X$7oOy0bo%ddP(cQV3vw3ZHy( z6xeOos)aEUYJnu+q2EusZG*Ld3CxztuM?@u1!O(vg8%)OCz`1NBks=1$z?@O>ArOL z{*K3(sygD&mb2iw9QH~?h*6O!BP#C|ezMsj^nix-6Y;s{|#g9{{bV*<%p5RHccB?S_d-zNuBXq0@Q z)!oCIZO2Q{FSN+UC$w#+eGm&N;F>^u1y6nW39pYDJomZ-f`}%q3}pPq$#8{}e2m?#(mElERnF2r{_+xbqBP$Uyc=O#_^J#um;NQp{h-As9$v zJ0Xd{@nE`Y?;{eB3d2H<8YfVl?nJ$ZfYG=d%_WAMVjcuu)htDX@*hHjes*}Z018KD zkc}YbF3IMKAqQ;WFkcV%E$~Nhtwn`Kw$;5@MH~MX|Ko5gRC>83<;rY=L`_> zCmL2O)a%;i^3_|2KnK`8c&dT9h9`3RUr3sK337OHaS>mv1~`f!u)Eqv1@V8NQO|&- zgnz0ba}GHt&>G|W^baSxS%j@iGXO0GYBFmeWCO@xh_JK-yW{MM!}-Ux;(zOk!NRh1 zV-XW6>SSmg9o@Jff)qR^WR!d{@JG-TALqZe{l(vGka?oM`Sj3@?PmE_H0l7}>l0oa%V`3ph;Y)}(SXiFp@*%ZggS4!Fr~(5@o+ao& zM+m-E0eY~Dt|L^f7*Y-f-Ff0f(8K9R*3HJt+tb#{75d5D+6hlgn30ze`hkz1mrq3Sx0gl5PF|LfVB}G= zb+h+&U=-s8uW2*#==yosGV&N|SUcHX^Je7H@v-(syrS%CWe@&f;AjKh%*W5jqh#x7 z@8FI2QQ6Jg*3-k?#md_j&)d@nyiLyC#obfK!|IwXqbTxwb=0lAoEiCqp%H<{$U1s^ zY1(?qxx0F}yV<&VgSYW0IJ$uE0sli^bj9|XyNxZLq$J+Sly0J5v!p&H$A*Q5UW$hP zi@8D*K+N@T^Ar&{dFgla6cI!Y;NLyZlWG5-pC@7h|2R+KU(8eZPtEhMs#D~|YW~_Z z841wP07>|N>Jl+{#2op4vspeqq5r8&KE4xk`qSF{y@}%ST(R|ayk@KDY30Yr^E+c^ zL>luyIXT9Ev{mcr$C|o&nCNJy;OGd8{6}aahmM$|$lvD3&-aTt^7H>0i~R>p{vC$@ zcjkx~`XA@`D?Rf6RXqYn^zWUdS;PchFe5>ji0gS0c_s%D>xiX8hB6~FA8w+lUGz5P z6my-mZ$Fk|ziPGZzhPd^lEO^4csU@cOVO(}0v~uOR2m{TNleP8pyB+0jXC*^ACsMM zD(>@zFCW~llHpALPt)f^hys$CnDF22LgZI=apGeBwC(?YP-c)Q{vb2K-`2k0Ie$Ju zNRVLp`O`A{cbl1x4km`4Jnn5_meXi>_`h@*h_NHUB>FeQ5QMxu^x8j65<=KtndHCS zFl^fqV3Pi#K2CBa7H90yPUzlI^hkc4xB&z9XL2SQ^^#^Xz8~tU@9$G*j+JY)u`vbT zJK(Mv81k;WkQFV)`gtz;)3-h}wr94VOf5xK0)H<3+&&c_{7WDS{^n@J`2Qv^MELPX zc?thjqxkpBD?$}7n3v$x(;=)3!3Q{Y=MQ@~j}!%OH-)`S@eB82m_!S{#mLlR>G-aj z?LBodNo=4A)2ScEs**O3=HKKwxf>;o620y9b-C^zu=$!bz$}t?ci-OOa2NO0jbDlk zAqfPIg#IQnp2Azlb>b z1pZXW3BdEaEyTZf5x4yh;|L*%KjP@OHR|`SS_s;S{nM`cKPa((Vm(5P{f~C>s{mHu zPX)05NxRVN{D=tdzywm3gEqxI z4M}t>E%d4e&sX>k$^DcjlDVQXi7`&VmPM_<;b3RS^!@Z3BT?J79BeP#U;0;sB>rF* z{J(+{|6hgDe}TV1jQtOGAq1HK^x8idKQ!}y+W7yIcJZI~7ec?;fY|TW-QV&V{~$65 zU;mXN`;YIFgnn~1zuPAX{VE(5`mYU#|I_XVA&Eb@AK~A`_U02q;THp#X&f z6beu%K%oGI0u%~RC_teAg#r``P$)p50EGe+3Q#COp#X&f6beu%K%oGI0u%~RC_teA zg#r``P$)p50EGe+3Q#COp#X&f6beu%K%oGI0u%~RC_tg$zYYb_n3!c66^NpS`G0Yo*lw~RrH&p)3I(?m5BM>SGMb(2SR(np~Hg#r``P$)p50EGe+3Q#COp#X&f z6beu%K%oGI0u%~RC_teAg#r``P$)p50EGe+3Q#COp#X(~{|*%FU}Ba8jH3VI%pP$N z51rYIh>D3pe@FJ2`BeuUk&_NOR~ZEud1PF?Z9UzrylsEG&0AYhmXSxn5qcT??*jgE z#rB%JjV+_3B;E;6;3j&JfxV$l2TiI@Tue+a25AhLKe>DiH}j?@R^FXQF<4|O1h_IS z#hmP~d`Z#I)v~{sH%2qUw>F@BGf9Qd&aq_u=3eLXcr^bT$DbH(q}spt`yle&!2Z3M z0GfmTI=s(C7eAO*Ih9jbF!(Z?SeUvWo@Ow*_}XbJEZr$1pOU5u=e|jyht9Nnw9|i}qPQAdiz6tsFbSq`Nzw>?U4IEk8A5i#CRR zt_NvIzwRs3j__#0yU*&;azFY>7@bC=d8GON{Wp8*OElFm-#DBWGBm+K#-Fp1H!&W( zWu#Ax7YJ>N!FK7tT^hfRIk(<-&voPG*|&y?_yc#NLR&l?h24`jz8w2?bm%wvvpp4h z-6vmOYIt)v%%|?T(D^$}-VbhFa3p^L8w%OUq_%@MUsQ+Vs0LGSXq|76AQ>VnW%MOm zyTimNd`I+3+~tsDciGe=ZlW);1O_<9jPIvg<<#c6DLO-^8Snl4$UgdTM8?8Fua+}L zAlIZVUIzX(P0NOv#IyOxH8$pw&9Pa=MM>OFt8Qv8i)W(f)`Sm?HVQ?V=&3R7j4t9` zm{EyRBUNp2$DK8oRdzsY;iY#muI0Sj{`_;7=L`wkwYNzTEvB+mt>Jnz;a|hWSeBP( zeFw4JFt5-*H&ben=Km*a|C&U*W~9LV(mDl2;18n zo0m0Z-H6I!C8XRmskVFK-;sM#vr5XOP{|FDw1(%{ZZwDc5Kq6sm%f05B@;||rZgwy zEF<}Yaeo^B&+$$du-1YJY|vmHvc~IVGOKpjULaG)ijbp^02U@lCW<%oVwDQG5}ss2$Py@X&pksbWnU(PYBdL(SQs zZYNRfwfeSUw{S zEh7>r%Nl_#OZei|J)nC_gLdvxxR7O975uH7_C8Yc+0$2|}xP4f_bgAr_x-t^HPjdv>OvC-?0k~Vg`yfW-KF~4N*BYoBR+XMY}Nk^e%;1#4}lLCc1XI8PdK-uVo}NsZ|txw5|=% z(1rS^(+XldQp|aA)`8pj0c8e9uV)3#MC4FrPH)#dVFh_$_)jY|oi@YdvvUvpEJF+h zlkQ|)<9DIgzmsS+Xh=7N^=+4CD16H@&4d4a%jm1~rf;lnV!UtjTKC)We2p>nXn5Lw z&3WhiYpP=f>zId1L>G9-9t(yi$fPK^p#!7AM>iboliL!u_*s28!z`|E=S&_T=fBt0d6NL^`~}nKQi9+^E=`@ z&Q^$z8I1*wnfUnlc=>2-g)drfom!k-#2({bWD;(OB98Wge=>Ra_58Kmf{bErqn3HHeL@b}22 z7zo~~$Tc1^4l$M}@-t2}R;+Y$+;SACY;#g{{^r0lJee7&wI4<#-)78esFQ<5(H7Qzur5>YRrGDD~Ip;&$ zp0DNQMjS@iMj}ROiqz4cACg2F3cqk`86;)cR9xI&!rxNe++LdMJ$7jl#u0AkH+1c_ zTx%i`sStVQAlmQo@{7^i)Y#hd^@nmLMEb!ihvz-m4T{iEy-H_do zyW6!?%ZB!V#$8POl(75qweQ3uDI?8Qb-FHgi6vDg-6+b{r`KmHAk_a>BxyKq$ZEt> z=2m*!D%1UKC0j*aWsQ4R>DJ4h%A%@zH^r(SrCsIYHp6bwV|8PZ%N#2uqUl6#MArA! ziRdzwG9?t174)KOXO(Wuh|XOa5|^aup`jD=|2Ti|V@a!Fl=9h^{@uHLQ_oDFmp}Iv zoj6w@ARMs#qD=kTl<4@SAI~|Sm(=l^%9_sA3D&-*|HgY&)a}Qs@nW^%i^I&tT*Vd> zpC)YFeB35hlUDOq^;WATyf0fzB;7JyyWn^F=KF1lx!sPg9*z%7>BCc#QIkfKw_aO3 ztUk*_+j#AOoWYAB^@DKQRf{V3{O|fxSEU4`QY|W{t_9%-g&bc!{ziN0DvLeOH@#q; zV9w7G^57~UizSP(5m`X ztEpu5z43Fa^pYY6YFL+D&; zjvo+n?1tCGbF-dfiHtv|bCE5a?UK$O3$}*2RE=}QQo>g@k(@C#u4Mn~DPJm<9our8 zuYOaPN>EWzSDA?43!~~OnA^&_-k;y+zZ-Huc<>k_=#98cb+6(pg&m0r)LG;C|cosO?F(05OFFl@gw@{pbf{ z;UzySr7t`$*GCUVxxBjPIG_D+;I?0_@@s!x;m7rZYmTbtyD>?v<+kitb75E`kEd~k z-N(9j%8^ea9!650xxCA^nKNen$l7SVhSZQEtC1GJ*TC`^8bguw~oqVcl*6@THGlP#ogTA!UaHP(^#N*|5 z`zK$@Kn|E`tXej8dq3ywZ0v$`g_Q+dlU*}$i$fN4kLsNM#=SnvEYAh|p+n^@>*ft| zsWC~U)ORc@PL*XU^NY4*8&lD7dz9CbQf%&zLtXUdhWI0zET}fe7tSNM(Wq`zn2o#k zr@Mp^V$ktRRI^_jYO!yZF0b*;EzB>K7RwlFU2B=`Lr++&r%(0^OnGNVGAA@s4NEHIE9ok}&8|FJF zQz(F(w_atH-ri3Vz`$CLZB-pu07vWhGG>N9ll`h^X8u#p{F{w_ zv(axh`prha+2}VL{br-zZ1kIrezVbUHu}v*{|~XzPp?4#U?cjUZX^E5Mz1LJ4>o%B z9`uS(|6rq6S0u8y)W6y2HyizCqu*@wn~i?6(Qh{T%|^f3=rn@@>5A>2Z1lsC>R`?L~C=N>PkIU`pdvR`-`$?^id7Kl0|+vC}cLG{mKrvbQrc zws^VEk7=wwZ~BAVs1*%tY>h1~aH(mj7=Fp{SMi^av5l?W%PiQRP8G#2-L{&Yt7qaKtm75OtSmOqOkZ)|5~fJ^f9tWHBiqw~5mf8FW+b!zi^WULnS zsr&Un;B_abKP;uL9&+(forac)6oOj7;H5l2Z9}cBqKf-6qen%<`r`MKnU$G}h5lty z&%)l!4EM#L(mFN8Ca<3nHZS< zy3>y;zj)XCtBR7Zg@64}z3y;d>m<{o@^Z}lQVB@|r`KZr*(iS%^T(9pujToP<}Y;p zYw)jKOWMZ3;U%v>z4-ozf`tBE3b_sgtB@8?3({8&=fBty;;-5<(z8-AvbcCDeE%`F|6%c;UGeYM{JT+pG?Ju_`O7cL ze`!QB9Yfnc8soJ&`Cfi%Ak-)s8Ch^C>0e|E%(N`Hv@{H~F9W3L%WsCU9Y-)ZTD-i=@LnAv}W_m^lYB@UtbH&#yl>ZE6dTn_L z9j8B<`;Yqm(UV@(;RbXsgVT@Sl3&M-pKlj`eQn74Y8Tm;9%Nu)XlF!=OH0e}T9W@s zV!q_!?1rRJ``m0_SU79fP(@9UO2mNYXNCpjSM%+C2(Ay`7cd#1O5gWn0t9`2Ja53j z#NW`MJHvdqxTX6vMs0g?0n65^y+iFM=}(2{SoeTl{ob{bKB4b?%O==Qwbi zG8GMzJ)AZ!lF_sWxVxR!lz+hxg~&&58N>Xm$~g`w_HBiym)K;J|0Me{`{qgT4)=F~ z-n&DN9#Kf_1a4qTuI99^oZc_~AvlZy6i39>t*RG&q2W8ufO=G1fP{aIBgA@#(Qb#3UwtlioJNF{EwdnfMs!8Hvms!pr2B6_y_M!sdW4jXAK zjMNC7FTDe^9Kg6qHEt2n7B{!haK%?QWuWP2Pfb4rpwBy!`{2BJJS;drM#T}OK;Kal zQ^4W#_M$V)^xd2H$8>j+8|VYIx;;7<{0`5TOy>FVf#q8XRrJ)QOxvB)CaYENq!JT> z>4~z+Fh-4pv>3Ph6J$csj;WCdKx(M-n0QNkO7C~m!FThvE;edD90~JsW6PFlP)?O9 zOw`C;?#CX_PxCeGE0x*hc|JrIJZ`W(Mv8i(UNmR4-tO9fxGPD-^Q;r{#se`YJu*+V zE(@(_Qp~iJX85xvNg%y5-Nl3d-Ov{%Y*`DZ6TLZ{umohgZy#4S)ASSn;SjBcgbUVmBru z8ky=yAN9^1$(>}^*MzNNq$m~cq9tQva4#X_8l;(gcXtM^M053gyk&ke%^4ch?A01vF5h^mkL--8FEdi9?v6tC<}KqV zL-sDTd>r1m@JcV8SEy>xaz3ZXxpKv&*-ls%S8I$hQB+Ed&A=5dMP*Jy%t!GuD!1k) z6{d4$C_uWP)LtF}y41=qHMJL+Q+*lsu#e8rf}Hn@;rc7fkK};M7%eiMEmr0&5Y(Yq;QH7Sfz*wgb%3IYs?Wo%F zB$T0(Ov(g-bJsBrUR!qt;W0Cu;RZ=qtc*mTk8j9O7#`aN%o{Sm6XjfKK4Zf%NQ^!D za2OB7!c}rx3kq!1w8a zMDLxfvG%dx=FxK&mn}VOOhXG z@q}6t91L(i_92`d{d+R8HGqjkPIgyLaw-VkZy=zcqC}3@mnP1#8N@%tiCsSD2bBrk`TSNYw$<`D}irqylrP` zY;-~MLQ;6Y`Li9W8j<*e8SK2NFo9ub^{IsPKIAJ#0{V6&bA(bUwy<9`52Df@q!39M zJS0i?cEVc~HCqtQo3%j_f+*&xix)tOc$)xwwcFspdH$Ij=jxe-TqG6Hp~K6i1-_&h zAE;Pe#rs9n30;owBt)AA`5n=XawXzPateibj_eBf>x63)wM*EIx%Ym%8S;X~Lql+j zj@T|P`*3)b?XG+0`J2|Qyu`rgw+F?|^#Ocg@37iy{Ll-$8gJdM-g`p76I1r0)+a}* z9uUxek~Z%-3@pZ6kvJ5rg>&z3CmVZ4Tj-prAU6xK`Ml)&O3>R>V2Oe7vO>3rX39CxV91 z7m~Cw^#ONOv8QVC#{F$$$Z=NS9wxP1^mOmu`y$zQ5_-0Cq9uY6+6uY)iaDd3V$;Sw zUqR+R#Pm)ZH>GKSjDYVEhjM0^HMZ00xyA&|JDGDhr3KX=_0*6#O{X66H^9_e%7ftK zY)`^{m;16jQd^GImJHNSQyXbhamT0lr3BVPg|lUE-lZ@=0NVIFrUBeS>Col@&k>`D zEnb+mdxemXLt^YHXGU^jTvfL!q~7k8{-H#4X@QmH40->z3@2e!=9?1=5jMqL(UZ#% zov6u}sWu`8KMDzm#X8zdp0HU6jA| zVUvI@358(LlR?n8w`co&t6NrBHU5s;vvN2>LJ?E6K#2VS5}||y)R_VUWfWF01g_{G$jR2p+k|^r+F%}aeX6u1Z`g@|#MiwDgx=$b1JE(_ zr#Z2HJ`}z38tH&nULaJUM~HEOn3M^tQkf6~#I8{pABT}oUPBxfba|tGGbNN3m9Wp-;Guiskv5=N%Te@4)(YlVLiH&{yw&VHb(a6H!WvQL5t%@fdCT;(B_th zh58t83BOGQ^zgeh;{`1j!{Xx5pwS?Hf2YlufnJVSj{2QyRgXtxTsq9%e?-Acwi;*u zv~sz$oa4rl$ih7&=l~$c$cO+VOcGC0`%eW0iO>4paMqhTrgdaBKi}3OByf5xE;@gWNY}ttKGI70 zytbg#^091pUaWa`*2UloYh!yDgW1BuIb*SvfRoepdH)Omht6YU(fTE#KFdu@QD{4h zf8eUM|F*N~bdCBXAq>NIC$u;0;)0N0>BLtZP07wh^j0+j2z8tnV~s{g4ZV_K5Q--4 z?VUzH)z-o2W&TxI@KAUPw0`|QlRZw8mCJW#^e$5|NE$4` zCEOC_Tt8IReq(gsX^M(TAVi#L@g%IGDha8Xh_;!in5}}zKuIyQS#7kd&mt%@uB-2+ zH0IsDwIS<564lzM^Ec;jvGL%Z&30?Sv;%{K>l&ARk{Yh`PNg~!<-pO^V2t&q%tL;T zTmWh^A7d?DYR!go4ZH8jnV+l*Zv(@~i_%foJ3y^BZmn~~;$urmR8$KwOA7MyOA6E* zx2Q(bkQQhn>=d(^y zCh3i($Tz&5-`G7DH;T;QL_G5dj3Hr7$u7P+_Ymez!|u2b(sQf5k>MCl|AO(QM>()N zgn6oDUbnB5`Cixx3wDv8%|G0VQVne&r`u1vWVJyEX=2Th?LjhdW$!yX7EDp6jy+2; zOyKldIUA{V%1ZrVIDFY08Meq4wXY_Aeq`$58*TcXl!4|U5zX2Ws zJqSO$472+lV_~y;>MvLrK2dM0RGPy6wc5tlak>jtGTlR`W8VTAqbz9j=H2n}B!k6n zp!53IFQ2&E!I`v_ILN=6tKDAIny*ts28Q0Ot+wNwfkS^=CQ%G_Uzex|O`e8E{|fi~ zN$G6*;d*_3hj$5gr=aW$nL?z@*PB2(YQw@@GEE6aK)bTs0u*YMVSwYBs90y2&}jR2 z_;S#e8H^~yx=edQc&6iYB}_ZFV&gOyL*A(KRNjtA3`BXi?9p=fBS;U8IfhTABKq`0eZiHvM^T#pSKNW$P9Wly^^qQ=1BPLMo0sitStQP^NjT4;-KgK zL77s?B1O;lN3F+dlj+>}F2+--&t(tIk6}@l_p@Za-9v(o-0Dr|OU&3#RPXmUD-em{ zQD^a<4Dg@2E1^8q4-d>7<)QtXELM(+E9B+O)$2w_Dldy!3>OH~i^br!^)nhBSO+0( zr6ADl7+n{`UAs}&-{5p09xu^ah(B&-Nk|AoI)sf#NNi2J!E3dd z%E?K=oQ24(^Pjp}4{*O}B7?(}BX-!W3*?q9t0>lC%w&VCN$4{iWg=v&>fjMsAnZ~{ zv!~}3i<_P8-8wuLb@0cS%CyNCyS^%OdIC&N4#ilMTa_K1bDhP!h020 zjWLGvI5Nu?uIi=*sF3>$uv20WsNSsrH!~DT3Ec8Xm+QgH9`g9A=0Y^LB_n6fo8>F3D%G*rLU4 z4cv1ZjmM@O$Ph0&uhUxdGFWppre6j?TC{!|AYIbKT-d3xTiT$-r>m@C&OQfK+u-d{c# zVhm8RFNlp!-U;Tz1|||Dw$j-=;)nep#3Mtzj4L98)k}sg%OjhWwlyd5>9EV@C#xda z&Y6U1bd3#n(($f4)x-Qs_i|HXkrwiJH9b6IpYS=$DdU+ekK3@`tx<(|%vs(I zy5@+OU_nL8#aF_}R5bPm22ZFmYq~n;gh`X0ZQ*nO3)>0-F-ee5UMA{71NHB(`Bo{F z)$uP1^0Ow|h|BJ3&zKixLl$n+4%Fe9Ra@P)2Fx<1MN4V4P4+pQ}D+ruQJCcT# z=~L`=Y#$yY=lTj=MG1$C3K_(eR0s+_Gsb;NiezL`uoYCXtBeYu5}CUKZhJRoV+W^poSDZec(V0Q#FsUL>A%3xt^1Nj_R4(oIT@Iq*23(P{QQc z6l5PUYO61taMtkg5`bO8Jf5m8I8^(tYz$`Ix9OeRqQ2o*mCtBPCb~oxvVGZBiS1#1 z)R3CYbBy~7$E5W^9QnoKR0k^-a}eXP_g4}n^Q_O`SaGlrxNC;F=p9c3t4t2!%~Blq zaba&KH-*f51A)QfI62%EZN#sHEvwzq(mWh4KGhJ0V2cG0n-#NPT_b%DPi<%bc44TX z@(9GXYkG?5yEFY*XOnE#oNNK*ZCDs0c=z@**h%y9mmE>GRWrm*<~J+|%7xr)F&*W= zU#GgJ*lv@$#srjVqxvdCjf@G{1wctdt!@*jQ2K4 zrJVSVPIk6ni_+;EJ#PA*25{#mKBj9Sv1_lk0ajcOXPKH;wYV7ec|ARXLj@p;sGhZr zqKtj1T6ip2EQ;)CVk@L!5UTkoZ`_&Ca}7sEdsq@9PB{qDjAmio?-gbUI_5uzP3-6E-1`wCid& zIEW!OIe|@DOdnVRg};Z+yl+ScWcZnf$0;0?pl!R^@Roz?8gcD&g>y?~!I9Iw?|?1X ziMW|e#1c<^Kc4}GKw9ine^a5Li_NrN$s_u~!CIrim{O&x&qK-p%8hHs-Fnm^Z6;AI zbB{dQwo3ek1E0ayp7QwwIP9z*w&DhNfv7@ZnB%>s_rnW2^jC(75~!usppHi0HCV`< z`p$^;>#L$FIaD+bl(EX>_KWRnv1IJW?P={R?5)G!$1q^A*SFNWe}-W{xCi7=ieqme zb)=!-oA;k!+4X-`WwIPJbz5lxEQ@bB<2~T^*eqg};Ium#0)F{F49tF7_9jP@QO}pj zdpS!>t@^5)Cr1ZNmXvZAy22JzgW44ZDKPY z9BZm(SdI{0T_`gwa}2n8ewgZ~27`(_tpu_2Thm_%a!TBam{b-}&%ysPp?+Yrxj>?7 z&|s5HKEy-@NEi1$u2%cd6VDVapIslh7(B~@6E)Kn$UAg?9TBzF-Lpp#z9F~RY^kBL zl5wgS>Uk&UGCKD=AO`}d`U+{s)oI?&Xm!ecb#$6>q!_{O-rVSbXLO`Ok`L6kEhJn- z6hDnGGJ^j6K9x!PK$@YCX4%BD@qzR<#3&T1wr)DP83@zH=AKNgkE-pNw^b2~o@V8i>T*|wDg{l#&-==)P0V;j9M~gAi_}lb=ewl=pcASQZSk_Wc~&zRvNkr2 z&SfLy-SCU_a&h!k;=GU^_``C4 zm7!GH!=~J+O0v-bLMwCDY&h8YS;6(`(_+_f)AOb)lfxYQCaPs$Sv+^oI_`t#C(S*R zYRjtirr@x*6}z<$Z|2?!5Xn&%x(nzT>-3l6R_t!pK@rPuTn{@tZ8h-V-fwl69m-zS zM#x;vaGYr`iR0Oz;YQHXrFq66)oVH`5YF%F&ludluc55elZWXUpL|*{E;AT0wg%Fb zyAVneON}2hN`o`^Fbqd1g=P!I<;!5#I(>D!aevhhcR8%JoUSt1N=>b%Ayd&%?B;%N zy`;)$9PFgzM*c(;F14n_{dA!|n0Wq4`V*)d%tZqPL;W1qlhu+w{`MDi;jFFl{=$PH z%x#WI{X+(xu0&XUn1M6b)S>>=Qi8ayXmu5ekR6sPb&u_x&Qy+4gRb9?Fevxe^`C}B&1jsa-i#EFzQn;zCpy- zCUHj7(e~r2M_QO+%nz=KAvjDYHA&_uX5@B;2^%{yHxsem(!w+v{Wlo!_8}ql-y0kQ z6WHwg`>7>o25E{piD8s#W;t(Hb5n_9j1}ByW@75uG*j;&wTf!aIvvoy_TYMNBsA{s1-su@I&wbj4?Y;6HRA>-FoW^Q1$E4I;E6J z@`op{NeRLTwWDAaFMN4xy2pWD_E~?)L3xs~$zx_}+${U;w&m1IP2@`I*g-7|LmpsM zrjq-M3zOCrV%}>a6|Y5wtZd^uKP{|(K8Ykg`J$wDzyrx;LT^-I8}FG7ZBR6=D>!XZ zqrw{pT9b`!PM6F8XUvq0G-lYFH9o{OR7k2%l!{fGc*IoHjp-(5L0>Th>k}9vEw*#a zZ;}0^Fuk5}%F=_Cfz=DBm#-kJXT=2n>QbCH5 zFxmCnc?_3oTAJYiEo^lJLqzT7LeCs=G9-nZng3PdfLT(($jr9#$2n}dIAf`V^lA=W zG1!_OG7B5vf`$FsTvFqCZG&>AF}49!qtvXSnU5-BG04^O8ktm+Y1O6WJv|Fy7?q*;c84z+iBOdht@q z(_?E1xfjN=u942Lb9*#=*t68+a#H=#Ol~^Huq!qo=XAVC)3`K=;^+`_xBnKDQzPtB zg9RqmuFtWoQjl;&*Z1P)&MS$_eQio)2uwx>mvC1u-|D^HkhRp-5(W%T(M!fSew6C1 zX5R)7;Ns}(3vXi;#!tmC>K+qG6_U9S&85mpT`Zm|Mw#%7`mg(L7M&oTE8Z8}ZxLf4 zo{9Gbo9*eIvNXn@xHUWB;I3Rbv>Xg~6|_Wbm8&=5?+)JStT0AG>*0T|U1ef|=>4iU zr|iJq=)2TUR4ICB(Zrk30b!Nx3cGA9)d_)VSk+#$WeUw?qW-kxk3Z2ftW0C6bJv9i zOPP4xh9CM!&2kgo^(sI67YA^EOxeGR6T+<^8)bobwNA}_kOUvwhOzv@I@I>bxlUpf(S1D!wA zLob>BA{u#3@|S3YmhM$U@+w&QQ#Hc+LpAtojenKr{ZNU#sQ-S5`~FgmFuiD7{%6t1 zma?gtd@nK={dEoXN$HUWb!|m+B(pAd2sVzu687a}Q9PSsUQ=Xeto&OaFOVT;IQamg~8O9P8W1}~iqHJc9Mn|R3Y zN0izf-);}p8;+!~+I~DfUamKc1gcJF*l)HROghT;_z3>_=E&XEu{X8$;a!vc($%F} zN-a~Y$dmKqHuwyyxl{@XL%ELCuxg&fGtkK*fcGRyxo{0e?cwrHwFFj^`+4pCq6h3( z^4j8WJK<70aSEG8^cA3<7k%sI+Y02#!wy-N)Vctu@vkR!A7!2pq!R zLt|nawz$SfHa=)>y9$Z+XF5WVcHVUkWh!-IK1S3%7{t#?EBEizxz!BfSl^ygac`Z$ z#RVRDte(Ydc!1l6o_`vYs((O?pQY)&F{jcxIwRI7y|~6*Rx?KB6an9bxN3rCFrG}UC}*CYn!LD7YH3s3vB`v$9+&0Bm-vCISwZyYn? z+VT(s<@yU=GM|~{P zm=hI_2-Nj9Iq|AC7dNnjU9GQD23Jg=zeFjh6`@HZ5Mh?HjGM9;C7LoDO;c-`P2E>& ztwfQo`#>pbw0B%fN{M|tnFf+h5E0J9Sm~5EW=vmJHk4JEXcpC2#cwn5YU|C~B%w}l z&&TSYmBTI`$I?1{9;s#8k_4#Uv8RH{|Bs#mY*>kk-k5(phfUc>wD0YzPvJy@+{lxx)^hrhULaL9e% zR;E^a2G~RQIdOg7v69)=P4~)SsCp7*F%930+$2x$TS}6z-lRR}EMSQ~TgV%0Be;Q5 zyPisRI-A`A%{}L>*rB4bkKZ|LKT_!P#SPBQQ} z=)55X#zy4q$Wxoh!;ppewpqhqtSrkMY&U6X6p(v>fFkj>I1m5xCt_`|(-4t@gQO|O zvf0S~e&0Cu+K{H%1xW~rc)kQ|xm9D6L*CD`y71~WIkVPy3LDoT9BE!-<7H?8qk3;- z*5O^oEO+_d@zxQmt!9(dglbFE=d2*Hte0K%51+IovwCu6r71nZ(NNs@2i-|jHL!gV z69a2X0B4~vp1@y&DL$C^exLnmmkpDl1CYOrVrDopG6Ahupo5VgF)1d^h++LYfDf;g z1`ah{D@h6bbZJN(4M|60vA7IO(>NCFqn122U8piofKS6mr=|%*KLW-6H=t{AK?o%I z)(;4e0cmZIHaa@sF3z3K&Yf%OT->uA2BKN48SFgs;q~avP`D05j8@Q7R_q-m_Jr9w zaiwPIGt=Gzl${pU(FHx*^YXT~g|*IUOJmv#VPS3&$7rntvwie+tT}?NI`Y24j6_HG zaFBfA7(zE7rKR)T@DP9sz%^2xpL%aANBD6ZyaWvaN4%eaChCZXISj%V>0Ljx=P4_;CTY+3}&GJlqMqzb<~dnE#?y^9%j}YjU1@Q5S$v#eC|ZcxhYpFa{n1TYY>HSn^FF=@ zAH|^G)+rBE5FKKVkHuS>Dl@?j`Yq;TKIh%YxA0QhSnI?BHIaCW7RBiEMkyI14dP_z zTZ=Nw<^qOUjFsTV(jfzo87Ln?M?!qN2k_hg3p~X3l2}|Fx+8Bh$jij*N>NdckAHXA z8p0JhXKB^GeCR4x^v*);tvuGZBU|e3UnFI!p;WHTpky@^ zw3zwWSYU>`kfspTpr+_ZJ+JcZ@N06O!cdYcoo0G@)z{G>ssZsJAqP;s0w;U-N~T7m zyNjGuWI12gA?>zFFFT>7pfFFokzm@&0_c^QSxxE=#Op9zL%Xp8_I)AQ0Uuf7?xJ$V z9FbG_LX%F4uQJG7j72GS&Iqf{7T+H>0FDZ2gJVf@$S$J-?1f2ru*g9R74peA_%3%& z2M`MLTzNm$D0v?l9fEU$PRJ<3--@L`shho6KKeaM)D+KKhFul7s0deyizG>)CD;tb zOI9+6nUg>m8fmRBN!r%TIi!!;a#Nou5-g<3s7;k0 z!kiWDh3}3F0w+qIl8(27{aBO`C@qglEJwR%Bjq8saI0OP(V6+y-5Xh!ti^C!SdBz854(@0#k%bUJR!hQdaC50RU2=o)U@V za#_ZS6R6{}47^hN6sRgrKQG#GvE0*o;s?egD7HZ@x;5~ zKv+*^RRZk7s+IGEka%uU2Ck76fQxVra=A?mV)2n@wB8T6G9n&igwsK2{v!UYG;5-t z&#KzEMOSsadzXSO*KOZ@Zs*n`xd&|8m9d7@yN}6?Xe(+P%v|H=NNi< z$j(Or@|!rjybvD+!N21;$>l?4J4`b64pgyV#}q*4Bi!GRwL`j*^Jp?6bi2v)I$NS{ zT$RO9Y##%ETI8};66CKO_td7z?nM!|Ojp2EOE6DSQL@Qs>bLkFWc3`{P*{=vDdc?p zXm7^ORDV9zw#S2dz;HyIum?`au%wHOHXLq6QJMT&(uuKXiOgAHb-THN5v&4C7HgDC zUKa6Sic3O$$61`Bl=SNY$D!eAC{r>mI!C3h4MG3HyN7^bm}a${Gk0X}%whV@IO73I>^N8efP)nkG8YxHS88Kz`?N>S(o`LcjotZY5VDIpd| ztyX2PG}vUGJo|(pe?PeWpTPX*nh<(A=D+{S`>*S5{?}wKq9CTEAor8Z|LaN-)<0J> z{a?X3^RIBu^xuJ+{$=SM-3!XUA~7Qa<6objko@{h_rjQeod1H_kSUUQ&lctLx**Q1xkRR4nQ zzb@YTyAl4&`i}o%`~OJpFEym3doc${%`BpUM&0fLa)ET{q-%2>@Ukb=$UE$7t1~T)gZ0Zr=7X3mz(dWj!kMBRXHQ8 zCsb9NDZZ(4KD1FsZlcZs=?ME+oB5iV0awtj>SnY@d--A!IjE%o;1N!e`i+Sfe{A~< zT9C;?KrpPFN{|gllxxg)o|M$jLcmb7cyd3=Ne&tv#PPg*^0c#fewV(pc))(WTq6;i z4?G|{^kz)Yf6qVQ1-XnK}GmtfbNMY+z}!&a&D3U`g|NpK{TVyY{lyyukx@ z!F;*uKAgSz9zwlrzREFKX$8IADNP}Db!3upbGE~BU_P^O3L9CvNsg>Irexh?tCN$p z_HL7Y#S~`&v-QETaN-uU$nD_hL~HXh@S5{HaIdxX-6=E+cenW)`%9cOTkM3VusiaV zW|Gzk=GB@bo<-s6I!@4<0 zMShIl_X_40BT|T)Whcf%ch?FAjG|WhE_I4MYrA|&SoYwdK>ASC^yP{4l>CmJ$_=Vj9J-LvBuF&}rGR~!jWIi5`pt2^c|{1M zd{B)#Xvu;?7h9RGgS_A&>Ue&K{KBkc{@7R_x7v)M5OHw`tYkhGBXK6_$acg?sypUh&Dr?=`$ZMvmz`NgI`OkWr5*cCULP#(=MU>985nGa`)<6S z_D{DSv!*XEzj>d=2MG0O?U$9}Ub2^X~_QZp)`2sK?hpSKdn6@0j2$-CfMhEL1pI+VOeo=ql^v*u*$hgM^ zmWdWS2c6gFnTQdEtLJhrnpscG(e zI@`sZ@YdQO3*NXqh#ieU^Qdi%!S&1&L84=Igpc+n7LKb^S=V!@(@Mt4PI>B7xPJF7JbcYEEpcJ%aMCx zD$1jdr^;?0$+m&MJz%E$KOYR@F$WP(?648Rx*mjP_AugHD9TFnK& z6E3kHkDf+B_e?sM+)cSRs$u%zjqNQ!K=aT5=tR9+?ijx9n7Ir8$(~S2fEyrrS5CX^ z)m9@={V~=3cYk+>@3Be^Ew{r|4=s25c5J)nf%s1%qGOk=OZJ4`QFlCT4rZ+w-CVL`@-iUdLh^rKz~l7GZ4fCZ zct=s;M&hW>jA=(%L}7HR3qNb9Vgc1zwWAxUNVnRf8@>{LL+KX;`FK@g*))DRqo#4@ zy2dTG2+-pnB=qn!KOZv+Ge?Lw1guoOIme69$$P-H5awr2zICqSWnmn2P$YTtU_IC~ zR7b=Vu+0{yx6p3KhWxgB!?cmSjNv6!!&r9dW2~hcTf*`#w2Sg+faoP_aIasg) zVTH4+GlqyO>bIf@3`y@fe}SnzE<_pb63+W&1Ygq#UpezI@pipqED;-QU{}w+-$~ zF&SqN9FC6&YZIAOkr7(xqKJshNQTOb7nvd1Pc&#cC2(`Kt9wzTp`}=8y;~VYJTN=u zh}zt{aJ#gTCk_k&*n_eDhGgSvgl07#SZU}74&7;WytsMj^-r&YP`uAcY!KE$O;!tW z-t*tGisU}M^FrdzCu6uN1}jDPHLQzX_U^C(YyvIl^l@6W8HO$}huhT10O#%H;VNBX zQh@!)iC!*C%Wjc#pTWf`rV%9m#N$z*uv=JD;@MBF7N6ewFud2iC03?B9<5PcZmgi- zZXu?%fQ*IgATfH(`E|(jTySPe|1OZ({MP8;0olI5zXm!MYcNA(tnqWmPRTRR@|2O3 z)Q5Ef7e{Ww_Rlzs+Ufux{{8~%w%7*O_`_#p@Zue0ktIA0T}ONetY%hDIQSLM-w=+C2Lu)pGX;K^zAVa=NmRK7l_S?LMdP?S<=w2!D$59 zX+6Vl$BHs$d96^U(va@0G7b9ahxJMimbE2&q>U{#BI(&#D8tjEL zFxcmbK{p4WL-3%;9$aeUo@rb!czqflD51#l57MP#nD3$!;zP)g^JOz#(WR#w{4MMg z!SukO-j$Dgxr5nt=83cdgP9?X&DKoCRo5FPBKKWRV1y=%#tz$-3L0N?Ag_zKSlXn^TK&CQ-=kxs|rC`?QI8|e9V*0hrBoW;5xCxA5buZ z=VkW398T(Ile%W(<%jNq2*-m3i&1?}i4`OGxP~FFsZ%;LsOdxdNE&VYox-@Ydu0LR zXi~Zj<-n3yaJ*aW`=V%EWwciz2609$Hlv^laJaN*!L+p(TcpvYip?z#F<^2VZM4Nj zzSrB>4vs=XWj}9pg@l$J4FQY2yL`Y65W#z^dV*t$`|nUfGkt>W#^WCIyeJ|?H<%^s z7tK>WUW!$|^ozK4yf!L=_t4|cLTG5iZ!l}c=|H4Omoa>?#>1nG`GM=88B2}ybA zK&pkCX)<-7O7NVN=IsrImJaq591>~N%2YbHiIo%b*(T0=-c`Llo{ zceGUkOC$4pKp1aq;QlcOz>H@v7vMGqA88~?rU@S(|BUGjnaK1kLOFtGC_vV&4jnKE z#}QiAeZXiXjvVizTHr(AAY!)|b(re|fR*?0Vi(nGHaL~-R=s@&r&vHvvT!n_oR_|J zw-h4i?&v&ZAEEjXtpdt`n}A(l3HC5;xS`arq40_u-V{--LkI5dM|SPgHSWAEU-z&h zzpe3Qj5VY7Y8s5Nztq$COk_ivHcedd7pB9 zJc14xfwKGtaJzSYF9XgBXkd+ic-{*He+>!)v?!)Ut6fR}$jdPNY5g|vc3xpT%=}nh_`&SXMyuFMG-|Gp$3~4l*E#yqFMG3}`r%4u2Ys zMh1-yU}T-4=;KWQ_y|!0>2;f8M)`A74h8u{={;Hg#1lwZ5&%c+ER0A|Nrz_VAj6v~zK6E}K!b_HpR42(4dk z-9C`}8Ea1DK(k}%D=m{0lu0_EI#7rUU>4OD%!#YdDqt#ezQL=X;B=~57fsfz-Y_ zBQ^9~1F-qM;uK{H#nz9s4fsk7QA1I52;da>DzNNd_fSX{-JfVM>=u~lYbCW$>erZz zKCNN^LE)D3GExoj})r@3xjhVTMNdC37ZJ`i)5>@Q(Cf4cXLM9R~-Qav+ilkh9v9^1?|YNjn5IOJr;w zcSS-v`_m|5D~(v!0U!Ol3|8N7^8wm&ZfuxGJmKFCPQ5hnPUv~|MDolU+!7Olj)I3z zH?(>SR)0DvN{(JPl$?PddOM#@vcd78byra$sj=C8V7|TZ(=XPS9Iai1kDTag;@q65 z>Xo(d@ND*0q>A=d03WgP4Z7=Reyk9;hyq&Y3z|q~vGB92Jz7u$P3yI}az?Hh*gJzs zA@Dy{BK&Gvq~FL6sy)EK0nX)x>>J#(psm16cV}=KsWzb~s6s`E6>4B+(}0TAB5s5w z(02IthQt2dLztW%<(%G+diMeBeA%yjQEKy|?~NZ^K&($&JW+N>WXK8A?+ya($Umhu z>>=|G?vlXXBlIpl3PT4%+yLzBa+unWATM&%#RWKostJG^pc1%$eJq{wv%(c{1DJCP zG3Kui>qA7ywrVUOHY8=BK^1V!<6NI^LVZCK(>jL)$%1HP`n52O>M)$T3p>iBXllj* zQ*U6{jBro(z;GBi-K*^L+|t1l9zU`D&imZ!>>jb|q$DwtBeJU*yuGHGyYW;=V!KDf z#!#Qqk8DwYu59j}#ovWU`&yz3VBstnLZM1 z#vMOo48gk?I+JNH1?HxODkqmil!VmVXEoqX1P@8RCckMrM2S2E-Claf{LXy}2$`Q< z*d?N#{A*dM9rux*eLEh^cJTxt`*A{5HofC&uh9}u=yd|XXr7^%+)9RB<;T#O=KM_D z@a9s>y)8Aif!BqOS8;% zv>IHWRKdH4$?Y+*c-+@|^vuq?dpzB~VaXfxON=YZz}-7!f)l@Oy-pVl_-=rr;&SQ7 z-WnXKi`(3AX%XX8b@#DR;8c(tBCnH?WQmGwv>IS+gvtrV(6J#u6h?l=y0m3R^jo_j zc%t-$_2f&7qx2ZxB`g|)Ab)ZUa!_VLE(T1`pXX%_7$oz;YVCuY?}Ll+jyp@+7Xp0f zRr|;v7B@;vC3TREK7sG|!NlUSiJCcwC~_-hG78@njhPj;re@5F*tW6^mgU_lPjMD@ z8MPjGBN2c*Aa0t<)H(Zr9|Wex2pF~_?76pypxgIJxfHXlLYdLSjLq-J@8x&3vgZE^ z|KOref8uX&KDg-!vr=M13uv3zJ(#o)lZ%PQka1{h;EHBsJ9Q<9GEp`wmZ0m()vAzO z3);X1Rv}stFam7K6zz2Xy5by5>OrfHWcHLk>sDyz^}plvaqp+mDCtY;<=!Ltt(X4Y z(iVSFZN0w8$V^~EwcBC0izIxCC3s0#Iy|4hqrSybAY6H~tBs>M&-oSO>^;Fs(onRA zhvRC^c%HBBaQbAN1l38_7xuLJgXDs+#-fVh@YStT+P8fxZO{g2`U)9P@If7TNMS5~ zqH5keC*di1f2xogVg#1l1~7*>G&Yw}Z(;$1afckRxK0=#`i5A6tAs>~m}xCKh4Esp z8v&iAdq2q7-m2WpjQ^xBiC=?u1Q*iXJ13ofRki_O%K zv}5jK963h9ft>EqT_ck14aWtqr-vfxE{=7X`SVd5I>fW`{>qMfee*f){r&1A*8+Q` zL+z6l*2T0s^)7JJ8FdL0k5@-a?*r^svI6W3%L_Dh;g$;yU(zAOuaM7xT@dV=0<4D^ zHD5|AU0PPrOZll~9ETQBPQet0%##16`Zqg=81v*#9p$$JbRCOkw`u(7cmgBHh+jAJ zqR*x;Vd?K%(E2&K;Y>lRWfQg4Zb8U*yaBg`@GJD*^xw9?+~M@%_x;hY;5ec{cl@hi z^y9&3u}QnbQpfRt`Rl3LdEnm(Y;(eIrHO3HG!8^a1i`2o2Xw8sXg&tQJmLWfY&Kt> z%d8*(Tu!9n<$5s`d!KWQ>Rl5Fij949Yl7kXz_oE67rpIuO`MNbq90OikC$$2?IHKJ zS?W7pmR=?uqu=0Y*I8GPBcxVy_1+`5xtAP_O;M~9Ze}zLr7IQQO6l}S0@G54dWm3e zrL<)MvjD?Os2s|fMOpOh0fx66Z~)`fDS)(PukhdFeB3OAK{!m%5vXC7(L&Ck4S(x@ z=sYk0sGoiVu)Ns=;8m#zMA2!plstU#t2dHmGWivmdND=MZJc0-C}{c29*S4J!Y*C+ zfcl8+=hOwN2jAX$7s*pCnP#C8!09LJ$NN)VkJ)?(z=qp+q8hAf){mpdXR|LA$YEnt z6;$CV4lw5x)Q)i{kn*|C`}&mErh=M&&x~Q`>yIRz*fA6k9;5CjfcuMdd=ut0517)J z41_TF&x}6j-W<0#e z{vwxIglfhw(y3}RVvE)pDJOpTbn(NfGk14Y{jGP&k7yGUU=tGzK$ZF2ke^Y_buAs$ zlQU-N=jP?4R!y%OBG=WxOJ%lFZefAo8r0d5 zD1o0K039a%XWE+cGC{rc8ohk&8>MP1*d~PctYK>0ucnfjS?#=?Gi1tt_hBV zOi6Lwv>eSI1wh8}k(t8?z|)NxN}&i!Z+THMK>G#(l@8cQ2$Y8F)GsL7qK*Uze_sI1 ziB7a>F8hf9E?iRY|(R?){!C@7?^c=2$}#3~tkK@= z?BGmSpIxcBn!li0LxWRRHCkeBrg1B4AxNybve9)mX7msymS0y>PNc}4XJhNk<50xC zicMkT2hIxx{yE5YE$kD0am|OfaBTLwC4CtKNa8VRH;ZK_><=MClb)5|J+95OLzZM& zkrrSo%0&ssg3IKUf$PTrjzO>6@>2x4XDkM>I;-#DM0ICfo$Xbq6&{!`fel08a@?3l*+)4f>J>C1yg`&s(EPwf9#?8k$j&cC&o{&% zu3He7wo2%_!V_+oe_N7qy7W4+6L)qc*`Ds4%0_1{u6b1YN!81qE3t4mS?xR@JJa4_ z;@!i=ag|Z50)8j_()rBYoBA2oi|m$U4rH#3nIZ=qnNtd-cA?XZ=;BNB8-5%Ndo6mb z5p^&O-^gB~dadq}2rFbgqy}s2wVvxxr}N^l(`K_n^@EQt*(&G%RJH$868#hDW@BPw z_*a~s;ZLdgf9Jgao~QqkJbp!~|6O?(H~RAUnLD}Di2W;|ZKVH|tNzMXf2kN5IR4bv z6||w~{w;(2uA-zQBuXV_?x1gKZe#l=gD?7>MwRuy<<0->_J4aEg5pAdp5sqZ`|EK2 zpGSwH``Sa%-SNvNC~jk7i~mOk_?0g(|5{|=j!!P|)tTP-%Ty}sU}Ws@M?Fa{`kmqH zzM`|8ot3fmA5-BUhwWE#;41|Fm!|T|4*T`{AKmI-<>0e~{%Ey1n6gj?v^h`j{LqU!QLp{+f$^J^U>GVl7o$#;-SQAd zh267+pVhDgUy0qcWM`4A%a0!bmr1RIt|nS_Sc0wA)4W@vT3S*SY<6v(Jo=+?yZ1(Vp(^ zFEB5>N1B(G;FTrPGQKt*tM@5n%RktW=y`V1{Wf(JJ0E@FYRR6ZAJ+(0c5n_IhibRr z{nwxQN_alpj`zTWN8gY57~aHl-AhSDWFarJGgf>ij<$xpfAP7S1t5S=?tObCZDVf{aaC<`f*(knt*G)aZ*Jg(8+rWxxfk2x57 zkpjWcEAlAsjq+ zE+z~Sv*Z`&G+k8`hi77A!H`4D6H(gac{k4zzQ!EQDA`$*e0a$EkX7!m@U$zocSHC{ z!S0A?EWuL2@~c@5D%)HvmEV>;>l+clJ(vgCo*ANVcsoaFxj=zz0@E%rC15ZtxK_kG zge>wRh2(9ea$E0fDV>eiD^i!6G=OT%pfmh#iG6NfeRyRhgg4u!iwX?O3cwM)v)Ed! z($5P~3z?MFES#_k&0qWliK-$oEx8=FjP@dloEy>T5we(Q)h}@QZQKG{ z9jAQ0k^10D{9XOpvv0`B;6zp*m}<+HL}H<|jM7RX_9xlg3Csi!@o7bCl4^=K??6Ke zBQZL{LD?_F8s1H0l%4Xx&UHM8z;MCkf|hcc z2|_y~ z65qv&w$s4Ma``jIzP9Aio7(1}}ta2dV9|f}!o#;p!xww>rb?$a3yG z6#%W<{a|b8=+rjlm8rMptKSDLc_Yb~rC{FZB}Skjh~i*F^e_3T@l}q>py|a13B+)J z&O{btc@dgaDm1#mBq`Kv8U~GvtjM65zzBmBib5bBz!9KFyA*ct?hxT?J9&9W21iSz zW#KJw&((kcP(UCBOE1wO3y>g_K3DD+{edY35#RDkoAF5-$1(;5a+XyqO-p%~vKKvmqOVC{SXl`p<*H(tGAv2JtgcE?hU)-{YoU4u4FZ#CL}{?n z3BLoSOj@J|A{GS$4rN!<0X)x&g4}OKboKAEj3GMY%{d%8QKB<-T6sNMn^C#(2#yw^ zSH!oLRg6uoN>ZD6hF2nso~U(f<&916hVvW5qf$kOv(MWYTa4>!wbW9qgtkvaM$w>p zAxBP9Eufv+vu{VYQ(~+QlnNeQnl^|nD$@^fO#MI%3M3>cg+v2?wef374vHA3HxiFx zMOzja5cw%T^HYY>ZVha0WPv5p*6@c`_Bbhn*`gG3xE4M^CX=yT2@EZlFn5Y=9Y9naQ0&R1w#x0RJ>voqj7qvQ+X0i4iag`QbxTNGcC6M-^S-l#?IDLXs72r91nLr(v%+ zE(m)+agMrrPHAMzik2nUl4sYZfpv|x`y(9&%Fxmi`{Gi?%jKhGqf{cEUwxlsXrQ}lA%pa?{G{c!XZq5a5noHQUvdZ~9Y;Kn3a;)FcE@w%^f6f+{#2HC=w3iM8L;(3r4m$C zY^CSq3lt{+ZpHT{2l=5@-7Yf#yplD1Kt5!K>4BYid!HoJ?U#=8phf16Nh~TfrZRHt z)VodaW>M69aF6D;gQzjNkBcl>4^-ku8}|+dlPslNyut+egybO&b_X|3rQt5>u4->+ z-dR7T72btID5BnW#vcSE9jo49n9MaFzbm(fHA&6#&>VR|5IDIg9zH(nkWRVB9|l#M zntHG{;Dg@guscX6WA)~>en4*NYaQ8m@Q|ZTCP=64wEzYzvBSuH{lljutr_FGkG5$< z>iB0=7K;7kFS=5l8BaM4Q|3&!igbR_um(;;d-sn!R{5TDvCDjHFRZY6yAmY}Kobj# zeGM@#xyyMR6(Qf8H8ne07H;>#!{V-#wuvbyP~^7WGak}p z3~7>NmDW2ibz5UTUAE{p0y>>qf~Ov#^V9=M#s%AUHP$TyTbgq6`W>^>Bj#B|^}@RD z55r7yOX6)3p1?sFqIjen1gh9q9ZCxf>SibHk&orgz(Ci^5y?x4ooUh5%l^%;KJ|By z5kVh0EJ6q_{xoc^pKL}tzg@?o@_6%9kKxQ6WT#MJTUnPlEO_|_8 zsEKw0PC2|azd6lnHPTPm7c2L&wz4%C_A!0`(FpvZo$KPI^dW$7-`^{U)@NcmJ!6Pt zT+JFjtiGUe%deB6je1d{%TR=3aRVF*hpA+gFl|(<_aR=jIgP%Nl%G@I61MA-lS$Kc z#6<3p4&nfBhEryV(G%{)DZEPXw&D6ab_PcA*}d#Z>!N+$v+|%m z>dLcy?@NW=Im&a~1)JNw`PoNy%-7n-J3~uIn0jy$J#S|oZ{S3MvHl~F$X_E2rMbNw z!Qq!N{Vx+I>jm->Susx4{*2n#8aLy4?hbQ`04oCUfxTJ;}6 z31$Fk)YOyd(kRrX)Pwr4=*KUL(b`hdt!^M^hgPIM+2>-M30YmR1;tt70_k&YQy})_ zDPv}^=7@&cY;0oR;n2TQflh$amFi_45u#}NhcO~H&okwN+|KAR`wdmQ-*{~u5N7w% z(PE{i0}+H;O>>afcd0V7mV^5bdbD%g8Lj3***L>aLy`SBjXagnMa!qu}nKSTbM#9#U2l$g*R4WwpR&Dk7ajAZ`fw5RFx`(2Q%9P|B9q zlFNuG+|!$Qo-9A2XX^DN6GI192_oF(3X^LqR!(1p#N+6&q;0z~lX7mu=h6N3JF~HE zV@2zJicFclt-Nu{NtIc&kh_5wFLb_M=P12!w8KNSSrkUtr};*N-HUaO*UH?Em8P-U z(e~`pk&ROXEEOg^!RRHpvjE9Wu;_9aSacTFEmiFsTEV7Mo_EWbyvh)Za+d4kf|L3U zHg%57VyOf^02rC#cq@GK3Pa8Gc^p1W+I&=o5JOiv$zg!2V(I(3)zrC*7#~2wo7wW34=%g&wLyXEQ=I(eS=ZO)SmoI;H!X_?A!f)5vuc;s*B+>ynZWl#urr z_i;OFY}RHad1$uBiRL{SfN2NnQ<|vcewF|c_ug>T7^6Fw$TIi@cqq3hM;OGkmuAs8 z9v82z1>L!onryxyzRc|r5Q5Mx_y)Qw$BgpHoht_BZ7^sls~x_BAG^?n^v7^8m?(Y9 zdv$;wFCgB04C_o1ecyi0JRwQ@`sfztz5K4jz4p@8+Eg`QN>1*K)6Eo1O>3pIR=wZ2 zUaVAYCevuDS&`I*_vUbeuFdQMaABn2EC!JQ=VaS_?LjxN16|q_x+8%BO`{e5A(Uta z3KK9Xz_tz1`uimxnFUAiwu6UblqrxMv6qjP$ck>DrKinB(mP7nJpBB;!uFEgA1kHa zsRG^EP~Ry*hH_U~Wm)(M7tJ%IKE^hS=pfpdR#9A-=kn@+=G!-D?j$=}t`A?Iy89nl z4Cp(Mzn@lO0mp`>@7ITb#+iHD@4l--{%AqnZE4;r_$u|G#lKM*6>G4F8$KF@Cuj z{wHnv*Gi?oakzgx?0=u*f8cQc;k5pOQD2Qv{s)I+{_2Cm$o$oog#N2%6vJ0Lr~k;| zIKD{f-!L2l!{0F+`$2E+ZKu>TFi{m(@+jDNX5 z{xt{xh2j3WXy*SbhKu%aS5jQ=;Umv-RFa^vvy!q3XjxlQv2CtyvA~hDM+F51Oh2Mv zy5lD0;J`o^^aCIO))xW%+Sb3i)Y4j*t$TIhw0+jinpl;GA(xdy+(V~v#>n|~8I>y7G<(finpAvckTAlM3g@wtSPF+vytC-E&3y;k^ zH6b82|Cu9uo{xec1_rGe$9j7%p4eNLo>O-f?C$wHF^7iI+7C}c*==EjYB1G6f<*g zJu6O6n9f|QpEPXM`L3rHnC53T96cwj3j+%S2P-~fQwJl>P;jO>irD31GL!==dtF~; zkKWq21ZrPKIoVPlWH0Ow27!O}Q+cl&0zoXjgL`3iM^| zsY+7uN*>=O!&h;QVOcYZQyme|?%o-zs<&*WRVdKb+Ss&}Fr+LmHIE;hW)wa|4g5S$ z8v_Qp9|;I<${6Jcsa(@QVN=t?LbXpqJ8=BQe#eLGoq;?oJ03ORLGx-yIW&(oSJ7Az z?AmOd?A`4808c$AQqWr7SiA|cr1O!&8FFl_5OGf`QT09@<*2xsJr1LWy|t;S^}4cYzDoW!kUQ z1jl~{5nVXZ;oDU8P<*m8ONm9`NP(R45?+m=*o*M{R2jgr(_=K-)5+~=!`YBGaJ0uH z%Xb~pRPpp%#P+5jakMc>VySgr?kdLScE9W;&tBv#L;uz-RV67$u*vl^%=_1v3R;@J zpbBmjSxpeO1G3;lq~Q8a;-9&otEtM1-(_Uli#mcK5&6lstn-RIVybc~I;TOfZd^KK z%nc+H=E{*#l?=nnn+T)l-4(YN7bMj(=_xi+;9C!HiE5BhQA6sVAt!FtWTN@ym2$+! z*3Kw+D!XG;%-BtuYg7!aFcyHXpuUOgmYyO12qGp?N1Y5XNE02%wnIK2((VsT@0|&_ zGqc#z0hyumS4Hj1KUl==t+uMGA^40Lar_98S{jJYAou1&igWAjl3$!oxqe|gXOLRCd}ENVE}0KlBrU2!|kDxZsEuJ*0q7X z@hWU?TZbY2BAT@BpdU6cuiE19i#$_J8rNH{KU&X5DK(Wd94;Vv3aNR?@INHTwLE=} z&TN^(4{i^G!Z8FxjT7R69v1yeyO49FG_ zsKpK3ix_5hY;>l}XHkkpMc?rHF#nNNG*;Ac-evZu=M0ZC=z70a<3CTCKYKxJByD9C zP}9f~_05}9p9;?JEqPi`^5{KK)X{u9-!tb-KJ@_JPy*NOZFgQ%tF5kCMt7D(YHk*? z)U>n|=FsPglMaBdqXHCQ2o;ps7OT_BA=~-Ay-jRr%*`C>N?MKb+m$|C*ra^~#Mlkb zS{`#nA>S{;V0ED0jG-FYAqgl0Z&>S?)ZYvmSwHVc6|uY>1e z^AMB%3Zlfe8f;#B|K@CU2WN*_eoIeZcX@ecAwc`PvrIcb6&LAVsu0Qx+5uEVIHG;o zJ~y<-HB|@4_D*aHCRWI(KlI8qq}tQBu8oya>RNNowGpvRXIz8M{K%(ifNkJW2>RG3 zN{vO-N3JwZh4_r2Am7neyc@}T7)B$dqh-imyk494J8S}V4oRSpPfz)6kEa>NmMyMD zST?C|Kf!qk@3yET5Z#M&f9#QFX-64S2bjWsSwC7KDmkLhxm0JEUYu~vAy}`}E{(MS z*9CoU!73qDseIQE&Rbg&R1r}q3679!nw%&_)OK*racVyhuG_;#swTcy1Dt1kI=oKy zBI>cZ_>$T2Ks@SHur4gH9q**zwwcTx0Rqw*5M?u#r#phDLL?diQm<=N^|Y4=EPGVN z$dAoPqL3i`#Hbs)xy--R?d6^yal3QwIW~fbAD$#1*Bh5G??_Vg)Tn(Hf>fzw1!_U; zQfSxOD$6^7jR?tF+lUFu*B{x6t*H;>pZyAH8}2d5$XM zZS1O(fLLngq4(vS+bpW?V3MNFB6XZ|H{S(1_Cn+o9b6WI#W?+iNz1+t%{l|vYPyHEeAS0=#jOz`#_YNl|dP}Y3g z#0tM_{iQx*yOzobZN#oSbPaKiI7CcT*<{Ul4TA6WkLzXBCiVui6F3H>lwDP$8${eU z9w~zq=PCN?%IimR$0iOZJ3j=zoS#RHLNGFW)zL{!;f+y}jf`><55TqthZX)E=_zI6 zBa3Z6#O)kCtWAVvj86rtWj1xh)m+!b5NiTu-0&x?AA?n!G0dB;Wv)5gAns)Im4Q5Y z&ny&S>Z`Eo34*G$v)IWP#QykNP(a|Uc$Tl7B^+4Xht5cqb%JwwB(V|p~^3D?9izcNp2iWW|_>=+sKGH+%e7Q+fU-w0&&_s zs4@r@iayvC{!$nkunbrI0O~BzSRgPrgif0x;)|@#vP3FD#TNs-rTm|jQs+cz5_#Vo z(&fL661n@rNG{swMn$$L?*A%YDuS%N0Sws&&L1}ovFq+I`Y`DL&NqGjWpQS2$&hZR zgN}!iQj$C=NbLy!in?Z_%#vccfHK|M1k z;XF4($9mvaMyst{a3HH|Yt!bL>BsY|tUiRNxj}rEp&yX>T*{j<0QqTY7zHBf16x~O z+1l{Bga%><6kLRiOR+ZT$qUuvWrF3jra$&`-;$}%gMR~5jc>*BwfU6`#Vr%yh#^fG z0Cd$3*wfWVTpxgnmS6i|rW}^bFW>K9E!9&Q;@EDOlrugH{bDaZ$QG zV~{P5nqJoT;Ul*_O2=4(C}W~N%m$_kirI6n)&4NeX;$J#gql>;+8+QX%sVr0(w*Lo zAV8=*ZI@bdjkm1o%jH<}(m3;s5x@>W^MPSb)b#Vyj!EA&ypqiochSn!!u`CTsJ_`9 z!jZC!b%dlXIdp}xQb?T>Mo5AxfN(p7g8<5b0M|*%lo1HSK4G32pS>A*>o%*;Yu6}R zCLe4y!>C*6TiHmRFXizZ$+96IH&`^1UTTDNULt)pNZ5%?a$Wt6I5y=;LKP%7F-a9+ zJRDrXa>T*%T{$H{GKogrh^HzTZqp8_^k`3`ud2aguKQM^I3}*-?puIOb2jnECxZRa zIk;*9hT~{$cD6rlkmdE9sH7hkY!@oFz)Ha4)uY8XFm$6H;>x&Of{BQTPE2m>F*jES z{fVK9g(OsEJCDHw2W_9&GO8;P4Ed!`;Z<&EE5I12huExJxZsYS(6MdcaD`_e=#E}( zo$S8wDwN}2Z_q5prwI-~Lc4-Z;)^3G5KQI7DSW8W2^JMR<(Q}P3f++cQxk-EIJUbp zR~UFES)XEhHyyH7ZG%VS2wn-Sl%5x!eCepUreK3uZb}K?vqx?QlwWB_c!GO;E%iaF!MR)7Wa`6^mcBs@(*XO?s^S?HfOv-eLW)YDgf9{PtLH~h z!5#uCAX~P7OzlO*%+kKuFfiML_2KT;!J+(zXV8MK3wsJ=;f-ShvN+Tmf{IB#Cu;hR zYZu@3T5-<2p>*UXdj$anTr(LRuPTrPn3vjaSgnY`6U2CEURe&1m*3pbYR>BFq1QDwEvGBrRAF1O+c%HWPm0ztx`KjHo9L$UH+#bv9 zCa+qba*axNO#0m}=L7+K7489$L(c%4MOMa=?JlSml_A+h1*-O~fiEj-aG-6L_X$K{ zy|9F{7lYMgbofkJpdS3o8nuy$b58zNcVBTnr4v?-TynWYais#C`D3c?GJ* zBKwjQN+O{Q*0|4^1RGsctFKKH1DN)@+Rd*Yyg(zruF&uLeXdW%luP3`t!Ae!`YM`k zn<(YI8Jx)PXJkki9SkP%rvoeUp&uo_C3gQjsuZ3%&tzF>WgQQ4`Dr&6*D34A;j zHJhKUK#!pemf;<0VtDy^M>1x>hPsH#T;dC|bPu2k+s5xF*M`e!@D8Iv@ZCv0=V&G< z^WwWOrO`G<|3)Kj;K`0>T9LMi58(lrS>oJOO1_(vi2zzs==Km8tWx?a9G90r5BsA5 z8i2reAh5lr9GgO0gxtPdae7$zKJ)XwuT}Isp4|_iV|CWU$1d#^fhRiMxU3`b`Dz9k zIN}IKn3^c0OZ58&wqz(0NEdEfa^=-#s*#*JJ?&-_RuPko`bEJx8BR1XaRc$I6W!SQv{WhCsC+_6=W(oTJ0d& zzS_!w9x*5D$FJHCO|BT%01YD+2v>u7#6JM+y_%F< z6CxEJ+zN?rf`$~VR+X6o5_n;z82UpLTo4Ov>bFQv-)gHRNy(vFFGp`SECC*};bY)g zC5zXrQV2+J9s~xUaKkF zUaB@)0dCjY2#$gflo0kIGB^XH<~L#^REl?6z$M*B?&o>2ZM<_a1=Zy-8>=C=KHPq9 z^r^XID!A#L=+~$l(z+Qck$YuWQHPk@L7-N>a<5#bwoCgRi@|0&HP>zo-}6?xgnp)3 zWQ|2jnM_PtL`6wgXE{`XjglMp*enm0()lny@(PZ94OLUDQKr$-q)A(vfoWI+9^f-!tpM!(%lNJ_V_2Dn^H-G-9QMoDGzAVT6hX0A8^g*QMWQ-M*_nNnB| zTw~Q}eOC5#i?teh1a1$G#gvGUQT2?z`p++v<`HCc8ZpK)DqNi&_ zzvts1`ntz7w5(jDDi^2cs%502MBCdWCNdsroL!2JLJ6u}tuRL^+@2tpROm`wXJb`< zac4j9urIk%V9nHv$=s;mw zftq^>&PWu&-;a}^ff-ojd%k-suxCQMjpQ=hPt;l zPedlQP_(G0Q36~{r5UP{biikt%ezOu&&k=#RSICUR%_duVfNs}ZxmMd-_1WdcN{|@ z>mY3lv6R4C*h&y*($v>X!sdK^#B~fnbJ&k-J`9l zQN!dr+=)yxrXX}e7cCQOY7>gQv%_ZAEn)lFkR}tY_uI@n`Tk0BL6<>vm_a1C`+Zh! zpal|_1drPc@7M#~NT1k~EF`y!zlwXB?1K-rCA$MBVMwiQ1%LC?v~tVTk3=?`z0*xZu-iew6U^XqW)iD=4YYG%)KRMam_E5(+*B5xxh(W6E#CQ%dJY1o*= za*R7p$UVv={nUVirRAhE_-VB+2*l(RVltK8P@Pn5MO2EgfE91j;^^Fl7wN|Mx+j}} z5)j2NCWuFk7|oYn_-lSZj`SUe&twXCCa~|+;+x9Mp(=8PJLhS(L2iH(VPTwRN2D|9 z7kTZYkRM;*ELnuj8ns2I03lN#GQKqJ$kDvc1hkz32*s$gcx0H{-bWPE0f{21`$!qh z7k_#eKjWg(F}nYln=sUU4nt2wd+37mZR)Mh!`8qtJ?gn|`O)d{t<%45(m)iJcjgj9 zNbjp5l}PPOHUqsuY$VbL-M4G$%=jQF?q;O{6v$*02A4QG6-P+(TEOtfw~klgmonw@LkMV!nD<8afc9REsPwe z@%=Zxx7J@8+S|PeAG(rwF(vB`Tp1WS@cKxefAK%Lr#j zqVwoskoZeN!a|n}-pHVTtmtWl#wn`=?(-;QLn0$9s`$QAeNdebU=KkthWTTb>41TJ z&OSlX5|I)cRS3Z>!gJplj_wt48)dn09_5ll|JrUW*rkj!R7m2Rg6I5!!;V*S8KbH> z7L1+rQGX5-(&Swt-o0;rjl8NV+c!_f@oOANfg`UOnv$9&m1`SE3j| zCus7N$$bHDbs$Ez1csdRIjgQ{prz2O6!zohJwQke2UrTIonRhQ-XWJ0`XqM5T8$?D zPN+sR>>7pg@!*T}ydbwI0SnYha4D|k>nD`Wmu8@Ww26{MQ)Ry*F}PND)e2Uo3*@^n zq9a&U$Vk@)HgOuC0>3$q*i9`0tG;h@ucv8dR!YCU>U>_`F9*hFZ1hxm&qw*4M2)yf z&b)=poc2>HTfP__ga$E7iv`z7`53W!(Kwqt6~eG`4T> zs^E8luCnbk?AT_fOkK`^Kf?&#`W@$t%9~6}PI*A|`J-P&C}1}7sh$zoSVSPpZwB9q zslFJ4n{q`bA?#pQf>?8inJxe_@)|+{U)pfHAHWHyaEu#67*UxE@{}~dbkom#$#Aaw ziLE!;bez$L(}Na0mPLx9(vr+#_G57k$r)yRH42!!_Bq@B2x$83u2RzJL_5%#CS;cM zc<%M_K6mt#d)Sp42|lVe)-K8EqkBD*=2js;=t(g&Uk^S^;#i7YQx=(00qS4$&R#_eT z)=k(*R)&}LS}c6cQ?#!+og<17{l~QyrE12g2y=1D+!6!Y4OF;I`=IT@zBnG=Dd@~ApLs|%b1f;IYzv`ffoLI$1e3C6GRsmCCBpbf=@@}6>r*?zY0xko%8p_?y>z185o@dZWRr7|JV zAq{VhaWo@C9`R~6dt*~dJPnEk*P2d9j9bJfF8d*v)Jd{bpxDMh*~o+M*?p4(#l{<6 zAHpJ*$y&UcweqZ~ttu{rC)X?dH1t4B@TFk;;^Y#v0_*oze7{O)z_t;t+Pho5 zwB$Dzp5KSk?YH6aVy57?x^(1aCKoLRcs}Ybno#^-M8%H1jar|118mCl*G)WM#MRHj zueL?fJ<~3q;A)5b0^JtwFm_X0VqV#9o#TbHy@b86nF#6lw9;zipjVX5E3iRx<4!F1 zovZv&jfH_rr8SoshR-Qaeq8C9!kaXKPQ!bfsGs{(Emc%5DBFL)mpgBSXuueCwX~1 z5sRF3;8bWB11k|9f5d45IsWnofLGlO98~+_SRx|v_`6zf?I0}?zeJw68m&|B*K-ev%$yx{5eEPig@gQ9R z;Z}lwe1{9iwB?86?F3|bEC``eYrlG4*FUpwpeE6B@i%VNM;k;rC7#w4>xQQ)QCGog zSsFJ*c%g&Oow(rBaynO=tY*S`HV?%uxd zKc8=Z=PqmSwbtHit-bg8_S!4MuQ5hL#qGDx!`z3}1{C+pQOy1|!DH0nr&k(IClw#d z**Y*@LHC*Se5<(rJ{NMwybCGYx2xv$^P^F1XCoIRoeJ@eF)UC~4}B{AXNcY;lh~zB zVW;!$4sF@ekt)fV_s&18%d7qLh`@0h)CcX%Q#iPMpTn>t+U~QHwYBE(Q@5Jc&zl`w z8#~bQQUorz@VcRk&`rz@unsdSO|0RFh#_iTk2GK3DGl zmYFp1uM2nFKK#1U>g1%VLqEVGKoF5Ov)@ut1M@a?`rTO}iBz{|UH@fkJgpZ?*TrVh zNy{cUK0okloOJm5^xR54-;mLkM}Aouw0ucXgQ~PhMO zZ2P6{lK-D=p$q41%PBSuDM)zQJ@3TyR=3Ozm1;T9c)fnR_8G+_#&+Gj99N?~%u4cs z?Y^6)cIe^2rPEw;oAzgqJeYCucKFDIaHcV*yZuU)ijj>)=NY9yy$N0!7l!ZH#LhqV zGUvp?GdY@Ng>E5(j9fbDF!zRSpYnIx@{Rs~3sWyV3zB#2Q#GMlaRG{Z zPv1&tb{f!n@3mda`Hrg%4_ngzxoqj3Q~5$_VjDiWJ@WAs?~KYBE7ljC&+oTpVDjJ{ z*@N93c5Y6JY5KV(VbR6sQ5i$3orMO+l2SK2rmLtut@B{cYUX>-%&EIbb7Sv+X0s@R ze`4olxrA80ytlHCyXMPXH<}hLG%9aYTfEl*dgj2Hgg`1RPGxYA$6y@mfW}V~S#G*yXHZDz@77VK#qj7!3ZU1uNfB~NrLe=KHP#t&T zuGjV|<#OKCe6@AtlDxPu({t8`jB{)+)J&}tKA8Wie+!8>Q`hn zwy?vPPX~m>b61;g|IXyu+7!#u(wZfTi4a_DrhE9^)lGJ@JD7`4S57es(A4149ygnc zxVpN446X5xqVwt$)qD<3S~#|AT>k*eJ(+`liAaC&Chq7}sfOvGg72SbOI)RgZ$$P} zuQGbzn*WGiRa|Yg!R~Un%0=bTZ-*W#dgW`s>!z1UtF4!rpZaO59WN7YD|I$SjySJ8 z#G~}+-Jk&LKL6?XFjbKFzVdx@%8a07_1_0=zEj`rWX0Dew4F)so@3j0uj;u@i*WVI z{H*G-8%%j>-=ht`Wz5o7d6YY(?5~kAEwR^w-*mgIf7oo7wfFOgPQRl04+8gpC>Rxy zx^A%E#r@RK&o|u5n_2_TW}P!HG063HT*eG>POoOFw&`h|JFoxB+||YYsIH~qAAg_t z{)}6I=7b8vQ;tgc=EesdYu{XQ5VwuhzWYH-d(yn5_}Y*9$0PC%gsRicNn<7t)%`g1 zP5*P3ncDcr8D4MdtKRuUOnWdmFJ?;h6FoD_krhd_s>6|v8G*){74}h2*A{mKSJZ4b zx8dGh?c8VY&99vKaIVhT$O9844e396Ufb1ibDI9pw6$kO>y2}>{%LB_Pb2s4o;-J1 z>+%Jx^9rMY>3#2L#05Pl+*!KP(z2~FEH}gW;h%P&?|A+ZV{gAo4=34z|NknwrI&-Ohe!_I&ErVCh_%@1|z_iFz_z`|KK2V2v&v*<7FYx z$|g3!K0N_F4&APk`1-*m@H9m!gEaqp;7)m#VRig;WH7iloo0a^dUS%mz&H*rqX$lV z!jMc5pe+XjJdjZyh7f2ta*Up@?V-9P9! z4eGKeKwTaW?vuxO0X)c4NZ`>dNkev{Kh~arCVA#*tp9+97$0Bg7a<4kpl5pk|c zqFe+R9!=qOITq68VR!)JvHqhtA&ZYWz(nYnwj2$Pk!3Z&U?P$R3=u(tMH%<-_C#>P z8iNVZfolxLFeq~ZdcNv71-zgP4)QRefW!I<Ld>g01^wxj) z7y*Ca&+)npjq4=EvEaOLJWyRiUS2H1<>e_Fw*?qnu?VwEaE>U`guEPpN9YjG2=G3D zju+s405GU5-yTPSm&5o27!Swj<>*3Pp2y?`7}?LfkR^187xF0c2712g0YicPVtNHI zbo&L4M>8Z$$YT8hJTv&XA=z+$O7TK5?sF&}RFaTa2r4Mgi%={CuP6_L_GpwFrKfBF zV{tu2vY|wvJ@LT9QjpL?4hD4vgf9UaJcb8-&*Jum#t5IIKsGU9gFFYd<>^9S32+(! zqv6^Sj6YUL^2~BLU65CRLLB7fWoQa(5A`WNzad^&+>ZEO?2TuIG}fnZ4+|wE#si!V z0fS^SBuqqMJX1U)f>nT=9B7PC5?OiKXc$fc#?qK=!FX`E9RW;)(xJ+?#|TlPSPaI& z^)7NS;DyI+0bpW|;F)9aIT`dtz+rTu?|GD=rzc&2iAmi6zlQl|juKJ092`T_9A+~B zBV*3qy%nd_15}Q63_BJa9vpP6B`60WmyQ#1Xd7fN#R= z5~g1fuFudOEZ4F2pe}CH9K{p!Ilvg)reQop=H()IBxd$vkj8a$2;lUW;Ya}EazTQB3Mfd}-55kwgHUOW`Vcfu2F}lDD zp&K9vzOO(VGGadi`b_u;fD!&crW_bu*uN3_%#rIW*uRnX#=#N}A0rO(bK&CwFnrAb z?Ez(i7ZzWG03MI86A_FU4~~Jf*m!#|=D1G69+4O~o@NLi!PCUPf&;4f8lB@|PQh)J zN4xJ=;nY7X@)e6IZ*XRer;m@fFE5y`{QptD + + + 2.0 + + + DOLPHIN + common-services/DOLPHIN/1.2.1 + + + \ No newline at end of file From 5ecd3b30b923dc4dfa6ac846b6ac44bfb2f8e11f Mon Sep 17 00:00:00 2001 From: "DK.Pino" Date: Sun, 23 Feb 2020 16:59:24 +0800 Subject: [PATCH 027/221] fix bug #1996 (#1998) --- install.sh | 25 ++++++++++++++----------- 1 file changed, 14 insertions(+), 11 deletions(-) diff --git a/install.sh b/install.sh index 354cdd2be3..8735543c47 100644 --- a/install.sh +++ b/install.sh @@ -422,17 +422,8 @@ fi done -# 3,stop server -echo "3,stop server" -sh ${workDir}/script/stop-all.sh - -# 4,delete zk node -echo "4,delete zk node" - -sh ${workDir}/script/remove-zk-node.sh $zkRoot - -# 5,scp resources -echo "5,scp resources" +# 3,scp resources +echo "3,scp resources" sh ${workDir}/script/scp-hosts.sh if [ $? -eq 0 ] then @@ -442,6 +433,18 @@ else exit -1 fi + +# 4,stop server +echo "4,stop server" +sh ${workDir}/script/stop-all.sh + + +# 5,delete zk node +echo "5,delete zk node" + +sh ${workDir}/script/remove-zk-node.sh $zkRoot + + # 6,startup echo "6,startup" sh ${workDir}/script/start-all.sh \ No newline at end of file From 3d6eee5d8065fc6ac7ef73b01a5815ec4fb901ff Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=AD=99=E6=9C=9D=E5=92=8C?= <649790970@qq.com> Date: Mon, 24 Feb 2020 10:30:17 +0800 Subject: [PATCH 028/221] #203 add sqoop task (#1974) * add sqoop task * add sqoop task * add sqoop task test * add sqoop task test * add sqoop task test * modify add sqoop task test * modify sqoop task test * modify sqoop task test * modify sqoop task test * modify sqoop task test 2 * modify sqoop task test 3 * modify sqoop task test 3 * modify sqoop task test 3 * modify sqoop task test 4 --- .../common/enums/QueryType.java | 33 + .../common/enums/TaskType.java | 4 +- .../common/task/sqoop/SqoopParameters.java | 117 +++ .../sqoop/sources/SourceHdfsParameter.java | 36 + .../sqoop/sources/SourceHiveParameter.java | 72 ++ .../sqoop/sources/SourceMysqlParameter.java | 136 +++ .../sqoop/targets/TargetHdfsParameter.java | 97 ++ .../sqoop/targets/TargetHiveParameter.java | 120 +++ .../sqoop/targets/TargetMysqlParameter.java | 132 +++ .../common/utils/TaskParametersUtils.java | 3 + .../common/task/EntityTestUtils.java | 81 ++ .../common/task/SqoopParameterEntityTest.java | 52 + .../server/worker/task/AbstractTask.java | 4 + .../server/worker/task/TaskManager.java | 3 + .../server/worker/task/sqoop/SqoopTask.java | 78 ++ .../task/sqoop/generator/CommonGenerator.java | 45 + .../sqoop/generator/ISourceGenerator.java | 32 + .../sqoop/generator/ITargetGenerator.java | 32 + .../sqoop/generator/SqoopJobGenerator.java | 109 ++ .../sources/HdfsSourceGenerator.java | 56 + .../sources/HiveSourceGenerator.java | 61 ++ .../sources/MysqlSourceGenerator.java | 120 +++ .../targets/HdfsTargetGenerator.java | 75 ++ .../targets/HiveTargetGenerator.java | 85 ++ .../targets/MysqlTargetGenerator.java | 93 ++ .../worker/task/sqoop/SqoopTaskTest.java | 139 +++ .../js/conf/home/pages/dag/_source/config.js | 4 + .../js/conf/home/pages/dag/_source/dag.scss | 3 + .../pages/dag/_source/formModel/formModel.vue | 9 + .../dag/_source/formModel/tasks/sqoop.vue | 981 ++++++++++++++++++ .../conf/home/pages/dag/img/toolbar_SQOOP.png | Bin 0 -> 1157 bytes .../src/js/module/i18n/locale/en_US.js | 51 +- .../src/js/module/i18n/locale/zh_CN.js | 48 +- pom.xml | 2 + 34 files changed, 2910 insertions(+), 3 deletions(-) create mode 100644 dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/QueryType.java create mode 100644 dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sqoop/SqoopParameters.java create mode 100644 dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sqoop/sources/SourceHdfsParameter.java create mode 100644 dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sqoop/sources/SourceHiveParameter.java create mode 100644 dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sqoop/sources/SourceMysqlParameter.java create mode 100644 dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sqoop/targets/TargetHdfsParameter.java create mode 100644 dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sqoop/targets/TargetHiveParameter.java create mode 100644 dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sqoop/targets/TargetMysqlParameter.java create mode 100644 dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/task/EntityTestUtils.java create mode 100644 dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/task/SqoopParameterEntityTest.java create mode 100644 dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/SqoopTask.java create mode 100644 dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/CommonGenerator.java create mode 100644 dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/ISourceGenerator.java create mode 100644 dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/ITargetGenerator.java create mode 100644 dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/SqoopJobGenerator.java create mode 100644 dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/sources/HdfsSourceGenerator.java create mode 100644 dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/sources/HiveSourceGenerator.java create mode 100644 dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/sources/MysqlSourceGenerator.java create mode 100644 dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/targets/HdfsTargetGenerator.java create mode 100644 dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/targets/HiveTargetGenerator.java create mode 100644 dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/targets/MysqlTargetGenerator.java create mode 100644 dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/sqoop/SqoopTaskTest.java create mode 100644 dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/sqoop.vue create mode 100644 dolphinscheduler-ui/src/js/conf/home/pages/dag/img/toolbar_SQOOP.png diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/QueryType.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/QueryType.java new file mode 100644 index 0000000000..13820b4bab --- /dev/null +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/QueryType.java @@ -0,0 +1,33 @@ +/* + * 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.common.enums; + +public enum QueryType { + + FORM, + SQL; + + public static QueryType getEnum(int value){ + for (QueryType e:QueryType.values()) { + if(e.ordinal() == value) { + return e; + } + } + //For values out of enum scope + return null; + } +} diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/TaskType.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/TaskType.java index b996c3aec9..b56e72560c 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/TaskType.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/TaskType.java @@ -34,6 +34,7 @@ public enum TaskType { * 8 FLINK * 9 HTTP * 10 DATAX + * 11 SQOOP */ SHELL(0, "shell"), SQL(1, "sql"), @@ -45,7 +46,8 @@ public enum TaskType { DEPENDENT(7, "dependent"), FLINK(8, "flink"), HTTP(9, "http"), - DATAX(10, "datax"); + DATAX(10, "datax"), + SQOOP(11, "sqoop"); TaskType(int code, String descp){ this.code = code; diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sqoop/SqoopParameters.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sqoop/SqoopParameters.java new file mode 100644 index 0000000000..fb65df6c1b --- /dev/null +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sqoop/SqoopParameters.java @@ -0,0 +1,117 @@ +/* + * 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.common.task.sqoop; + +import org.apache.dolphinscheduler.common.task.AbstractParameters; +import org.apache.dolphinscheduler.common.utils.StringUtils; + +import java.util.ArrayList; +import java.util.List; + +/** + * sqoop parameters + */ +public class SqoopParameters extends AbstractParameters { + + /** + * model type + */ + private String modelType; + /** + * concurrency + */ + private int concurrency; + /** + * source type + */ + private String sourceType; + /** + * target type + */ + private String targetType; + /** + * source params + */ + private String sourceParams; + /** + * target params + */ + private String targetParams; + + public String getModelType() { + return modelType; + } + + public void setModelType(String modelType) { + this.modelType = modelType; + } + + public int getConcurrency() { + return concurrency; + } + + public void setConcurrency(int concurrency) { + this.concurrency = concurrency; + } + + public String getSourceType() { + return sourceType; + } + + public void setSourceType(String sourceType) { + this.sourceType = sourceType; + } + + public String getTargetType() { + return targetType; + } + + public void setTargetType(String targetType) { + this.targetType = targetType; + } + + public String getSourceParams() { + return sourceParams; + } + + public void setSourceParams(String sourceParams) { + this.sourceParams = sourceParams; + } + + public String getTargetParams() { + return targetParams; + } + + public void setTargetParams(String targetParams) { + this.targetParams = targetParams; + } + + @Override + public boolean checkParameters() { + return StringUtils.isNotEmpty(modelType)&& + concurrency != 0 && + StringUtils.isNotEmpty(sourceType)&& + StringUtils.isNotEmpty(targetType)&& + StringUtils.isNotEmpty(sourceParams)&& + StringUtils.isNotEmpty(targetParams); + } + + @Override + public List getResourceFilesList() { + return new ArrayList<>(); + } +} diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sqoop/sources/SourceHdfsParameter.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sqoop/sources/SourceHdfsParameter.java new file mode 100644 index 0000000000..07f1157be0 --- /dev/null +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sqoop/sources/SourceHdfsParameter.java @@ -0,0 +1,36 @@ +/* + * 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.common.task.sqoop.sources; + +/** + * source hdfs parameter + */ +public class SourceHdfsParameter { + + /** + * export dir + */ + private String exportDir; + + public String getExportDir() { + return exportDir; + } + + public void setExportDir(String exportDir) { + this.exportDir = exportDir; + } +} diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sqoop/sources/SourceHiveParameter.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sqoop/sources/SourceHiveParameter.java new file mode 100644 index 0000000000..a37840f9ea --- /dev/null +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sqoop/sources/SourceHiveParameter.java @@ -0,0 +1,72 @@ +/* + * 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.common.task.sqoop.sources; + +/** + * source hive parameter + */ +public class SourceHiveParameter { + + /** + * hive database + */ + private String hiveDatabase; + /** + * hive table + */ + private String hiveTable; + /** + * hive partition key + */ + private String hivePartitionKey; + /** + * hive partition value + */ + private String hivePartitionValue; + + public String getHiveDatabase() { + return hiveDatabase; + } + + public void setHiveDatabase(String hiveDatabase) { + this.hiveDatabase = hiveDatabase; + } + + public String getHiveTable() { + return hiveTable; + } + + public void setHiveTable(String hiveTable) { + this.hiveTable = hiveTable; + } + + public String getHivePartitionKey() { + return hivePartitionKey; + } + + public void setHivePartitionKey(String hivePartitionKey) { + this.hivePartitionKey = hivePartitionKey; + } + + public String getHivePartitionValue() { + return hivePartitionValue; + } + + public void setHivePartitionValue(String hivePartitionValue) { + this.hivePartitionValue = hivePartitionValue; + } +} diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sqoop/sources/SourceMysqlParameter.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sqoop/sources/SourceMysqlParameter.java new file mode 100644 index 0000000000..f80d681b59 --- /dev/null +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sqoop/sources/SourceMysqlParameter.java @@ -0,0 +1,136 @@ +/* + * 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.common.task.sqoop.sources; + +import org.apache.dolphinscheduler.common.process.Property; + +import java.util.List; + +/** + * source mysql parameter + */ +public class SourceMysqlParameter { + + /** + * src datasource + */ + private int srcDatasource; + /** + * src table + */ + private String srcTable; + /** + * src query type + */ + private int srcQueryType; + /** + * src query sql + */ + private String srcQuerySql; + /** + * src column type + */ + private int srcColumnType; + /** + * src columns + */ + private String srcColumns; + /** + * src condition list + */ + private List srcConditionList; + /** + * map column hive + */ + private List mapColumnHive; + /** + * map column java + */ + private List mapColumnJava; + + public int getSrcDatasource() { + return srcDatasource; + } + + public void setSrcDatasource(int srcDatasource) { + this.srcDatasource = srcDatasource; + } + + public String getSrcTable() { + return srcTable; + } + + public void setSrcTable(String srcTable) { + this.srcTable = srcTable; + } + + public int getSrcQueryType() { + return srcQueryType; + } + + public void setSrcQueryType(int srcQueryType) { + this.srcQueryType = srcQueryType; + } + + public String getSrcQuerySql() { + return srcQuerySql; + } + + public void setSrcQuerySql(String srcQuerySql) { + this.srcQuerySql = srcQuerySql; + } + + public int getSrcColumnType() { + return srcColumnType; + } + + public void setSrcColumnType(int srcColumnType) { + this.srcColumnType = srcColumnType; + } + + public String getSrcColumns() { + return srcColumns; + } + + public void setSrcColumns(String srcColumns) { + this.srcColumns = srcColumns; + } + + public List getSrcConditionList() { + return srcConditionList; + } + + public void setSrcConditionList(List srcConditionList) { + this.srcConditionList = srcConditionList; + } + + public List getMapColumnHive() { + return mapColumnHive; + } + + public void setMapColumnHive(List mapColumnHive) { + this.mapColumnHive = mapColumnHive; + } + + public List getMapColumnJava() { + return mapColumnJava; + } + + public void setMapColumnJava(List mapColumnJava) { + this.mapColumnJava = mapColumnJava; + } +} diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sqoop/targets/TargetHdfsParameter.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sqoop/targets/TargetHdfsParameter.java new file mode 100644 index 0000000000..524921dcee --- /dev/null +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sqoop/targets/TargetHdfsParameter.java @@ -0,0 +1,97 @@ +/* + * 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.common.task.sqoop.targets; + +/** + * target hdfs parameter + */ +public class TargetHdfsParameter { + + /** + * target dir + */ + private String targetPath; + /** + * delete target dir + */ + private boolean deleteTargetDir; + /** + * file type + */ + private String fileType; + /** + * compression codec + */ + private String compressionCodec; + /** + * fields terminated + */ + private String fieldsTerminated; + /** + * lines terminated + */ + private String linesTerminated; + + public String getTargetPath() { + return targetPath; + } + + public void setTargetPath(String targetPath) { + this.targetPath = targetPath; + } + + public boolean isDeleteTargetDir() { + return deleteTargetDir; + } + + public void setDeleteTargetDir(boolean deleteTargetDir) { + this.deleteTargetDir = deleteTargetDir; + } + + public String getFileType() { + return fileType; + } + + public void setFileType(String fileType) { + this.fileType = fileType; + } + + public String getCompressionCodec() { + return compressionCodec; + } + + public void setCompressionCodec(String compressionCodec) { + this.compressionCodec = compressionCodec; + } + + public String getFieldsTerminated() { + return fieldsTerminated; + } + + public void setFieldsTerminated(String fieldsTerminated) { + this.fieldsTerminated = fieldsTerminated; + } + + public String getLinesTerminated() { + return linesTerminated; + } + + public void setLinesTerminated(String linesTerminated) { + this.linesTerminated = linesTerminated; + } + +} diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sqoop/targets/TargetHiveParameter.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sqoop/targets/TargetHiveParameter.java new file mode 100644 index 0000000000..f9bfde3374 --- /dev/null +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sqoop/targets/TargetHiveParameter.java @@ -0,0 +1,120 @@ +/* + * 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.common.task.sqoop.targets; + +/** + * target hive parameter + */ +public class TargetHiveParameter { + + /** + * hive database + */ + private String hiveDatabase; + /** + * hive table + */ + private String hiveTable; + /** + * create hive table + */ + private boolean createHiveTable; + /** + * drop delimiter + */ + private boolean dropDelimiter; + /** + * hive overwrite + */ + private boolean hiveOverWrite; + /** + * replace delimiter + */ + private String replaceDelimiter; + /** + * hive partition key + */ + private String hivePartitionKey; + /** + * hive partition value + */ + private String hivePartitionValue; + + public String getHiveDatabase() { + return hiveDatabase; + } + + public void setHiveDatabase(String hiveDatabase) { + this.hiveDatabase = hiveDatabase; + } + + public String getHiveTable() { + return hiveTable; + } + + public void setHiveTable(String hiveTable) { + this.hiveTable = hiveTable; + } + + public boolean isCreateHiveTable() { + return createHiveTable; + } + + public void setCreateHiveTable(boolean createHiveTable) { + this.createHiveTable = createHiveTable; + } + + public boolean isDropDelimiter() { + return dropDelimiter; + } + + public void setDropDelimiter(boolean dropDelimiter) { + this.dropDelimiter = dropDelimiter; + } + + public boolean isHiveOverWrite() { + return hiveOverWrite; + } + + public void setHiveOverWrite(boolean hiveOverWrite) { + this.hiveOverWrite = hiveOverWrite; + } + + public String getReplaceDelimiter() { + return replaceDelimiter; + } + + public void setReplaceDelimiter(String replaceDelimiter) { + this.replaceDelimiter = replaceDelimiter; + } + + public String getHivePartitionKey() { + return hivePartitionKey; + } + + public void setHivePartitionKey(String hivePartitionKey) { + this.hivePartitionKey = hivePartitionKey; + } + + public String getHivePartitionValue() { + return hivePartitionValue; + } + + public void setHivePartitionValue(String hivePartitionValue) { + this.hivePartitionValue = hivePartitionValue; + } +} diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sqoop/targets/TargetMysqlParameter.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sqoop/targets/TargetMysqlParameter.java new file mode 100644 index 0000000000..47126ae993 --- /dev/null +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/sqoop/targets/TargetMysqlParameter.java @@ -0,0 +1,132 @@ +/* + * 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.common.task.sqoop.targets; + +/** + * target mysql parameter + */ +public class TargetMysqlParameter { + + /** + * target datasource + */ + private int targetDatasource; + /** + * target table + */ + private String targetTable; + /** + * target columns + */ + private String targetColumns; + /** + * fields terminated + */ + private String fieldsTerminated; + /** + * lines terminated + */ + private String linesTerminated; + /** + * pre query + */ + private String preQuery; + /** + * is update + */ + private boolean isUpdate; + /** + * target update key + */ + private String targetUpdateKey; + /** + * target update mode + */ + private String targetUpdateMode; + + public int getTargetDatasource() { + return targetDatasource; + } + + public void setTargetDatasource(int targetDatasource) { + this.targetDatasource = targetDatasource; + } + + public String getTargetTable() { + return targetTable; + } + + public void setTargetTable(String targetTable) { + this.targetTable = targetTable; + } + + public String getTargetColumns() { + return targetColumns; + } + + public void setTargetColumns(String targetColumns) { + this.targetColumns = targetColumns; + } + + public String getFieldsTerminated() { + return fieldsTerminated; + } + + public void setFieldsTerminated(String fieldsTerminated) { + this.fieldsTerminated = fieldsTerminated; + } + + public String getLinesTerminated() { + return linesTerminated; + } + + public void setLinesTerminated(String linesTerminated) { + this.linesTerminated = linesTerminated; + } + + public String getPreQuery() { + return preQuery; + } + + public void setPreQuery(String preQuery) { + this.preQuery = preQuery; + } + + public boolean isUpdate() { + return isUpdate; + } + + public void setUpdate(boolean update) { + isUpdate = update; + } + + public String getTargetUpdateKey() { + return targetUpdateKey; + } + + public void setTargetUpdateKey(String targetUpdateKey) { + this.targetUpdateKey = targetUpdateKey; + } + + public String getTargetUpdateMode() { + return targetUpdateMode; + } + + public void setTargetUpdateMode(String targetUpdateMode) { + this.targetUpdateMode = targetUpdateMode; + } +} diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/TaskParametersUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/TaskParametersUtils.java index a2ae6a68e3..7a0e069a9a 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/TaskParametersUtils.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/TaskParametersUtils.java @@ -28,6 +28,7 @@ import org.apache.dolphinscheduler.common.task.python.PythonParameters; import org.apache.dolphinscheduler.common.task.shell.ShellParameters; import org.apache.dolphinscheduler.common.task.spark.SparkParameters; import org.apache.dolphinscheduler.common.task.sql.SqlParameters; +import org.apache.dolphinscheduler.common.task.sqoop.SqoopParameters; import org.apache.dolphinscheduler.common.task.subprocess.SubProcessParameters; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -71,6 +72,8 @@ public class TaskParametersUtils { return JSONUtils.parseObject(parameter, HttpParameters.class); case DATAX: return JSONUtils.parseObject(parameter, DataxParameters.class); + case SQOOP: + return JSONUtils.parseObject(parameter, SqoopParameters.class); default: return null; } diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/task/EntityTestUtils.java b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/task/EntityTestUtils.java new file mode 100644 index 0000000000..5d867bc4d9 --- /dev/null +++ b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/task/EntityTestUtils.java @@ -0,0 +1,81 @@ +/* + * 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.common.task; + +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.*; + +/** + * entity test utils + */ +public class EntityTestUtils { + + private static final Map OBJECT_MAP = new HashMap<>(); + + private static final String SKIP_METHOD = "getClass,notify,notifyAll,wait,equals,hashCode,clone"; + + static { + OBJECT_MAP.put("java.lang.Long", 1L); + OBJECT_MAP.put("java.lang.String", "test"); + OBJECT_MAP.put("java.lang.Integer", 1); + OBJECT_MAP.put("int", 1); + OBJECT_MAP.put("long", 1L); + OBJECT_MAP.put("java.util.Date", new Date()); + OBJECT_MAP.put("char", '1'); + OBJECT_MAP.put("java.util.Map", new HashMap()); + OBJECT_MAP.put("boolean", true); + } + + public static void run(List classList) + throws IllegalAccessException, InvocationTargetException, InstantiationException { + for (Class temp : classList) { + Object tempInstance = new Object(); + Constructor[] constructors = temp.getConstructors(); + for (Constructor constructor : constructors) { + final Class[] parameterTypes = constructor.getParameterTypes(); + if (parameterTypes.length == 0) { + tempInstance = constructor.newInstance(); + } else { + Object[] objects = new Object[parameterTypes.length]; + for (int i = 0; i < parameterTypes.length; i++) { + objects[i] = OBJECT_MAP.get(parameterTypes[i].getName()); + } + tempInstance = constructor.newInstance(objects); + } + } + + Method[] methods = temp.getMethods(); + for (final Method method : methods) { + if (SKIP_METHOD.contains(method.getName())) { + break; + } + final Class[] parameterTypes = method.getParameterTypes(); + if (parameterTypes.length != 0) { + Object[] objects = new Object[parameterTypes.length]; + for (int i = 0; i < parameterTypes.length; i++) { + objects[i] = OBJECT_MAP.get(parameterTypes[i].getName()); + } + method.invoke(tempInstance, objects); + } else { + method.invoke(tempInstance); + } + } + } + } +} \ No newline at end of file diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/task/SqoopParameterEntityTest.java b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/task/SqoopParameterEntityTest.java new file mode 100644 index 0000000000..5f35e89ddd --- /dev/null +++ b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/task/SqoopParameterEntityTest.java @@ -0,0 +1,52 @@ +/* + * 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.common.task; + +import org.apache.dolphinscheduler.common.task.sqoop.SqoopParameters; +import org.apache.dolphinscheduler.common.task.sqoop.sources.SourceHdfsParameter; +import org.apache.dolphinscheduler.common.task.sqoop.sources.SourceHiveParameter; +import org.apache.dolphinscheduler.common.task.sqoop.sources.SourceMysqlParameter; +import org.apache.dolphinscheduler.common.task.sqoop.targets.TargetHdfsParameter; +import org.apache.dolphinscheduler.common.task.sqoop.targets.TargetHiveParameter; +import org.apache.dolphinscheduler.common.task.sqoop.targets.TargetMysqlParameter; +import org.junit.Assert; +import org.junit.Test; +import java.util.ArrayList; +import java.util.List; + +/** + * sqoop parameter entity test + */ +public class SqoopParameterEntityTest { + + @Test + public void testEntity(){ + try { + List classList = new ArrayList<>(); + classList.add(SourceMysqlParameter.class); + classList.add(SourceHiveParameter.class); + classList.add(SourceHdfsParameter.class); + classList.add(SqoopParameters.class); + classList.add(TargetMysqlParameter.class); + classList.add(TargetHiveParameter.class); + classList.add(TargetHdfsParameter.class); + EntityTestUtils.run(classList); + } catch (Exception e) { + Assert.fail(e.getMessage()); + } + } +} diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractTask.java index f2772d0747..5c50df2265 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractTask.java @@ -30,6 +30,7 @@ import org.apache.dolphinscheduler.common.task.python.PythonParameters; import org.apache.dolphinscheduler.common.task.shell.ShellParameters; import org.apache.dolphinscheduler.common.task.spark.SparkParameters; import org.apache.dolphinscheduler.common.task.sql.SqlParameters; +import org.apache.dolphinscheduler.common.task.sqoop.SqoopParameters; import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.dao.TaskRecordDao; import org.apache.dolphinscheduler.server.utils.ParamUtils; @@ -198,6 +199,9 @@ public abstract class AbstractTask { case DATAX: paramsClass = DataxParameters.class; break; + case SQOOP: + paramsClass = SqoopParameters.class; + break; default: logger.error("not support this task type: {}", taskType); throw new IllegalArgumentException("not support this task type"); diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/TaskManager.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/TaskManager.java index 67deb7a3fa..1ab6f2f3da 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/TaskManager.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/TaskManager.java @@ -29,6 +29,7 @@ import org.apache.dolphinscheduler.server.worker.task.python.PythonTask; import org.apache.dolphinscheduler.server.worker.task.shell.ShellTask; import org.apache.dolphinscheduler.server.worker.task.spark.SparkTask; import org.apache.dolphinscheduler.server.worker.task.sql.SqlTask; +import org.apache.dolphinscheduler.server.worker.task.sqoop.SqoopTask; import org.slf4j.Logger; /** @@ -68,6 +69,8 @@ public class TaskManager { return new HttpTask(props, logger); case DATAX: return new DataxTask(props, logger); + case SQOOP: + return new SqoopTask(props, logger); default: logger.error("unsupport task type: {}", taskType); throw new IllegalArgumentException("not support task type"); diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/SqoopTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/SqoopTask.java new file mode 100644 index 0000000000..64bc7924d2 --- /dev/null +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/SqoopTask.java @@ -0,0 +1,78 @@ +/* + * 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.server.worker.task.sqoop; + +import com.alibaba.fastjson.JSON; +import org.apache.dolphinscheduler.common.process.Property; +import org.apache.dolphinscheduler.common.task.AbstractParameters; +import org.apache.dolphinscheduler.common.task.sqoop.SqoopParameters; +import org.apache.dolphinscheduler.common.utils.ParameterUtils; +import org.apache.dolphinscheduler.server.utils.ParamUtils; +import org.apache.dolphinscheduler.server.worker.task.AbstractYarnTask; +import org.apache.dolphinscheduler.server.worker.task.TaskProps; +import org.apache.dolphinscheduler.server.worker.task.sqoop.generator.SqoopJobGenerator; +import org.slf4j.Logger; +import java.util.Map; + +/** + * sqoop task extends the shell task + */ +public class SqoopTask extends AbstractYarnTask { + + private SqoopParameters sqoopParameters; + + public SqoopTask(TaskProps props, Logger logger){ + super(props,logger); + } + + @Override + public void init() throws Exception { + logger.info("sqoop task params {}", taskProps.getTaskParams()); + sqoopParameters = + JSON.parseObject(taskProps.getTaskParams(),SqoopParameters.class); + if (!sqoopParameters.checkParameters()) { + throw new RuntimeException("sqoop task params is not valid"); + } + + } + + @Override + protected String buildCommand() throws Exception { + //get sqoop scripts + SqoopJobGenerator generator = new SqoopJobGenerator(); + String script = generator.generateSqoopJob(sqoopParameters); + + Map paramsMap = ParamUtils.convert(taskProps.getUserDefParamsMap(), + taskProps.getDefinedParams(), + sqoopParameters.getLocalParametersMap(), + taskProps.getCmdTypeIfComplement(), + taskProps.getScheduleTime()); + + if(paramsMap != null){ + String resultScripts = ParameterUtils.convertParameterPlaceholders(script, ParamUtils.convert(paramsMap)); + logger.info("sqoop script: {}", resultScripts); + return resultScripts; + } + + return null; + } + + @Override + public AbstractParameters getParameters() { + return sqoopParameters; + } +} diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/CommonGenerator.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/CommonGenerator.java new file mode 100644 index 0000000000..4944bac5ba --- /dev/null +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/CommonGenerator.java @@ -0,0 +1,45 @@ +/* + * 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.server.worker.task.sqoop.generator; + +import org.apache.dolphinscheduler.common.task.sqoop.SqoopParameters; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * common script generator + */ +public class CommonGenerator { + + private Logger logger = LoggerFactory.getLogger(getClass()); + + public String generate(SqoopParameters sqoopParameters) { + StringBuilder result = new StringBuilder(); + try{ + result.append("sqoop ") + .append(sqoopParameters.getModelType()); + if(sqoopParameters.getConcurrency() >0){ + result.append(" -m ") + .append(sqoopParameters.getConcurrency()); + } + }catch (Exception e){ + logger.error(e.getMessage()); + } + + return result.toString(); + } +} diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/ISourceGenerator.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/ISourceGenerator.java new file mode 100644 index 0000000000..6c1d1fdca8 --- /dev/null +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/ISourceGenerator.java @@ -0,0 +1,32 @@ +/* + * 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.server.worker.task.sqoop.generator; + +import org.apache.dolphinscheduler.common.task.sqoop.SqoopParameters; + +/** + * Source Generator Interface + */ +public interface ISourceGenerator { + + /** + * generate the source script + * @param sqoopParameters sqoop params + * @return + */ + String generate(SqoopParameters sqoopParameters); +} diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/ITargetGenerator.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/ITargetGenerator.java new file mode 100644 index 0000000000..be307af5f2 --- /dev/null +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/ITargetGenerator.java @@ -0,0 +1,32 @@ +/* + * 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.server.worker.task.sqoop.generator; + +import org.apache.dolphinscheduler.common.task.sqoop.SqoopParameters; + +/** + * Target Generator Interface + */ +public interface ITargetGenerator { + + /** + * generate the target script + * @param sqoopParameters sqoop params + * @return + */ + String generate(SqoopParameters sqoopParameters); +} diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/SqoopJobGenerator.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/SqoopJobGenerator.java new file mode 100644 index 0000000000..24c76e027d --- /dev/null +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/SqoopJobGenerator.java @@ -0,0 +1,109 @@ +/* + * 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.server.worker.task.sqoop.generator; + +import org.apache.dolphinscheduler.common.task.sqoop.SqoopParameters; +import org.apache.dolphinscheduler.server.worker.task.sqoop.generator.sources.HdfsSourceGenerator; +import org.apache.dolphinscheduler.server.worker.task.sqoop.generator.sources.HiveSourceGenerator; +import org.apache.dolphinscheduler.server.worker.task.sqoop.generator.sources.MysqlSourceGenerator; +import org.apache.dolphinscheduler.server.worker.task.sqoop.generator.targets.HdfsTargetGenerator; +import org.apache.dolphinscheduler.server.worker.task.sqoop.generator.targets.HiveTargetGenerator; +import org.apache.dolphinscheduler.server.worker.task.sqoop.generator.targets.MysqlTargetGenerator; + +/** + * Sqoop Job Scripts Generator + */ +public class SqoopJobGenerator { + + private static final String MYSQL = "MYSQL"; + private static final String HIVE = "HIVE"; + private static final String HDFS = "HDFS"; + + /** + * target script generator + */ + private ITargetGenerator targetGenerator; + /** + * source script generator + */ + private ISourceGenerator sourceGenerator; + /** + * common script generator + */ + private CommonGenerator commonGenerator; + + public SqoopJobGenerator(){ + commonGenerator = new CommonGenerator(); + } + + private void createSqoopJobGenerator(String sourceType,String targetType){ + sourceGenerator = createSourceGenerator(sourceType); + targetGenerator = createTargetGenerator(targetType); + } + + /** + * get the final sqoop scripts + * @param sqoopParameters + * @return + */ + public String generateSqoopJob(SqoopParameters sqoopParameters){ + createSqoopJobGenerator(sqoopParameters.getSourceType(),sqoopParameters.getTargetType()); + if(sourceGenerator == null || targetGenerator == null){ + return null; + } + + return commonGenerator.generate(sqoopParameters) + + sourceGenerator.generate(sqoopParameters) + + targetGenerator.generate(sqoopParameters); + } + + /** + * get the source generator + * @param sourceType + * @return + */ + private ISourceGenerator createSourceGenerator(String sourceType){ + switch (sourceType){ + case MYSQL: + return new MysqlSourceGenerator(); + case HIVE: + return new HiveSourceGenerator(); + case HDFS: + return new HdfsSourceGenerator(); + default: + return null; + } + } + + /** + * get the target generator + * @param targetType + * @return + */ + private ITargetGenerator createTargetGenerator(String targetType){ + switch (targetType){ + case MYSQL: + return new MysqlTargetGenerator(); + case HIVE: + return new HiveTargetGenerator(); + case HDFS: + return new HdfsTargetGenerator(); + default: + return null; + } + } +} diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/sources/HdfsSourceGenerator.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/sources/HdfsSourceGenerator.java new file mode 100644 index 0000000000..47b01363e6 --- /dev/null +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/sources/HdfsSourceGenerator.java @@ -0,0 +1,56 @@ +/* + * 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.server.worker.task.sqoop.generator.sources; + +import org.apache.dolphinscheduler.common.task.sqoop.SqoopParameters; +import org.apache.dolphinscheduler.common.task.sqoop.sources.SourceHdfsParameter; +import org.apache.dolphinscheduler.common.utils.JSONUtils; +import org.apache.dolphinscheduler.common.utils.StringUtils; +import org.apache.dolphinscheduler.server.worker.task.sqoop.generator.ISourceGenerator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * hdfs source generator + */ +public class HdfsSourceGenerator implements ISourceGenerator { + + private Logger logger = LoggerFactory.getLogger(getClass()); + + @Override + public String generate(SqoopParameters sqoopParameters) { + StringBuilder result = new StringBuilder(); + try{ + SourceHdfsParameter sourceHdfsParameter + = JSONUtils.parseObject(sqoopParameters.getSourceParams(),SourceHdfsParameter.class); + + if(sourceHdfsParameter != null){ + if(StringUtils.isNotEmpty(sourceHdfsParameter.getExportDir())){ + result.append(" --export-dir ") + .append(sourceHdfsParameter.getExportDir()); + }else{ + throw new Exception("--export-dir is null"); + } + + } + }catch (Exception e){ + logger.error("get hdfs source failed",e); + } + + return result.toString(); + } +} diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/sources/HiveSourceGenerator.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/sources/HiveSourceGenerator.java new file mode 100644 index 0000000000..91363e296a --- /dev/null +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/sources/HiveSourceGenerator.java @@ -0,0 +1,61 @@ +/* + * 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.server.worker.task.sqoop.generator.sources; + +import org.apache.commons.lang.StringUtils; +import org.apache.dolphinscheduler.common.task.sqoop.SqoopParameters; +import org.apache.dolphinscheduler.common.task.sqoop.sources.SourceHiveParameter; +import org.apache.dolphinscheduler.common.utils.JSONUtils; +import org.apache.dolphinscheduler.server.worker.task.sqoop.generator.ISourceGenerator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * hive source generator + */ +public class HiveSourceGenerator implements ISourceGenerator { + + private Logger logger = LoggerFactory.getLogger(getClass()); + + @Override + public String generate(SqoopParameters sqoopParameters) { + StringBuilder sb = new StringBuilder(); + try{ + SourceHiveParameter sourceHiveParameter + = JSONUtils.parseObject(sqoopParameters.getSourceParams(),SourceHiveParameter.class); + if(sourceHiveParameter != null){ + if(StringUtils.isNotEmpty(sourceHiveParameter.getHiveDatabase())){ + sb.append(" --hcatalog-database ").append(sourceHiveParameter.getHiveDatabase()); + } + + if(StringUtils.isNotEmpty(sourceHiveParameter.getHiveTable())){ + sb.append(" --hcatalog-table ").append(sourceHiveParameter.getHiveTable()); + } + + if(StringUtils.isNotEmpty(sourceHiveParameter.getHivePartitionKey())&& + StringUtils.isNotEmpty(sourceHiveParameter.getHivePartitionValue())){ + sb.append(" --hcatalog-partition-keys ").append(sourceHiveParameter.getHivePartitionKey()) + .append(" --hcatalog-partition-values ").append(sourceHiveParameter.getHivePartitionValue()); + } + } + }catch (Exception e){ + logger.error(e.getMessage()); + } + + return sb.toString(); + } +} diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/sources/MysqlSourceGenerator.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/sources/MysqlSourceGenerator.java new file mode 100644 index 0000000000..050fef7cc7 --- /dev/null +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/sources/MysqlSourceGenerator.java @@ -0,0 +1,120 @@ +/* + * 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.server.worker.task.sqoop.generator.sources; + +import org.apache.commons.lang.StringUtils; +import org.apache.dolphinscheduler.common.enums.QueryType; +import org.apache.dolphinscheduler.common.process.Property; +import org.apache.dolphinscheduler.common.task.sqoop.SqoopParameters; +import org.apache.dolphinscheduler.common.task.sqoop.sources.SourceMysqlParameter; +import org.apache.dolphinscheduler.common.utils.JSONUtils; +import org.apache.dolphinscheduler.dao.datasource.BaseDataSource; +import org.apache.dolphinscheduler.dao.datasource.DataSourceFactory; +import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; +import org.apache.dolphinscheduler.dao.entity.DataSource; +import org.apache.dolphinscheduler.server.worker.task.sqoop.generator.ISourceGenerator; +import org.apache.dolphinscheduler.service.process.ProcessService; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; + +/** + * mysql source generator + */ +public class MysqlSourceGenerator implements ISourceGenerator { + + private Logger logger = LoggerFactory.getLogger(getClass()); + + @Override + public String generate(SqoopParameters sqoopParameters) { + StringBuilder result = new StringBuilder(); + try { + SourceMysqlParameter sourceMysqlParameter + = JSONUtils.parseObject(sqoopParameters.getSourceParams(),SourceMysqlParameter.class); + + if(sourceMysqlParameter != null){ + ProcessService processService = SpringApplicationContext.getBean(ProcessService.class); + DataSource dataSource= processService.findDataSourceById(sourceMysqlParameter.getSrcDatasource()); + BaseDataSource baseDataSource = DataSourceFactory.getDatasource(dataSource.getType(), + dataSource.getConnectionParams()); + if(baseDataSource != null){ + result.append(" --connect ") + .append(baseDataSource.getJdbcUrl()) + .append(" --username ") + .append(baseDataSource.getUser()) + .append(" --password ") + .append(baseDataSource.getPassword()); + + if(sourceMysqlParameter.getSrcQueryType() == QueryType.FORM.ordinal()){ + if(StringUtils.isNotEmpty(sourceMysqlParameter.getSrcTable())){ + result.append(" --table ").append(sourceMysqlParameter.getSrcTable()); + } + + if(StringUtils.isNotEmpty(sourceMysqlParameter.getSrcColumns())){ + result.append(" --columns ").append(sourceMysqlParameter.getSrcColumns()); + } + + }else if(sourceMysqlParameter.getSrcQueryType() == QueryType.SQL.ordinal()){ + if(StringUtils.isNotEmpty(sourceMysqlParameter.getSrcQuerySql())){ + + String srcQuery = sourceMysqlParameter.getSrcQuerySql(); + if(srcQuery.toLowerCase().contains("where")){ + srcQuery += " AND "+"$CONDITIONS"; + }else{ + srcQuery += " WHERE $CONDITIONS"; + } + result.append(" --query \'"+srcQuery+"\'"); + } + } + + List mapColumnHive = sourceMysqlParameter.getMapColumnHive(); + + if(mapColumnHive != null && !mapColumnHive.isEmpty()){ + String columnMap = ""; + for(Property item:mapColumnHive){ + columnMap = item.getProp()+"="+ item.getValue()+","; + } + + if(StringUtils.isNotEmpty(columnMap)){ + result.append(" --map-column-hive ") + .append(columnMap.substring(0,columnMap.length()-1)); + } + } + + List mapColumnJava = sourceMysqlParameter.getMapColumnJava(); + + if(mapColumnJava != null && !mapColumnJava.isEmpty()){ + String columnMap = ""; + for(Property item:mapColumnJava){ + columnMap = item.getProp()+"="+ item.getValue()+","; + } + + if(StringUtils.isNotEmpty(columnMap)){ + result.append(" --map-column-java ") + .append(columnMap.substring(0,columnMap.length()-1)); + } + } + } + } + }catch (Exception e){ + logger.error(e.getMessage()); + } + + return result.toString(); + } +} diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/targets/HdfsTargetGenerator.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/targets/HdfsTargetGenerator.java new file mode 100644 index 0000000000..411e9b4450 --- /dev/null +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/targets/HdfsTargetGenerator.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.server.worker.task.sqoop.generator.targets; + +import org.apache.commons.lang.StringUtils; +import org.apache.dolphinscheduler.common.task.sqoop.SqoopParameters; +import org.apache.dolphinscheduler.common.task.sqoop.targets.TargetHdfsParameter; +import org.apache.dolphinscheduler.common.utils.JSONUtils; +import org.apache.dolphinscheduler.server.worker.task.sqoop.generator.ITargetGenerator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * hdfs target generator + */ +public class HdfsTargetGenerator implements ITargetGenerator { + + private Logger logger = LoggerFactory.getLogger(getClass()); + + @Override + public String generate(SqoopParameters sqoopParameters) { + StringBuilder result = new StringBuilder(); + try{ + TargetHdfsParameter targetHdfsParameter = + JSONUtils.parseObject(sqoopParameters.getTargetParams(),TargetHdfsParameter.class); + + if(targetHdfsParameter != null){ + + if(StringUtils.isNotEmpty(targetHdfsParameter.getTargetPath())){ + result.append(" --target-dir ").append(targetHdfsParameter.getTargetPath()); + } + + if(StringUtils.isNotEmpty(targetHdfsParameter.getCompressionCodec())){ + result.append(" --compression-codec ").append(targetHdfsParameter.getCompressionCodec()); + } + + if(StringUtils.isNotEmpty(targetHdfsParameter.getFileType())){ + result.append(" ").append(targetHdfsParameter.getFileType()); + } + + if(targetHdfsParameter.isDeleteTargetDir()){ + result.append(" --delete-target-dir"); + } + + if(StringUtils.isNotEmpty(targetHdfsParameter.getFieldsTerminated())){ + result.append(" --fields-terminated-by '").append(targetHdfsParameter.getFieldsTerminated()).append("'"); + } + + if(StringUtils.isNotEmpty(targetHdfsParameter.getLinesTerminated())){ + result.append(" --lines-terminated-by '").append(targetHdfsParameter.getLinesTerminated()).append("'"); + } + + result.append(" --null-non-string 'NULL' --null-string 'NULL'"); + } + }catch(Exception e){ + logger.error(e.getMessage()); + } + + return result.toString(); + } +} diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/targets/HiveTargetGenerator.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/targets/HiveTargetGenerator.java new file mode 100644 index 0000000000..ad59173ad0 --- /dev/null +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/targets/HiveTargetGenerator.java @@ -0,0 +1,85 @@ +/* + * 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.server.worker.task.sqoop.generator.targets; + +import org.apache.commons.lang.StringUtils; +import org.apache.dolphinscheduler.common.task.sqoop.SqoopParameters; +import org.apache.dolphinscheduler.common.task.sqoop.targets.TargetHiveParameter; +import org.apache.dolphinscheduler.common.utils.JSONUtils; +import org.apache.dolphinscheduler.server.worker.task.sqoop.generator.ITargetGenerator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * hive target generator + */ +public class HiveTargetGenerator implements ITargetGenerator { + + private Logger logger = LoggerFactory.getLogger(getClass()); + + @Override + public String generate(SqoopParameters sqoopParameters) { + + StringBuilder result = new StringBuilder(); + + try{ + TargetHiveParameter targetHiveParameter = + JSONUtils.parseObject(sqoopParameters.getTargetParams(),TargetHiveParameter.class); + if(targetHiveParameter != null){ + + result.append(" --hive-import "); + + if(StringUtils.isNotEmpty(targetHiveParameter.getHiveDatabase())&& + StringUtils.isNotEmpty(targetHiveParameter.getHiveTable())){ + result.append(" --hive-table ") + .append(targetHiveParameter.getHiveDatabase()) + .append(".") + .append(targetHiveParameter.getHiveTable()); + } + + if(targetHiveParameter.isCreateHiveTable()){ + result.append(" --create-hive-table"); + } + + if(targetHiveParameter.isDropDelimiter()){ + result.append(" --hive-drop-import-delims"); + } + + if(targetHiveParameter.isHiveOverWrite()){ + result.append(" --hive-overwrite -delete-target-dir"); + } + + if(StringUtils.isNotEmpty(targetHiveParameter.getReplaceDelimiter())){ + result.append(" --hive-delims-replacement ").append(targetHiveParameter.getReplaceDelimiter()); + } + + if(StringUtils.isNotEmpty(targetHiveParameter.getHivePartitionKey())&& + StringUtils.isNotEmpty(targetHiveParameter.getHivePartitionValue())){ + result.append(" --hive-partition-key ") + .append(targetHiveParameter.getHivePartitionKey()) + .append(" --hive-partition-value ") + .append(targetHiveParameter.getHivePartitionValue()); + } + + } + }catch(Exception e){ + logger.error(e.getMessage()); + } + + return result.toString(); + } +} diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/targets/MysqlTargetGenerator.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/targets/MysqlTargetGenerator.java new file mode 100644 index 0000000000..0733338812 --- /dev/null +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/targets/MysqlTargetGenerator.java @@ -0,0 +1,93 @@ +/* + * 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.server.worker.task.sqoop.generator.targets; + +import org.apache.commons.lang.StringUtils; +import org.apache.dolphinscheduler.common.task.sqoop.SqoopParameters; +import org.apache.dolphinscheduler.common.task.sqoop.targets.TargetMysqlParameter; +import org.apache.dolphinscheduler.common.utils.JSONUtils; +import org.apache.dolphinscheduler.dao.datasource.BaseDataSource; +import org.apache.dolphinscheduler.dao.datasource.DataSourceFactory; +import org.apache.dolphinscheduler.dao.entity.DataSource; +import org.apache.dolphinscheduler.server.worker.task.sqoop.generator.ITargetGenerator; +import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; +import org.apache.dolphinscheduler.service.process.ProcessService; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * mysql target generator + */ +public class MysqlTargetGenerator implements ITargetGenerator { + + private Logger logger = LoggerFactory.getLogger(getClass()); + + @Override + public String generate(SqoopParameters sqoopParameters) { + + StringBuilder result = new StringBuilder(); + try{ + + TargetMysqlParameter targetMysqlParameter = + JSONUtils.parseObject(sqoopParameters.getTargetParams(),TargetMysqlParameter.class); + + if(targetMysqlParameter != null && targetMysqlParameter.getTargetDatasource() != 0){ + + ProcessService processService = SpringApplicationContext.getBean(ProcessService.class); + DataSource dataSource= processService.findDataSourceById(targetMysqlParameter.getTargetDatasource()); + // get datasource + BaseDataSource baseDataSource = DataSourceFactory.getDatasource(dataSource.getType(), + dataSource.getConnectionParams()); + + if(baseDataSource != null){ + result.append(" --connect ") + .append(baseDataSource.getJdbcUrl()) + .append(" --username ") + .append(baseDataSource.getUser()) + .append(" --password ") + .append(baseDataSource.getPassword()) + .append(" --table ") + .append(targetMysqlParameter.getTargetTable()); + + if(StringUtils.isNotEmpty(targetMysqlParameter.getTargetColumns())){ + result.append(" --columns ").append(targetMysqlParameter.getTargetColumns()); + } + + if(StringUtils.isNotEmpty(targetMysqlParameter.getFieldsTerminated())){ + result.append(" --fields-terminated-by '").append(targetMysqlParameter.getFieldsTerminated()).append("'"); + } + + if(StringUtils.isNotEmpty(targetMysqlParameter.getLinesTerminated())){ + result.append(" --lines-terminated-by '").append(targetMysqlParameter.getLinesTerminated()).append("'"); + } + + if(targetMysqlParameter.isUpdate()){ + if(StringUtils.isNotEmpty(targetMysqlParameter.getTargetUpdateKey())&& + StringUtils.isNotEmpty(targetMysqlParameter.getTargetUpdateMode())){ + result.append(" --update-key ").append(targetMysqlParameter.getTargetUpdateKey()) + .append(" --update-mode ").append(targetMysqlParameter.getTargetUpdateMode()); + } + } + } + } + }catch (Exception e){ + logger.error(e.getMessage()); + } + + return result.toString(); + } +} diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/sqoop/SqoopTaskTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/sqoop/SqoopTaskTest.java new file mode 100644 index 0000000000..f8688e7c0c --- /dev/null +++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/sqoop/SqoopTaskTest.java @@ -0,0 +1,139 @@ +/* + * 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.server.worker.task.sqoop; + +import com.alibaba.fastjson.JSONObject; +import org.apache.dolphinscheduler.common.enums.DbType; +import org.apache.dolphinscheduler.common.task.sqoop.SqoopParameters; +import org.apache.dolphinscheduler.common.task.sqoop.sources.SourceHdfsParameter; +import org.apache.dolphinscheduler.common.task.sqoop.sources.SourceHiveParameter; +import org.apache.dolphinscheduler.common.task.sqoop.sources.SourceMysqlParameter; +import org.apache.dolphinscheduler.common.task.sqoop.targets.TargetHdfsParameter; +import org.apache.dolphinscheduler.common.task.sqoop.targets.TargetHiveParameter; +import org.apache.dolphinscheduler.common.task.sqoop.targets.TargetMysqlParameter; +import org.apache.dolphinscheduler.dao.entity.DataSource; +import org.apache.dolphinscheduler.server.worker.task.TaskProps; +import org.apache.dolphinscheduler.server.worker.task.sqoop.generator.SqoopJobGenerator; +import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; +import org.apache.dolphinscheduler.service.process.ProcessService; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.mockito.junit.MockitoJUnitRunner; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.context.ApplicationContext; + +import java.util.*; + +/** + * sqoop task test + */ +@RunWith(MockitoJUnitRunner.Silent.class) +public class SqoopTaskTest { + + private static final Logger logger = LoggerFactory.getLogger(SqoopTaskTest.class); + + private ProcessService processService; + private ApplicationContext applicationContext; + private SqoopTask sqoopTask; + + @Before + public void before() throws Exception{ + processService = Mockito.mock(ProcessService.class); + Mockito.when(processService.findDataSourceById(2)).thenReturn(getDataSource()); + applicationContext = Mockito.mock(ApplicationContext.class); + SpringApplicationContext springApplicationContext = new SpringApplicationContext(); + springApplicationContext.setApplicationContext(applicationContext); + Mockito.when(applicationContext.getBean(ProcessService.class)).thenReturn(processService); + + TaskProps props = new TaskProps(); + props.setTaskDir("/tmp"); + props.setTaskAppId(String.valueOf(System.currentTimeMillis())); + props.setTaskInstId(1); + props.setTenantCode("1"); + props.setEnvFile(".dolphinscheduler_env.sh"); + props.setTaskStartTime(new Date()); + props.setTaskTimeout(0); + props.setTaskParams("{\"concurrency\":1,\"modelType\":\"import\",\"sourceType\":\"MYSQL\",\"targetType\":\"HIVE\",\"sourceParams\":\"{\\\"srcDatasource\\\":2,\\\"srcTable\\\":\\\"person_2\\\",\\\"srcQueryType\\\":\\\"1\\\",\\\"srcQuerySql\\\":\\\"SELECT * FROM person_2\\\",\\\"srcColumnType\\\":\\\"0\\\",\\\"srcColumns\\\":\\\"\\\",\\\"srcConditionList\\\":[],\\\"mapColumnHive\\\":[],\\\"mapColumnJava\\\":[{\\\"prop\\\":\\\"id\\\",\\\"direct\\\":\\\"IN\\\",\\\"type\\\":\\\"VARCHAR\\\",\\\"value\\\":\\\"Integer\\\"}]}\",\"targetParams\":\"{\\\"hiveDatabase\\\":\\\"stg\\\",\\\"hiveTable\\\":\\\"person_internal_2\\\",\\\"createHiveTable\\\":true,\\\"dropDelimiter\\\":false,\\\"hiveOverWrite\\\":true,\\\"replaceDelimiter\\\":\\\"\\\",\\\"hivePartitionKey\\\":\\\"date\\\",\\\"hivePartitionValue\\\":\\\"2020-02-16\\\"}\",\"localParams\":[]}"); + + sqoopTask = new SqoopTask(props,logger); + sqoopTask.init(); + } + + @Test + public void testGenerator(){ + String data1 = "{\"concurrency\":1,\"modelType\":\"import\",\"sourceType\":\"MYSQL\",\"targetType\":\"HDFS\",\"sourceParams\":\"{\\\"srcDatasource\\\":2,\\\"srcTable\\\":\\\"person_2\\\",\\\"srcQueryType\\\":\\\"0\\\",\\\"srcQuerySql\\\":\\\"\\\",\\\"srcColumnType\\\":\\\"0\\\",\\\"srcColumns\\\":\\\"\\\",\\\"srcConditionList\\\":[],\\\"mapColumnHive\\\":[],\\\"mapColumnJava\\\":[]}\",\"targetParams\":\"{\\\"targetPath\\\":\\\"/ods/tmp/test/person7\\\",\\\"deleteTargetDir\\\":true,\\\"fileType\\\":\\\"--as-textfile\\\",\\\"compressionCodec\\\":\\\"\\\",\\\"fieldsTerminated\\\":\\\"@\\\",\\\"linesTerminated\\\":\\\"\\\\\\\\n\\\"}\",\"localParams\":[]}"; + SqoopParameters sqoopParameters1 = JSONObject.parseObject(data1,SqoopParameters.class); + + SqoopJobGenerator generator = new SqoopJobGenerator(); + String script = generator.generateSqoopJob(sqoopParameters1); + String expected = "sqoop import -m 1 --connect jdbc:mysql://192.168.0.111:3306/test --username kylo --password 123456 --table person_2 --target-dir /ods/tmp/test/person7 --as-textfile --delete-target-dir --fields-terminated-by '@' --lines-terminated-by '\\n' --null-non-string 'NULL' --null-string 'NULL'"; + Assert.assertEquals(expected, script); + + String data2 = "{\"concurrency\":1,\"modelType\":\"export\",\"sourceType\":\"HDFS\",\"targetType\":\"MYSQL\",\"sourceParams\":\"{\\\"exportDir\\\":\\\"/ods/tmp/test/person7\\\"}\",\"targetParams\":\"{\\\"targetDatasource\\\":2,\\\"targetTable\\\":\\\"person_3\\\",\\\"targetColumns\\\":\\\"id,name,age,sex,create_time\\\",\\\"preQuery\\\":\\\"\\\",\\\"isUpdate\\\":true,\\\"targetUpdateKey\\\":\\\"id\\\",\\\"targetUpdateMode\\\":\\\"allowinsert\\\",\\\"fieldsTerminated\\\":\\\"@\\\",\\\"linesTerminated\\\":\\\"\\\\\\\\n\\\"}\",\"localParams\":[]}"; + SqoopParameters sqoopParameters2 = JSONObject.parseObject(data2,SqoopParameters.class); + + String script2 = generator.generateSqoopJob(sqoopParameters2); + String expected2 = "sqoop export -m 1 --export-dir /ods/tmp/test/person7 --connect jdbc:mysql://192.168.0.111:3306/test --username kylo --password 123456 --table person_3 --columns id,name,age,sex,create_time --fields-terminated-by '@' --lines-terminated-by '\\n' --update-key id --update-mode allowinsert"; + Assert.assertEquals(expected2, script2); + + String data3 = "{\"concurrency\":1,\"modelType\":\"export\",\"sourceType\":\"HIVE\",\"targetType\":\"MYSQL\",\"sourceParams\":\"{\\\"hiveDatabase\\\":\\\"stg\\\",\\\"hiveTable\\\":\\\"person_internal\\\",\\\"hivePartitionKey\\\":\\\"date\\\",\\\"hivePartitionValue\\\":\\\"2020-02-17\\\"}\",\"targetParams\":\"{\\\"targetDatasource\\\":2,\\\"targetTable\\\":\\\"person_3\\\",\\\"targetColumns\\\":\\\"\\\",\\\"preQuery\\\":\\\"\\\",\\\"isUpdate\\\":false,\\\"targetUpdateKey\\\":\\\"\\\",\\\"targetUpdateMode\\\":\\\"allowinsert\\\",\\\"fieldsTerminated\\\":\\\"@\\\",\\\"linesTerminated\\\":\\\"\\\\\\\\n\\\"}\",\"localParams\":[]}"; + SqoopParameters sqoopParameters3 = JSONObject.parseObject(data3,SqoopParameters.class); + + String script3 = generator.generateSqoopJob(sqoopParameters3); + String expected3 = "sqoop export -m 1 --hcatalog-database stg --hcatalog-table person_internal --hcatalog-partition-keys date --hcatalog-partition-values 2020-02-17 --connect jdbc:mysql://192.168.0.111:3306/test --username kylo --password 123456 --table person_3 --fields-terminated-by '@' --lines-terminated-by '\\n'"; + Assert.assertEquals(expected3, script3); + + String data4 = "{\"concurrency\":1,\"modelType\":\"import\",\"sourceType\":\"MYSQL\",\"targetType\":\"HIVE\",\"sourceParams\":\"{\\\"srcDatasource\\\":2,\\\"srcTable\\\":\\\"person_2\\\",\\\"srcQueryType\\\":\\\"1\\\",\\\"srcQuerySql\\\":\\\"SELECT * FROM person_2\\\",\\\"srcColumnType\\\":\\\"0\\\",\\\"srcColumns\\\":\\\"\\\",\\\"srcConditionList\\\":[],\\\"mapColumnHive\\\":[],\\\"mapColumnJava\\\":[{\\\"prop\\\":\\\"id\\\",\\\"direct\\\":\\\"IN\\\",\\\"type\\\":\\\"VARCHAR\\\",\\\"value\\\":\\\"Integer\\\"}]}\",\"targetParams\":\"{\\\"hiveDatabase\\\":\\\"stg\\\",\\\"hiveTable\\\":\\\"person_internal_2\\\",\\\"createHiveTable\\\":true,\\\"dropDelimiter\\\":false,\\\"hiveOverWrite\\\":true,\\\"replaceDelimiter\\\":\\\"\\\",\\\"hivePartitionKey\\\":\\\"date\\\",\\\"hivePartitionValue\\\":\\\"2020-02-16\\\"}\",\"localParams\":[]}"; + SqoopParameters sqoopParameters4 = JSONObject.parseObject(data4,SqoopParameters.class); + + String script4 = generator.generateSqoopJob(sqoopParameters4); + String expected4 = "sqoop import -m 1 --connect jdbc:mysql://192.168.0.111:3306/test --username kylo --password 123456 --query 'SELECT * FROM person_2 WHERE $CONDITIONS' --map-column-java id=Integer --hive-import --hive-table stg.person_internal_2 --create-hive-table --hive-overwrite -delete-target-dir --hive-partition-key date --hive-partition-value 2020-02-16"; + Assert.assertEquals(expected4, script4); + + } + + private DataSource getDataSource() { + DataSource dataSource = new DataSource(); + dataSource.setType(DbType.MYSQL); + dataSource.setConnectionParams( + "{\"address\":\"jdbc:mysql://192.168.0.111:3306\",\"database\":\"test\",\"jdbcUrl\":\"jdbc:mysql://192.168.0.111:3306/test\",\"user\":\"kylo\",\"password\":\"123456\"}"); + dataSource.setUserId(1); + return dataSource; + } + + @Test + public void testGetParameters() { + Assert.assertNotNull(sqoopTask.getParameters()); + } + + /** + * Method: init + */ + @Test + public void testInit(){ + try { + sqoopTask.init(); + } catch (Exception e) { + Assert.fail(e.getMessage()); + } + } + +} diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/config.js b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/config.js index e8187043bf..be4daa6595 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/config.js +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/config.js @@ -283,6 +283,10 @@ let tasksType = { 'DATAX': { desc: 'DataX', color: '#1fc747' + }, + 'SQOOP': { + desc: 'SQOOP', + color: '#E46F13' } } diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.scss b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.scss index 420bae8c89..08918c969f 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.scss +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.scss @@ -104,6 +104,9 @@ .icos-DATAX { background: url("../img/toolbar_DATAX.png") no-repeat 50% 50%; } + .icos-SQOOP { + background: url("../img/toolbar_SQOOP.png") no-repeat 50% 50%; + } .toolbar { width: 60px; height: 100%; diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue index e05de8e880..177b252693 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue @@ -204,6 +204,13 @@ ref="DATAX" :backfill-item="backfillItem"> + +

    @@ -229,6 +236,7 @@ import mDependent from './tasks/dependent' import mHttp from './tasks/http' import mDatax from './tasks/datax' + import mSqoop from './tasks/sqoop' import mSubProcess from './tasks/sub_process' import mSelectInput from './_source/selectInput' import mTimeoutAlarm from './_source/timeoutAlarm' @@ -589,6 +597,7 @@ mDependent, mHttp, mDatax, + mSqoop, mSelectInput, mTimeoutAlarm, mPriority, diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/sqoop.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/sqoop.vue new file mode 100644 index 0000000000..eead745a06 --- /dev/null +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/sqoop.vue @@ -0,0 +1,981 @@ +/* + * 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. + */ + + + + diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/img/toolbar_SQOOP.png b/dolphinscheduler-ui/src/js/conf/home/pages/dag/img/toolbar_SQOOP.png new file mode 100644 index 0000000000000000000000000000000000000000..2ab3b6bd4a5c069a9a864431a68f37e4b49ed8fa GIT binary patch literal 1157 zcmV;01bX|4P)Px#1ZP1_K>z@;j|==^1poj532;bRa{vGi!vFvd!vV){sAK>D1R6<1K~zXf?UiXv zTvZguZ^JOvrlgu`z^F7)2EH^>T0>~mVrr>XQ9qCvV$-kJGt46`rKa~Bx8k%>*e`TOwR`JZ#{dH0@s z-+M^7XZ&CKCX%HGr-uE}026``hz6&E(@YQ?{_9$=feFIU^r+46VB!{V=~4IiX`?a} z3-42ZamL&Yo=&c(&wilt-xNSbg&a~y2WX8|J`kAWfAwz@+z;A|W}L36k{OUvY` zRzt_6R}T`GR_NPE2;&GiPaBjI9vzkaj=!`Xy^=diB?acuVOB9A2$6WS&wdf64^yEY zh5lE#qN$L3Y-XIgD7yweWESDPt-saK&IF;)cJWW!1tthbC*Sd z(}WC+rg6Dt&W5vy(>Tlhq&b#IBtSyw8HcZJK( zLbnhi)x=cDzO4n3T~LA|ju8?-C1vlk6ASq5)VY}E*4~dl+FMt@pQbc7{!m$6*{;GT zn#9wo-}HAUh=(K_ajy@5RY_u)ZEx%Z&pR(~0K2T>bCT?(A)x5_lI8fmBJZ(Pa4UV~ zCAjX`xeNP{2vS{E%laoC0r9r9o}Cb%O;`3>G08yL8(x1l50AX&qCsI(&I2H~?vN6d zAX(_D4KMP|7tl?Jl<>jo-G2BI(0IoUxfVb(1t z<%fT47#uN?h=7Lrl(LR~D&{u?733Gdb=&qm;NsLO>O^U)#&rCFPj*wawfqB@!@B0d zw2aJ0iG%fe0d%GN7zv;i_mMUtlv7+Tge}yjJeh?XQkY*z)H1nBJ*ie@5RtoXBYf5gGbrlC^&7D?y0Vk;{Fz8-wz{ocl?$_#A5OC)e@%W zx%pAI<*7^&l)hpNF`XJIm6oi;>%FW?{_j&ErP80)yj%KeslgdxIyF)%{olA}r3K+% XvR#xuLVs**/common/threadutils/*.java **/common/graph/*.java **/common/queue/*.java + **/common/task/SqoopParameterEntityTest.java **/api/utils/CheckUtilsTest.java **/api/utils/FileUtilsTest.java **/api/utils/FourLetterWordTest.java @@ -727,6 +728,7 @@ **/alert/template/AlertTemplateFactoryTest.java **/alert/template/impl/DefaultHTMLTemplateTest.java **/server/worker/task/datax/DataxTaskTest.java + **/server/worker/task/sqoop/SqoopTaskTest.java **/server/utils/DataxUtilsTest.java From 7a11d39f109008376d5f31678cf1f1a86cc05686 Mon Sep 17 00:00:00 2001 From: "xiaochun.liu" Date: Mon, 24 Feb 2020 20:11:27 +0800 Subject: [PATCH 029/221] docker for e2e --- .github/workflows/ci_e2e.yml | 53 ++++++++++++++++++++++++++++++++++++ dockerfile/README.md | 2 +- dockerfile/hooks/build | 15 ++++++++-- dockerfile/hooks/check | 26 ++++++++++++++++++ dockerfile/startup.sh | 4 ++- 5 files changed, 96 insertions(+), 4 deletions(-) create mode 100644 .github/workflows/ci_e2e.yml create mode 100644 dockerfile/hooks/check diff --git a/.github/workflows/ci_e2e.yml b/.github/workflows/ci_e2e.yml new file mode 100644 index 0000000000..0f867b60b3 --- /dev/null +++ b/.github/workflows/ci_e2e.yml @@ -0,0 +1,53 @@ +# +# 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. +# + +on: ["push", "pull_request"] +env: + DOCKER_DIR: ./docker + LOG_DIR: /tmp/dolphinscheduler + +name: e2e Test + +jobs: + + build: + name: Test + runs-on: ubuntu-latest + steps: + + - uses: actions/checkout@v1 + with: + submodules: true + - uses: actions/cache@v1 + with: + path: ~/.m2/repository + key: ${{ runner.os }}-maven-${{ hashFiles('**/pom.xml') }} + restore-keys: | + ${{ runner.os }}-maven- + - name: Build Image + run: export VERSION=latest && sh ./dockerfile/hooks/build + - name: Docker Run + run: docker run -dit -e POSTGRESQL_USERNAME=test -e POSTGRESQL_PASSWORD=test -p 8888:8888 dolphinscheduler all + - name: Check Server Status + run: sh ./dockerfile/hooks/check + - name: Run e2e Test + run: echo "Run e2e Test" + - name: Collect logs + run: | + mkdir -p ${LOG_DIR} + docker logs dolphinscheduler > ${LOG_DIR}/dolphinscheduler.txt + continue-on-error: true diff --git a/dockerfile/README.md b/dockerfile/README.md index fb1500b495..8259ce9a6f 100644 --- a/dockerfile/README.md +++ b/dockerfile/README.md @@ -12,7 +12,7 @@ Official Website: https://dolphinscheduler.apache.org #### You can start a dolphinscheduler instance ``` -$ docker run -d --name dolphinscheduler \ +$ docker run -dit --name dolphinscheduler \ -e POSTGRESQL_USERNAME=test -e POSTGRESQL_PASSWORD=test \ -p 8888:8888 \ dolphinscheduler all diff --git a/dockerfile/hooks/build b/dockerfile/hooks/build index 2394a8cd9c..32d2ee4e38 100644 --- a/dockerfile/hooks/build +++ b/dockerfile/hooks/build @@ -18,11 +18,22 @@ echo "------ dolphinscheduler start - build -------" printenv +if [[ "${VERSION}x" = "x" ]]; then + VERSION=`cat $(pwd)/pom.xml| grep "SNAPSHOT" | awk -F "-SNAPSHOT" '{print $1}' | awk -F ">" '{print $2}'` +fi + +if [[ "${DOCKER_REPO}x" = "x" ]]; then + DOCKER_REPO='dolphinscheduler' +fi + +echo "Version: $VERSION" +echo "Repo: $DOCKER_REPO" + echo -e "Current Directory is $(pwd)\n" # maven package(Project Directory) -echo -e "mvn clean compile package -Prelease" -mvn clean compile package -Prelease +echo -e "mvn -B clean compile package -Prelease -Dmaven.test.skip=true" +mvn -B clean compile package -Prelease -Dmaven.test.skip=true # mv dolphinscheduler-bin.tar.gz file to dockerfile directory echo -e "mv $(pwd)/dolphinscheduler-dist/target/apache-dolphinscheduler-incubating-${VERSION}-SNAPSHOT-dolphinscheduler-bin.tar.gz $(pwd)/dockerfile/\n" diff --git a/dockerfile/hooks/check b/dockerfile/hooks/check new file mode 100644 index 0000000000..1a6052cc79 --- /dev/null +++ b/dockerfile/hooks/check @@ -0,0 +1,26 @@ +#!/bin/bash +# +# 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. +# +echo "------ dolphinscheduler check - server - status -------" +server_num=$(docker top `docker container list | grep startup | awk '{print $1}'`| grep java | grep "dolphinscheduler" | awk -F 'classpath ' '{print $2}' | awk '{print $2}' | sort | uniq -c | wc -l) +if [ $server_num -eq 5 ] +then + echo "Server all start successfully" +else + echo "Server start failed "$server_num + exit 1 +fi diff --git a/dockerfile/startup.sh b/dockerfile/startup.sh index e52384d6b9..610a86f5d3 100644 --- a/dockerfile/startup.sh +++ b/dockerfile/startup.sh @@ -187,4 +187,6 @@ case "$1" in ;; esac -exec tee ${LOGFILE} \ No newline at end of file +echo "tee begin" +exec tee ${LOGFILE} + From 5aebdc8b26ae329216d70f6b5a4c188014eb0ce6 Mon Sep 17 00:00:00 2001 From: "xiaochun.liu" Date: Mon, 24 Feb 2020 20:43:31 +0800 Subject: [PATCH 030/221] docker github action --- .github/workflows/ci_e2e.yml | 8 ++++++-- dockerfile/hooks/build | 4 ---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/.github/workflows/ci_e2e.yml b/.github/workflows/ci_e2e.yml index 0f867b60b3..616f62d049 100644 --- a/.github/workflows/ci_e2e.yml +++ b/.github/workflows/ci_e2e.yml @@ -39,9 +39,13 @@ jobs: restore-keys: | ${{ runner.os }}-maven- - name: Build Image - run: export VERSION=latest && sh ./dockerfile/hooks/build + run: | + export VERSION=`cat $(pwd)/pom.xml| grep "SNAPSHOT" | awk -F "-SNAPSHOT" '{print $1}' | awk -F ">" '{print $2}'` + sh ./dockerfile/hooks/build - name: Docker Run - run: docker run -dit -e POSTGRESQL_USERNAME=test -e POSTGRESQL_PASSWORD=test -p 8888:8888 dolphinscheduler all + run: | + VERSION=`cat $(pwd)/pom.xml| grep "SNAPSHOT" | awk -F "-SNAPSHOT" '{print $1}' | awk -F ">" '{print $2}'` + docker run -dit -e POSTGRESQL_USERNAME=test -e POSTGRESQL_PASSWORD=test -p 8888:8888 dolphinscheduler:$VERSION all - name: Check Server Status run: sh ./dockerfile/hooks/check - name: Run e2e Test diff --git a/dockerfile/hooks/build b/dockerfile/hooks/build index 32d2ee4e38..ee173591af 100644 --- a/dockerfile/hooks/build +++ b/dockerfile/hooks/build @@ -18,10 +18,6 @@ echo "------ dolphinscheduler start - build -------" printenv -if [[ "${VERSION}x" = "x" ]]; then - VERSION=`cat $(pwd)/pom.xml| grep "SNAPSHOT" | awk -F "-SNAPSHOT" '{print $1}' | awk -F ">" '{print $2}'` -fi - if [[ "${DOCKER_REPO}x" = "x" ]]; then DOCKER_REPO='dolphinscheduler' fi From 8a362b5464b3950f5bf222f9db358d9f3956d9f2 Mon Sep 17 00:00:00 2001 From: "xiaochun.liu" Date: Mon, 24 Feb 2020 20:44:44 +0800 Subject: [PATCH 031/221] docker github action --- dockerfile/hooks/check | 1 + 1 file changed, 1 insertion(+) diff --git a/dockerfile/hooks/check b/dockerfile/hooks/check index 1a6052cc79..fbc3f8bd44 100644 --- a/dockerfile/hooks/check +++ b/dockerfile/hooks/check @@ -16,6 +16,7 @@ # limitations under the License. # echo "------ dolphinscheduler check - server - status -------" +sleep 20 server_num=$(docker top `docker container list | grep startup | awk '{print $1}'`| grep java | grep "dolphinscheduler" | awk -F 'classpath ' '{print $2}' | awk '{print $2}' | sort | uniq -c | wc -l) if [ $server_num -eq 5 ] then From e2cc538c6ec022cb13076a1992df6e992a22c1a9 Mon Sep 17 00:00:00 2001 From: "xiaochun.liu" Date: Mon, 24 Feb 2020 20:59:19 +0800 Subject: [PATCH 032/221] docker github action --- dockerfile/hooks/build | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dockerfile/hooks/build b/dockerfile/hooks/build index ee173591af..e4bb5a92b3 100644 --- a/dockerfile/hooks/build +++ b/dockerfile/hooks/build @@ -18,7 +18,8 @@ echo "------ dolphinscheduler start - build -------" printenv -if [[ "${DOCKER_REPO}x" = "x" ]]; then +if [ "${DOCKER_REPO}x" = "x" ] +then DOCKER_REPO='dolphinscheduler' fi From 5f3d88e4c269071bb88780ca3e318626a697d8e3 Mon Sep 17 00:00:00 2001 From: zhukai Date: Tue, 25 Feb 2020 09:47:51 +0800 Subject: [PATCH 033/221] Fix bug #1986. (#1987) When in the projects define page, click the toolbar's button(task type) will cause the task details window switch to new one. For now, the task details window will not switch. --- .../js/conf/home/pages/dag/_source/dag.vue | 3 ++- .../pages/dag/_source/plugIn/jsPlumbHandle.js | 19 ++++++++++--------- 2 files changed, 12 insertions(+), 10 deletions(-) diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.vue index 247f473bad..a21889f4ac 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.vue @@ -486,6 +486,7 @@ } this.taskId = id + type = type || self.dagBarId eventModel = this.$drawer({ closable: false, @@ -522,7 +523,7 @@ }, props: { id: id, - taskType: type || self.dagBarId, + taskType: type, self: self } }) diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/plugIn/jsPlumbHandle.js b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/plugIn/jsPlumbHandle.js index 454dfc7e03..b0a7a64a47 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/plugIn/jsPlumbHandle.js +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/plugIn/jsPlumbHandle.js @@ -135,15 +135,6 @@ JSP.prototype.draggable = function () { helper: 'clone', containment: $('.dag-model'), stop: function (e, ui) { - self.tasksEvent(selfId) - - // Dom structure is not generated without pop-up form form - if ($(`#${selfId}`).html()) { - // dag event - findComponentDownward(self.dag.$root, 'dag-chart')._createNodes({ - id: selfId - }) - } }, drag: function () { $('body').find('.tooltip.fade.top.in').remove() @@ -178,6 +169,16 @@ JSP.prototype.draggable = function () { self.initNode(thisDom[thisDom.length - 1]) }) selfId = id + + self.tasksEvent(selfId) + + // Dom structure is not generated without pop-up form form + if ($(`#${selfId}`).html()) { + // dag event + findComponentDownward(self.dag.$root, 'dag-chart')._createNodes({ + id: selfId + }) + } } }) } From b1624fde71c2293dbbe3b119db6c157c6f275fb1 Mon Sep 17 00:00:00 2001 From: dailidong Date: Tue, 25 Feb 2020 14:00:26 +0800 Subject: [PATCH 034/221] update English Readme about image update (#2013) update English Readme about image update --- README.md | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/README.md b/README.md index 3fbd6345b6..883f3fdd14 100644 --- a/README.md +++ b/README.md @@ -45,17 +45,17 @@ HA is supported by itself | All process definition operations are visualized, dr Overload processing: Task queue mechanism, the number of schedulable tasks on a single machine can be flexibly configured, when too many tasks will be cached in the task queue, will not cause machine jam. | One-click deployment | Supports traditional shell tasks, and also support big data platform task scheduling: MR, Spark, SQL (mysql, postgresql, hive, sparksql), Python, Procedure, Sub_Process | | - - ### System partial screenshot -![image](https://user-images.githubusercontent.com/48329107/61368744-1f5f3b00-a8c1-11e9-9cf1-10f8557a6b3b.png) - -![image](https://user-images.githubusercontent.com/48329107/61368966-9dbbdd00-a8c1-11e9-8dcc-a9469d33583e.png) - -![image](https://user-images.githubusercontent.com/48329107/61372146-f347b800-a8c8-11e9-8882-66e8934ada23.png) - - +![home page](https://user-images.githubusercontent.com/15833811/75218288-bf286400-57d4-11ea-8263-d639c6511d5f.jpg) +![dag](https://user-images.githubusercontent.com/15833811/75216789-25f74e80-57d0-11ea-8d9f-ad64a3db473c.png) +![process definition list page](https://user-images.githubusercontent.com/15833811/75216886-6f479e00-57d0-11ea-92dd-66e7640a186f.png) +![view task log online](https://user-images.githubusercontent.com/15833811/75216924-9900c500-57d0-11ea-91dc-3522a76bdbbe.png) +![resource management](https://user-images.githubusercontent.com/15833811/75216984-be8dce80-57d0-11ea-840d-58546edc8788.png) +![datasource management](https://user-images.githubusercontent.com/15833811/75217041-cfd6db00-57d0-11ea-8451-8b376d0dc8f1.png) +![monitor](https://user-images.githubusercontent.com/15833811/75217091-fac12f00-57d0-11ea-9665-d8dc19813707.png) +![security](https://user-images.githubusercontent.com/15833811/75217157-29d7a080-57d1-11ea-91ca-1fa8af936428.png) +![treeview](https://user-images.githubusercontent.com/15833811/75217191-3fe56100-57d1-11ea-8856-f19180d9a879.png) ### Document - Backend deployment documentation From 1ce6f356255a08a640a01b486510b8a42cc39670 Mon Sep 17 00:00:00 2001 From: dailidong Date: Tue, 25 Feb 2020 14:01:31 +0800 Subject: [PATCH 035/221] Update README_zh_CN.md (#2011) update image url --- README_zh_CN.md | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/README_zh_CN.md b/README_zh_CN.md index e782c1030d..6a4adc8daa 100644 --- a/README_zh_CN.md +++ b/README_zh_CN.md @@ -36,11 +36,19 @@ Dolphin Scheduler Official Website ### 系统部分截图 -![](http://geek.analysys.cn/static/upload/221/2019-03-29/0a9dea80-fb02-4fa5-a812-633b67035ffc.jpeg) +![home page](https://user-images.githubusercontent.com/15833811/75208819-abbad000-57b7-11ea-8d3c-67e7c270671f.jpg) -![](http://geek.analysys.cn/static/upload/221/2019-04-01/83686def-a54f-4169-8cae-77b1f8300cc1.png) +![dag](https://user-images.githubusercontent.com/15833811/75209584-93e44b80-57b9-11ea-952e-537fb24ec72d.jpg) -![](http://geek.analysys.cn/static/upload/221/2019-03-29/83c937c7-1793-4d7a-aa28-b98460329fe0.jpeg) +![log](https://user-images.githubusercontent.com/15833811/75209645-c55d1700-57b9-11ea-94d4-e3fa91ab5218.jpg) + +![gantt](https://user-images.githubusercontent.com/15833811/75209640-c0986300-57b9-11ea-878e-a2098533ad44.jpg) + +![resources](https://user-images.githubusercontent.com/15833811/75209403-11f42280-57b9-11ea-9b59-d4be77063553.jpg) + +![monitor](https://user-images.githubusercontent.com/15833811/75209631-b5ddce00-57b9-11ea-8d22-cdf15cf0ee25.jpg) + +![security](https://user-images.githubusercontent.com/15833811/75209633-baa28200-57b9-11ea-9def-94bef2e212a7.jpg) ### 文档 From 205b302634aa9dba4bc9173b67febb8be81a754c Mon Sep 17 00:00:00 2001 From: liwenhe1993 <32166572+liwenhe1993@users.noreply.github.com> Date: Tue, 25 Feb 2020 14:17:59 +0800 Subject: [PATCH 036/221] Modify dockerfile build script and add a new README_zh_CN.md file for dockerfile (#2014) * Support DS to create user and group in windows environment * Add unit test * delete combined-server config in dolphinscheduler-daemon.sh file * refactor dockerfile * modify dockerfile * modify dockerfile * 1. Modify dockerfile build script 2. Add something about how to build a docker image 3. Add a new README_zh_CN.md file for dockerfile --- dockerfile/README.md | 38 ++++- dockerfile/README_zh_CN.md | 306 +++++++++++++++++++++++++++++++++++++ dockerfile/hooks/build | 10 ++ dockerfile/hooks/build.bat | 25 ++- dockerfile/hooks/push.bat | 1 + 5 files changed, 371 insertions(+), 9 deletions(-) create mode 100644 dockerfile/README_zh_CN.md diff --git a/dockerfile/README.md b/dockerfile/README.md index 8259ce9a6f..b5a9d0d3aa 100644 --- a/dockerfile/README.md +++ b/dockerfile/README.md @@ -8,7 +8,10 @@ Official Website: https://dolphinscheduler.apache.org ![Dolphin Scheduler](https://dolphinscheduler.apache.org/img/hlogo_colorful.svg) -## How to use this image +[![EN doc](https://img.shields.io/badge/document-English-blue.svg)](README.md) +[![CN doc](https://img.shields.io/badge/文档-中文版-blue.svg)](README_zh_CN.md) + +## How to use this docker image #### You can start a dolphinscheduler instance ``` @@ -27,7 +30,7 @@ The default zookeeper is created in the `startup.sh`. You can specify **existing postgres service**. Example: ``` -$ docker run -d --name dolphinscheduler \ +$ docker run -dit --name dolphinscheduler \ -e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" \ -e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ -p 8888:8888 \ @@ -37,7 +40,7 @@ dolphinscheduler all You can specify **existing zookeeper service**. Example: ``` -$ docker run -d --name dolphinscheduler \ +$ docker run -dit --name dolphinscheduler \ -e ZOOKEEPER_QUORUM="l92.168.x.x:2181" -e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ -p 8888:8888 \ @@ -51,7 +54,7 @@ You can start a standalone dolphinscheduler server. * Start a **master server**, For example: ``` -$ docker run -d --name dolphinscheduler \ +$ docker run -dit --name dolphinscheduler \ -e ZOOKEEPER_QUORUM="l92.168.x.x:2181" -e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" \ -e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ @@ -61,7 +64,7 @@ dolphinscheduler master-server * Start a **worker server**, For example: ``` -$ docker run -d --name dolphinscheduler \ +$ docker run -dit --name dolphinscheduler \ -e ZOOKEEPER_QUORUM="l92.168.x.x:2181" -e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" \ -e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ @@ -71,7 +74,7 @@ dolphinscheduler worker-server * Start a **api server**, For example: ``` -$ docker run -d --name dolphinscheduler \ +$ docker run -dit --name dolphinscheduler \ -e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" \ -e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ -p 12345:12345 \ @@ -81,7 +84,7 @@ dolphinscheduler api-server * Start a **alert server**, For example: ``` -$ docker run -d --name dolphinscheduler \ +$ docker run -dit --name dolphinscheduler \ -e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" \ -e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ dolphinscheduler alert-server @@ -90,7 +93,7 @@ dolphinscheduler alert-server * Start a **frontend**, For example: ``` -$ docker run -d --name dolphinscheduler \ +$ docker run -dit --name dolphinscheduler \ -e FRONTEND_API_SERVER_HOST="192.168.x.x" -e FRONTEND_API_SERVER_PORT="12345" \ -p 8888:8888 \ dolphinscheduler frontend @@ -98,6 +101,25 @@ dolphinscheduler frontend **Note**: You must be specify `POSTGRESQL_HOST` `POSTGRESQL_PORT` `ZOOKEEPER_QUORUM` when start a standalone dolphinscheduler server. +## How to build a docker image + +You can build a docker image in A Unix-like operating system, You can also build it in Windows operating system. + +In Unix-Like, Example: + +```bash +$ cd path/incubator-dolphinscheduler +$ sh ./dockerfile/hooks/build +``` + +In Windows, Example: + +```bat +c:\incubator-dolphinscheduler>.\dockerfile\hooks\build.bat +``` + +Please read `./dockerfile/hooks/build` `./dockerfile/hooks/build.bat` script files if you don't understand + ## Environment Variables The Dolphin Scheduler image uses several environment variables which are easy to miss. While none of the variables are required, they may significantly aid you in using the image. diff --git a/dockerfile/README_zh_CN.md b/dockerfile/README_zh_CN.md new file mode 100644 index 0000000000..5830e30b07 --- /dev/null +++ b/dockerfile/README_zh_CN.md @@ -0,0 +1,306 @@ +## Dolphin Scheduler是什么? + +一个分布式易扩展的可视化DAG工作流任务调度系统。致力于解决数据处理流程中错综复杂的依赖关系,使调度系统在数据处理流程中`开箱即用`。 + +Github URL: https://github.com/apache/incubator-dolphinscheduler + +Official Website: https://dolphinscheduler.apache.org + +![Dolphin Scheduler](https://dolphinscheduler.apache.org/img/hlogo_colorful.svg) + +[![EN doc](https://img.shields.io/badge/document-English-blue.svg)](README.md) +[![CN doc](https://img.shields.io/badge/文档-中文版-blue.svg)](README_zh_CN.md) + +## 如何使用docker镜像 + +#### 你可以运行一个dolphinscheduler实例 +``` +$ docker run -dit --name dolphinscheduler \ +-e POSTGRESQL_USERNAME=test -e POSTGRESQL_PASSWORD=test \ +-p 8888:8888 \ +dolphinscheduler all +``` + +在`startup.sh`脚本中,默认的创建`Postgres`的用户、密码和数据库,默认值分别为:`root`、`root`、`dolphinscheduler`。 + +同时,默认的`Zookeeper`也会在`startup.sh`脚本中被创建。 + +#### 或者通过环境变量 **`POSTGRESQL_HOST`** **`POSTGRESQL_PORT`** **`ZOOKEEPER_QUORUM`** 使用已存在的服务 + +你可以指定一个已经存在的 **`Postgres`** 服务. 如下: + +``` +$ docker run -dit --name dolphinscheduler \ +-e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" \ +-e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ +-p 8888:8888 \ +dolphinscheduler all +``` + +你也可以指定一个已经存在的 **Zookeeper** 服务. 如下: + +``` +$ docker run -dit --name dolphinscheduler \ +-e ZOOKEEPER_QUORUM="l92.168.x.x:2181" +-e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ +-p 8888:8888 \ +dolphinscheduler all +``` + +#### 或者运行dolphinscheduler中的部分服务 + +你能够运行dolphinscheduler中的部分服务。 + +* 启动一个 **master server**, 如下: + +``` +$ docker run -dit --name dolphinscheduler \ +-e ZOOKEEPER_QUORUM="l92.168.x.x:2181" +-e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" \ +-e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ +dolphinscheduler master-server +``` + +* 启动一个 **worker server**, 如下: + +``` +$ docker run -dit --name dolphinscheduler \ +-e ZOOKEEPER_QUORUM="l92.168.x.x:2181" +-e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" \ +-e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ +dolphinscheduler worker-server +``` + +* 启动一个 **api server**, 如下: + +``` +$ docker run -dit --name dolphinscheduler \ +-e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" \ +-e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ +-p 12345:12345 \ +dolphinscheduler api-server +``` + +* 启动一个 **alert server**, 如下: + +``` +$ docker run -dit --name dolphinscheduler \ +-e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" \ +-e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ +dolphinscheduler alert-server +``` + +* 启动一个 **frontend**, 如下: + +``` +$ docker run -dit --name dolphinscheduler \ +-e FRONTEND_API_SERVER_HOST="192.168.x.x" -e FRONTEND_API_SERVER_PORT="12345" \ +-p 8888:8888 \ +dolphinscheduler frontend +``` + +**注意**: 当你运行dolphinscheduler中的部分服务时,你必须指定这些环境变量 `POSTGRESQL_HOST` `POSTGRESQL_PORT` `ZOOKEEPER_QUORUM`。 + +## 如何构建一个docker镜像 + +你能够在类Unix系统和Windows系统中构建一个docker镜像。 + +类Unix系统, 如下: + +```bash +$ cd path/incubator-dolphinscheduler +$ sh ./dockerfile/hooks/build +``` + +Windows系统, 如下: + +```bat +c:\incubator-dolphinscheduler>.\dockerfile\hooks\build.bat +``` + +如果你不理解这些脚本 `./dockerfile/hooks/build` `./dockerfile/hooks/build.bat`,请阅读里面的内容。 + +## 环境变量 + +Dolphin Scheduler映像使用了几个容易遗漏的环境变量。虽然这些变量不是必须的,但是可以帮助你更容易配置镜像并根据你的需求定义相应的服务配置。 + +**`POSTGRESQL_HOST`** + +配置`PostgreSQL`的`HOST`, 默认值 `127.0.0.1`。 + +**注意**: 当运行`dolphinscheduler`中`master-server`、`worker-server`、`api-server`、`alert-server`这些服务时,必须指定这个环境变量,以便于你更好的搭建分布式服务。 + +**`POSTGRESQL_PORT`** + +配置`PostgreSQL`的`PORT`, 默认值 `5432`。 + +**注意**: 当运行`dolphinscheduler`中`master-server`、`worker-server`、`api-server`、`alert-server`这些服务时,必须指定这个环境变量,以便于你更好的搭建分布式服务。 + +**`POSTGRESQL_USERNAME`** + +配置`PostgreSQL`的`USERNAME`, 默认值 `root`。 + +**`POSTGRESQL_PASSWORD`** + +配置`PostgreSQL`的`PASSWORD`, 默认值 `root`。 + +**`DOLPHINSCHEDULER_ENV_PATH`** + +任务执行时的环境变量配置文件, 默认值 `/opt/dolphinscheduler/conf/env/dolphinscheduler_env.sh`。 + +**`TASK_QUEUE`** + +配置`master-server`和`worker-serverr`的`Zookeeper`任务队列名, 默认值 `zookeeper`。 + +**`ZOOKEEPER_QUORUM`** + +配置`master-server`和`worker-serverr`的`Zookeeper`地址, 默认值 `127.0.0.1:2181`。 + +**注意**: 当运行`dolphinscheduler`中`master-server`、`worker-server`这些服务时,必须指定这个环境变量,以便于你更好的搭建分布式服务。 + +**`MASTER_EXEC_THREADS`** + +配置`master-server`中的执行线程数量,默认值 `100`。 + +**`MASTER_EXEC_TASK_NUM`** + +配置`master-server`中的执行任务数量,默认值 `20`。 + +**`MASTER_HEARTBEAT_INTERVAL`** + +配置`master-server`中的心跳交互时间,默认值 `10`。 + +**`MASTER_TASK_COMMIT_RETRYTIMES`** + +配置`master-server`中的任务提交重试次数,默认值 `5`。 + +**`MASTER_TASK_COMMIT_INTERVAL`** + +配置`master-server`中的任务提交交互时间,默认值 `1000`。 + +**`MASTER_MAX_CPULOAD_AVG`** + +配置`master-server`中的CPU中的`load average`值,默认值 `100`。 + +**`MASTER_RESERVED_MEMORY`** + +配置`master-server`的保留内存,默认值 `0.1`。 + +**`WORKER_EXEC_THREADS`** + +配置`worker-server`中的执行线程数量,默认值 `100`。 + +**`WORKER_HEARTBEAT_INTERVAL`** + +配置`worker-server`中的心跳交互时间,默认值 `10`。 + +**`WORKER_FETCH_TASK_NUM`** + +配置`worker-server`中的获取任务的数量,默认值 `3`。 + +**`WORKER_MAX_CPULOAD_AVG`** + +配置`worker-server`中的CPU中的最大`load average`值,默认值 `100`。 + +**`WORKER_RESERVED_MEMORY`** + +配置`worker-server`的保留内存,默认值 `0.1`。 + +**`XLS_FILE_PATH`** + +配置`alert-server`的`XLS`文件的存储路径,默认值 `/tmp/xls`。 + +**`MAIL_SERVER_HOST`** + +配置`alert-server`的邮件服务地址,默认值 `空`。 + +**`MAIL_SERVER_PORT`** + +配置`alert-server`的邮件服务端口,默认值 `空`。 + +**`MAIL_SENDER`** + +配置`alert-server`的邮件发送人,默认值 `空`。 + +**`MAIL_USER=`** + +配置`alert-server`的邮件服务用户名,默认值 `空`。 + +**`MAIL_PASSWD`** + +配置`alert-server`的邮件服务用户密码,默认值 `空`。 + +**`MAIL_SMTP_STARTTLS_ENABLE`** + +配置`alert-server`的邮件服务是否启用TLS,默认值 `true`。 + +**`MAIL_SMTP_SSL_ENABLE`** + +配置`alert-server`的邮件服务是否启用SSL,默认值 `false`。 + +**`MAIL_SMTP_SSL_TRUST`** + +配置`alert-server`的邮件服务SSL的信任地址,默认值 `空`。 + +**`ENTERPRISE_WECHAT_ENABLE`** + +配置`alert-server`的邮件服务是否启用企业微信,默认值 `false`。 + +**`ENTERPRISE_WECHAT_CORP_ID`** + +配置`alert-server`的邮件服务企业微信`ID`,默认值 `空`。 + +**`ENTERPRISE_WECHAT_SECRET`** + +配置`alert-server`的邮件服务企业微信`SECRET`,默认值 `空`。 + +**`ENTERPRISE_WECHAT_AGENT_ID`** + +配置`alert-server`的邮件服务企业微信`AGENT_ID`,默认值 `空`。 + +**`ENTERPRISE_WECHAT_USERS`** + +配置`alert-server`的邮件服务企业微信`USERS`,默认值 `空`。 + +**`FRONTEND_API_SERVER_HOST`** + +配置`frontend`的连接`api-server`的地址,默认值 `127.0.0.1`。 + +**Note**: 当单独运行`api-server`时,你应该指定`api-server`这个值。 + +**`FRONTEND_API_SERVER_PORT`** + +配置`frontend`的连接`api-server`的端口,默认值 `12345`。 + +**Note**: 当单独运行`api-server`时,你应该指定`api-server`这个值。 + +## 初始化脚本 + +如果你想在编译的时候或者运行的时候附加一些其它的操作及新增一些环境变量,你可以在`/root/start-init-conf.sh`文件中进行修改,同时如果涉及到配置文件的修改,请在`/opt/dolphinscheduler/conf/*.tpl`中修改相应的配置文件 + +例如,在`/root/start-init-conf.sh`添加一个环境变量`API_SERVER_PORT`: + +``` +export API_SERVER_PORT=5555 +``` + +当添加以上环境变量后,你应该在相应的模板文件`/opt/dolphinscheduler/conf/application-api.properties.tpl`中添加这个环境变量配置: +``` +server.port=${API_SERVER_PORT} +``` + +`/root/start-init-conf.sh`将根据模板文件动态的生成配置文件: + +```sh +echo "generate app config" +ls ${DOLPHINSCHEDULER_HOME}/conf/ | grep ".tpl" | while read line; do +eval "cat << EOF +$(cat ${DOLPHINSCHEDULER_HOME}/conf/${line}) +EOF +" > ${DOLPHINSCHEDULER_HOME}/conf/${line%.*} +done + +echo "generate nginx config" +sed -i "s/FRONTEND_API_SERVER_HOST/${FRONTEND_API_SERVER_HOST}/g" /etc/nginx/conf.d/dolphinscheduler.conf +sed -i "s/FRONTEND_API_SERVER_PORT/${FRONTEND_API_SERVER_PORT}/g" /etc/nginx/conf.d/dolphinscheduler.conf +``` diff --git a/dockerfile/hooks/build b/dockerfile/hooks/build index e4bb5a92b3..05fa09d0c9 100644 --- a/dockerfile/hooks/build +++ b/dockerfile/hooks/build @@ -15,11 +15,21 @@ # See the License for the specific language governing permissions and # limitations under the License. # + +set -e + echo "------ dolphinscheduler start - build -------" printenv +if [ -z "${VERSION}" ] +then + echo "set default environment variable [VERSION]" + VERSION=$(cat $(pwd)/sql/soft_version) +fi + if [ "${DOCKER_REPO}x" = "x" ] then + echo "set default environment variable [DOCKER_REPO]" DOCKER_REPO='dolphinscheduler' fi diff --git a/dockerfile/hooks/build.bat b/dockerfile/hooks/build.bat index 33a6c23ecf..b15c7b00df 100644 --- a/dockerfile/hooks/build.bat +++ b/dockerfile/hooks/build.bat @@ -13,14 +13,30 @@ :: See the License for the specific language governing permissions and :: limitations under the License. :: +@echo off + echo "------ dolphinscheduler start - build -------" set +if not defined VERSION ( + echo "set environment variable [VERSION]" + for /f %%l in (%cd%\sql\soft_version) do (set VERSION=%%l) +) + +if not defined DOCKER_REPO ( + echo "set environment variable [DOCKER_REPO]" + set DOCKER_REPO='dolphinscheduler' +) + +echo "Version: %VERSION%" +echo "Repo: %DOCKER_REPO%" + echo "Current Directory is %cd%" :: maven package(Project Directory) echo "call mvn clean compile package -Prelease" call mvn clean compile package -Prelease -DskipTests=true +if "%errorlevel%"=="1" goto :mvnFailed :: move dolphinscheduler-bin.tar.gz file to dockerfile directory echo "move %cd%\dolphinscheduler-dist\target\apache-dolphinscheduler-incubating-%VERSION%-SNAPSHOT-dolphinscheduler-bin.tar.gz %cd%\dockerfile\" @@ -29,5 +45,12 @@ move %cd%\dolphinscheduler-dist\target\apache-dolphinscheduler-incubating-%VERSI :: docker build echo "docker build --build-arg VERSION=%VERSION% -t %DOCKER_REPO%:%VERSION% %cd%\dockerfile\" docker build --build-arg VERSION=%VERSION% -t %DOCKER_REPO%:%VERSION% %cd%\dockerfile\ +if "%errorlevel%"=="1" goto :dockerBuildFailed + +echo "------ dolphinscheduler end - build -------" + +:mvnFailed +echo "MAVEN PACKAGE FAILED!" -echo "------ dolphinscheduler end - build -------" \ No newline at end of file +:dockerBuildFailed +echo "DOCKER BUILD FAILED!" \ No newline at end of file diff --git a/dockerfile/hooks/push.bat b/dockerfile/hooks/push.bat index ee0c37bd37..458a693f97 100644 --- a/dockerfile/hooks/push.bat +++ b/dockerfile/hooks/push.bat @@ -13,6 +13,7 @@ :: See the License for the specific language governing permissions and :: limitations under the License. :: +@echo off echo "------ push start -------" set From 5ca4862ea97a98052a4591719ae622f553a3a893 Mon Sep 17 00:00:00 2001 From: "gabry.wu" Date: Tue, 25 Feb 2020 14:19:26 +0800 Subject: [PATCH 037/221] remove String.format for org.slf4j.Logger (#2001) --- .../api/service/ExecutorService.java | 2 +- .../api/service/ResourcesService.java | 2 +- .../common/utils/ParameterUtils.java | 2 +- .../common/utils/placeholder/PlaceholderUtils.java | 2 +- .../utils/placeholder/TimePlaceholderUtils.java | 2 +- .../common/threadutils/ThreadPoolExecutorsTest.java | 2 +- .../server/master/runner/MasterSchedulerThread.java | 2 +- .../server/worker/task/AbstractCommandExecutor.java | 6 +++--- .../service/process/ProcessService.java | 12 ++++++------ .../service/quartz/QuartzExecutors.java | 4 ++-- .../service/queue/TaskQueueZkImpl.java | 4 ++-- 11 files changed, 20 insertions(+), 20 deletions(-) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java index 0389890691..152292a21b 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java @@ -242,7 +242,7 @@ public class ExecutorService extends BaseService{ } break; default: - logger.error(String.format("unknown execute type : %s", executeType.toString())); + logger.error("unknown execute type : {}", executeType); putMsg(result, Status.REQUEST_PARAMS_NOT_VALID_ERROR, "unknown execute type"); break; diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ResourcesService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ResourcesService.java index 6438e206f8..c1adb8874d 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ResourcesService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ResourcesService.java @@ -547,7 +547,7 @@ public class ResourcesService extends BaseService { } } catch (Exception e) { - logger.error(String.format("Resource %s read failed", hdfsFileName), e); + logger.error("Resource {} read failed", hdfsFileName, e); putMsg(result, Status.HDFS_OPERATION_ERROR); } diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/ParameterUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/ParameterUtils.java index a6dd53ea22..7f2888384e 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/ParameterUtils.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/ParameterUtils.java @@ -61,7 +61,7 @@ public class ParameterUtils { try { cronTime = DateUtils.parseDate(cronTimeStr, new String[]{Constants.PARAMETER_FORMAT_TIME}); } catch (ParseException e) { - logger.error(String.format("parse %s exception", cronTimeStr), e); + logger.error("parse {} exception", cronTimeStr, e); } } else { cronTime = new Date(); diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/placeholder/PlaceholderUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/placeholder/PlaceholderUtils.java index b52707bfde..0c756cb0b3 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/placeholder/PlaceholderUtils.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/placeholder/PlaceholderUtils.java @@ -90,7 +90,7 @@ public class PlaceholderUtils { try { return paramsMap.get(placeholderName); } catch (Exception ex) { - logger.error(String.format("resolve placeholder '%s' in [ %s ]" , placeholderName, value), ex); + logger.error("resolve placeholder '{}' in [ {} ]" , placeholderName, value, ex); return null; } } diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/placeholder/TimePlaceholderUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/placeholder/TimePlaceholderUtils.java index 1751df53c6..15e3282d38 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/placeholder/TimePlaceholderUtils.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/placeholder/TimePlaceholderUtils.java @@ -278,7 +278,7 @@ public class TimePlaceholderUtils { try { return calculateTime(placeholderName, date); } catch (Exception ex) { - logger.error(String.format("resolve placeholder '%s' in [ %s ]" , placeholderName, value), ex); + logger.error("resolve placeholder '{}' in [ {} ]" , placeholderName, value, ex); return null; } } diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/threadutils/ThreadPoolExecutorsTest.java b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/threadutils/ThreadPoolExecutorsTest.java index 265f7eabcd..9879154889 100644 --- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/threadutils/ThreadPoolExecutorsTest.java +++ b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/threadutils/ThreadPoolExecutorsTest.java @@ -48,7 +48,7 @@ public class ThreadPoolExecutorsTest { class Thread2 extends Thread { @Override public void run() { - logger.info(String.format("ThreadPoolExecutors instance's hashcode is: %s ",ThreadPoolExecutors.getInstance("a",2).hashCode())); + logger.info("ThreadPoolExecutors instance's hashcode is: {} ",ThreadPoolExecutors.getInstance("a",2).hashCode()); } } diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerThread.java index c0ddb1cb5c..cc5a7e76e4 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerThread.java @@ -117,7 +117,7 @@ public class MasterSchedulerThread implements Runnable { // make sure to scan and delete command table in one transaction Command command = processService.findOneCommand(); if (command != null) { - logger.info(String.format("find one command: id: %d, type: %s", command.getId(),command.getCommandType().toString())); + logger.info("find one command: id: {}, type: {}", command.getId(),command.getCommandType()); try{ processInstance = processService.handleCommand(logger, OSUtils.getHost(), this.masterExecThreadNum - activeCount, command); diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractCommandExecutor.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractCommandExecutor.java index c473f3a2aa..bac498c150 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractCommandExecutor.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractCommandExecutor.java @@ -176,7 +176,7 @@ public abstract class AbstractCommandExecutor { } catch (InterruptedException e) { exitStatusCode = -1; - logger.error(String.format("interrupt exception: {}, task may be cancelled or killed",e.getMessage()), e); + logger.error("interrupt exception: {}, task may be cancelled or killed", e.getMessage(), e); throw new RuntimeException("interrupt exception. exitCode is : " + exitStatusCode); } catch (Exception e) { exitStatusCode = -1; @@ -408,7 +408,7 @@ public abstract class AbstractCommandExecutor { } } } catch (Exception e) { - logger.error(String.format("yarn applications: %s status failed ", appIds.toString()),e); + logger.error("yarn applications: {} status failed ", appIds,e); result = false; } return result; @@ -458,7 +458,7 @@ public abstract class AbstractCommandExecutor { lineList.add(line); } } catch (Exception e) { - logger.error(String.format("read file: %s failed : ",filename),e); + logger.error("read file: {} failed",filename,e); } finally { if(br != null){ try { diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java index 34b1e48d63..b589cd4295 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java @@ -477,7 +477,7 @@ public class ProcessService { if(cmdParam == null || !cmdParam.containsKey(Constants.CMDPARAM_START_NODE_NAMES) || cmdParam.get(Constants.CMDPARAM_START_NODE_NAMES).isEmpty()){ - logger.error(String.format("command node depend type is %s, but start nodes is null ", command.getTaskDependType().toString())); + logger.error("command node depend type is {}, but start nodes is null ", command.getTaskDependType()); return false; } } @@ -500,7 +500,7 @@ public class ProcessService { if(command.getProcessDefinitionId() != 0){ processDefinition = processDefineMapper.selectById(command.getProcessDefinitionId()); if(processDefinition == null){ - logger.error(String.format("cannot find the work process define! define id : %d", command.getProcessDefinitionId())); + logger.error("cannot find the work process define! define id : {}", command.getProcessDefinitionId()); return null; } } @@ -973,21 +973,21 @@ public class ProcessService { return true; } if(taskInstance.getState().typeIsFinished()){ - logger.info(String.format("submit to task queue, but task [%s] state [%s] is already finished. ", taskInstance.getName(), taskInstance.getState().toString())); + logger.info("submit to task queue, but task [{}] state [{}] is already finished. ", taskInstance.getName(), taskInstance.getState()); return true; } // task cannot submit when running if(taskInstance.getState() == ExecutionStatus.RUNNING_EXEUTION){ - logger.info(String.format("submit to task queue, but task [%s] state already be running. ", taskInstance.getName())); + logger.info("submit to task queue, but task [{}] state already be running. ", taskInstance.getName()); return true; } if(checkTaskExistsInTaskQueue(taskInstance)){ - logger.info(String.format("submit to task queue, but task [%s] already exists in the queue.", taskInstance.getName())); + logger.info("submit to task queue, but task [{}] already exists in the queue.", taskInstance.getName()); return true; } logger.info("task ready to queue: {}" , taskInstance); boolean insertQueueResult = taskQueue.add(DOLPHINSCHEDULER_TASKS_QUEUE, taskZkInfo(taskInstance)); - logger.info(String.format("master insert into queue success, task : %s", taskInstance.getName()) ); + logger.info("master insert into queue success, task : {}", taskInstance.getName()); return insertQueueResult; }catch (Exception e){ logger.error("submit task to queue Exception: ", e); diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/QuartzExecutors.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/QuartzExecutors.java index 9d96264a60..60cdb1dd97 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/QuartzExecutors.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/QuartzExecutors.java @@ -223,7 +223,7 @@ public class QuartzExecutors { } } catch (SchedulerException e) { - logger.error(String.format("delete job : %s failed",jobName), e); + logger.error("delete job : {} failed",jobName, e); } finally { lock.writeLock().unlock(); } @@ -247,7 +247,7 @@ public class QuartzExecutors { return scheduler.deleteJobs(jobKeys); } catch (SchedulerException e) { - logger.error(String.format("delete all jobs in job group: %s failed",jobGroupName), e); + logger.error("delete all jobs in job group: {} failed",jobGroupName, e); } finally { lock.writeLock().unlock(); } diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/TaskQueueZkImpl.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/TaskQueueZkImpl.java index 874512c361..9c1d318ea5 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/TaskQueueZkImpl.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/queue/TaskQueueZkImpl.java @@ -269,7 +269,7 @@ public class TaskQueueZkImpl implements ITaskQueue { zookeeperOperator.remove(taskIdPath); }catch(Exception e){ - logger.error(String.format("delete task:%s from zookeeper fail, exception:" ,nodeValue) ,e); + logger.error("delete task:{} from zookeeper fail, exception:" ,nodeValue ,e); } } @@ -318,7 +318,7 @@ public class TaskQueueZkImpl implements ITaskQueue { zookeeperOperator.remove(path + value); }catch(Exception e){ - logger.error(String.format("delete task:" + value + " exception"),e); + logger.error("delete task:{} exception",value,e); } } From 1cadb1a2511174f59c28ebffb6223989865c3c19 Mon Sep 17 00:00:00 2001 From: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> Date: Wed, 26 Feb 2020 11:08:55 +0800 Subject: [PATCH 038/221] Add e2e automated testing (#2005) * add ui-test * add e2e license and notice * e2e delete dolphinscheduler parent project dependency * pom.xml delete e2e dependency Co-authored-by: chenxingchun <50446296+chenxingchun@users.noreply.github.com> --- dolphinscheduler-dist/pom.xml | 2 - e2e/pom.xml | 137 +++++++ .../constant/TestConstant.java | 37 ++ .../util/PropertiesReader.java | 49 +++ .../dolphinscheduler/util/RedisUtil.java | 204 ++++++++++ .../dolphinscheduler/base/BaseDriver.java | 133 +++++++ .../dolphinscheduler/base/BaseTest.java | 107 ++++++ .../common/BrowserCommon.java | 353 ++++++++++++++++++ .../dolphinscheduler/common/PageCommon.java | 50 +++ .../dolphinscheduler/data/LoginData.java | 43 +++ .../data/project/CreatProjectData.java | 26 ++ .../data/project/CreatWorkflowData.java | 36 ++ .../data/security/TenantManageData.java | 55 +++ .../data/security/UserManageData.java | 34 ++ .../locator/LoginLocator.java | 33 ++ .../locator/project/CreateProjectLocator.java | 36 ++ .../project/CreateWorkflowLocator.java | 97 +++++ .../locator/security/TenantManageLocator.java | 33 ++ .../locator/security/UserManageLocator.java | 45 +++ .../dolphinscheduler/page/LoginPage.java | 65 ++++ .../page/project/CreateProjectPage.java | 58 +++ .../page/project/CreateWorkflowPage.java | 108 ++++++ .../page/security/TenantManagePage.java | 69 ++++ .../page/security/UserManagePage.java | 67 ++++ .../dolphinscheduler/testcase/LoginTest.java | 37 ++ .../testcase/project/CreateProjectTest.java | 36 ++ .../testcase/project/CreateWorkflowTest.java | 34 ++ .../testcase/security/TenantManageTest.java | 34 ++ .../testcase/security/UserManageTest.java | 34 ++ .../test/resources/config/config.properties | 58 +++ e2e/suite.xml | 18 + e2e/testng.xml | 22 ++ pom.xml | 1 - 33 files changed, 2148 insertions(+), 3 deletions(-) create mode 100644 e2e/pom.xml create mode 100644 e2e/src/main/java/org/apache/dolphinscheduler/constant/TestConstant.java create mode 100644 e2e/src/main/java/org/apache/dolphinscheduler/util/PropertiesReader.java create mode 100644 e2e/src/main/java/org/apache/dolphinscheduler/util/RedisUtil.java create mode 100644 e2e/src/test/java/org/apache/dolphinscheduler/base/BaseDriver.java create mode 100644 e2e/src/test/java/org/apache/dolphinscheduler/base/BaseTest.java create mode 100644 e2e/src/test/java/org/apache/dolphinscheduler/common/BrowserCommon.java create mode 100644 e2e/src/test/java/org/apache/dolphinscheduler/common/PageCommon.java create mode 100644 e2e/src/test/java/org/apache/dolphinscheduler/data/LoginData.java create mode 100644 e2e/src/test/java/org/apache/dolphinscheduler/data/project/CreatProjectData.java create mode 100644 e2e/src/test/java/org/apache/dolphinscheduler/data/project/CreatWorkflowData.java create mode 100644 e2e/src/test/java/org/apache/dolphinscheduler/data/security/TenantManageData.java create mode 100644 e2e/src/test/java/org/apache/dolphinscheduler/data/security/UserManageData.java create mode 100644 e2e/src/test/java/org/apache/dolphinscheduler/locator/LoginLocator.java create mode 100644 e2e/src/test/java/org/apache/dolphinscheduler/locator/project/CreateProjectLocator.java create mode 100644 e2e/src/test/java/org/apache/dolphinscheduler/locator/project/CreateWorkflowLocator.java create mode 100644 e2e/src/test/java/org/apache/dolphinscheduler/locator/security/TenantManageLocator.java create mode 100644 e2e/src/test/java/org/apache/dolphinscheduler/locator/security/UserManageLocator.java create mode 100644 e2e/src/test/java/org/apache/dolphinscheduler/page/LoginPage.java create mode 100644 e2e/src/test/java/org/apache/dolphinscheduler/page/project/CreateProjectPage.java create mode 100644 e2e/src/test/java/org/apache/dolphinscheduler/page/project/CreateWorkflowPage.java create mode 100644 e2e/src/test/java/org/apache/dolphinscheduler/page/security/TenantManagePage.java create mode 100644 e2e/src/test/java/org/apache/dolphinscheduler/page/security/UserManagePage.java create mode 100644 e2e/src/test/java/org/apache/dolphinscheduler/testcase/LoginTest.java create mode 100644 e2e/src/test/java/org/apache/dolphinscheduler/testcase/project/CreateProjectTest.java create mode 100644 e2e/src/test/java/org/apache/dolphinscheduler/testcase/project/CreateWorkflowTest.java create mode 100644 e2e/src/test/java/org/apache/dolphinscheduler/testcase/security/TenantManageTest.java create mode 100644 e2e/src/test/java/org/apache/dolphinscheduler/testcase/security/UserManageTest.java create mode 100644 e2e/src/test/resources/config/config.properties create mode 100644 e2e/suite.xml create mode 100644 e2e/testng.xml diff --git a/dolphinscheduler-dist/pom.xml b/dolphinscheduler-dist/pom.xml index b43daff41b..80ddfec9b0 100644 --- a/dolphinscheduler-dist/pom.xml +++ b/dolphinscheduler-dist/pom.xml @@ -41,7 +41,6 @@ org.apache.dolphinscheduler dolphinscheduler-api - @@ -97,7 +96,6 @@ - diff --git a/e2e/pom.xml b/e2e/pom.xml new file mode 100644 index 0000000000..ac9cea94c3 --- /dev/null +++ b/e2e/pom.xml @@ -0,0 +1,137 @@ + + + 4.0.0 + org.apache.dolphinscheduler-e2e + dolphinscheduler-e2e + 1.0.0 + + + 6.14.3 + 3.141.59 + 22.0 + 2.6 + 1.1.4 + 2.8.0 + 4.2.1 + testng.xml + + + + + + org.seleniumhq.selenium + selenium-java + ${selenium.version} + + + com.google.guava + guava + ${guava.version} + + + com.google.inject + guice + ${inject.version} + + + + org.testng + testng + ${testng.version} + + + + org.uncommons + reportng + ${reportng.version} + test + + + org.testng + testng + + + + + + commons-io + commons-io + ${commons-io.version} + + + + org.apache.servicemix.bundles + org.apache.servicemix.bundles.jedis + 2.6.0_2-SNAPSHOT + + + org.apache.commons + commons-pool2 + ${commons-pool2.version} + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.8.0 + + 1.8 + 1.8 + UTF-8 + + + + + org.apache.maven.plugins + maven-surefire-plugin + 2.20.1 + + + ${xml.file} + + -Dfile.encoding=UTF-8 + + + usedefaultlisteners + + false + + + listener + org.uncommons.reportng.HTMLReporter,org.uncommons.reportng.JUnitXMLReporter + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/e2e/src/main/java/org/apache/dolphinscheduler/constant/TestConstant.java b/e2e/src/main/java/org/apache/dolphinscheduler/constant/TestConstant.java new file mode 100644 index 0000000000..8a3b8eb6b3 --- /dev/null +++ b/e2e/src/main/java/org/apache/dolphinscheduler/constant/TestConstant.java @@ -0,0 +1,37 @@ +/* + * 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.constant; + +public class TestConstant { + /** + * 1000 + */ + public static final int ONE_THOUSANG = 1000; + + + + /** + * 3000 + */ + public static final int THREE_THOUSANG = 3000; + + /** + * 10000 + */ + public static final int TEN_THOUSANG = 10000; + +} diff --git a/e2e/src/main/java/org/apache/dolphinscheduler/util/PropertiesReader.java b/e2e/src/main/java/org/apache/dolphinscheduler/util/PropertiesReader.java new file mode 100644 index 0000000000..cbf7d07bd7 --- /dev/null +++ b/e2e/src/main/java/org/apache/dolphinscheduler/util/PropertiesReader.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.dolphinscheduler.util; + +import java.io.*; +import java.util.Properties; + +/** + * read properties + */ +public class PropertiesReader { + private static Properties properties = new Properties(); + + /** + * @param propertiesPath properties path + * @return Properties + * @throws IOException IOException + */ + public static Properties readProperties(String propertiesPath) throws IOException { + System.out.println("read properties "); + InputStream inputStream = new FileInputStream(propertiesPath); + InputStreamReader inputStreamReader = new InputStreamReader(inputStream, "UTF-8"); + BufferedReader bufferedReader = new BufferedReader(inputStreamReader); + properties.load(bufferedReader); + return properties; + } + + /*** + * @param key key + * @return value + */ + public static String getKey(String key) { + return properties.getProperty(key); + } +} diff --git a/e2e/src/main/java/org/apache/dolphinscheduler/util/RedisUtil.java b/e2e/src/main/java/org/apache/dolphinscheduler/util/RedisUtil.java new file mode 100644 index 0000000000..7f9340f0ca --- /dev/null +++ b/e2e/src/main/java/org/apache/dolphinscheduler/util/RedisUtil.java @@ -0,0 +1,204 @@ +/* + * 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.util; + +import org.apache.dolphinscheduler.constant.TestConstant; +import redis.clients.jedis.Jedis; +import redis.clients.jedis.JedisPool; +import redis.clients.jedis.JedisPoolConfig; + +/** + * redis util + */ +public class RedisUtil { + /** + * redis ip + */ + private static String redisIp; + + /** + * redis port + */ + private static int redisPort; + + /** + * redis password + */ + private static String redisPwd; + + /** + * redis pool config + */ + private static JedisPoolConfig jedisPoolConfig; + + /** + * redis pool + */ + private static JedisPool jedisPool; + + /** + * jedis connection + */ + private Jedis jedis; + + /** + * jedis expire time + */ + private int jedisExpireTime; + + /** + * jedis max total + */ + private static int jedisPoolMaxTotal; + + /** + * jedis max idle + */ + private static int jedisPoolMaxIdle; + + /** + * jedis max wait time + */ + private static int jedisPoolMaxWaitMillis; + + /** + * Whether to perform a valid check when calling the borrowObject method + */ + private static boolean jedisPoolTestOnBorrow; + + /** + * Whether to perform a valid check when calling the returnObject method + */ + private static boolean jedisPoolTestOnReturn; + + /** + * storage local thread + */ + public static ThreadLocal threadLocal = new ThreadLocal<>(); + + /* + * redis init + */ + static { + // redis properties + redisIp = PropertiesReader.getKey("redis.ip"); + redisPort = Integer.valueOf(PropertiesReader.getKey("redis.port")); + redisPwd = PropertiesReader.getKey("redis.pwd"); + //redis pool properties + jedisPoolMaxTotal = Integer.valueOf(PropertiesReader.getKey("jedis.pool.maxTotal")); + jedisPoolMaxIdle = Integer.valueOf(PropertiesReader.getKey("jedis.pool.maxIdle")); + jedisPoolMaxWaitMillis = Integer.valueOf(PropertiesReader.getKey("jedis.pool.maxWaitMillis")); + jedisPoolTestOnBorrow = Boolean.valueOf(PropertiesReader.getKey("jedis.pool.testOnBorrow")); + jedisPoolTestOnReturn = Boolean.valueOf(PropertiesReader.getKey("jedis.pool.testOnReturn")); + // redis pool start properties + jedisPoolConfig = new JedisPoolConfig(); + jedisPoolConfig.setMaxTotal(jedisPoolMaxTotal); + jedisPoolConfig.setMaxIdle(jedisPoolMaxIdle); + jedisPoolConfig.setMaxWaitMillis(jedisPoolMaxWaitMillis); + jedisPoolConfig.setTestOnBorrow(jedisPoolTestOnBorrow); + jedisPoolConfig.setTestOnReturn(jedisPoolTestOnReturn); + // connect redis + try { + System.out.println("redis init"); + if (redisPwd.isEmpty()) + jedisPool = new JedisPool(jedisPoolConfig, redisIp, redisPort, TestConstant.THREE_THOUSANG); + else { + jedisPool = new JedisPool(jedisPoolConfig, redisIp, redisPort, TestConstant.TEN_THOUSANG, redisPwd); + } + } catch (Exception e) { + e.printStackTrace(); + System.out.println("redis connect fail"); + } + } + + /** + * get redis pool + * + * @return redis pool + */ + public static JedisPool getJedisPool() { + return jedisPool; + } + + /** + * get jedis connection + * + * @return jedis connection + */ + public Jedis getNewJedis() { + Jedis newJedis = null; + try { + newJedis = jedisPool.getResource(); + } catch (Exception e) { + e.printStackTrace(); + System.out.println("redis connection fail"); + } + System.out.println("redis connection success"); + return newJedis; + } + + /** + * get jedis connection + * + * @return jedis connection + */ + public Jedis getJedis() { + return jedis; + } + + public void setJedisAndExpire(Jedis jedis) { + this.jedis = jedis; + threadLocal.set(jedis); + // jedis expire time(s) + jedisExpireTime = Integer.valueOf(PropertiesReader.getKey("jedis.expireTime")); + System.out.println("redisUtil sets up a redis connection"); + } + + /** + * set key + * + * @param key key + * @param value value + * + */ + + public void setKey(String key, String value) { + jedis.set(key, value); + // set expire time 1h + jedis.expire(key, jedisExpireTime); + } + + /** + * get key + * + * @param key key + * @return value + */ + public String getKey(String key) { + return jedis.get(key); + } + + /** + * Return jedis connection + */ + public void returnJedis() { + if (jedis != null) { + jedis.close(); + } + System.out.println("jedis has been returned"); + } +} diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/base/BaseDriver.java b/e2e/src/test/java/org/apache/dolphinscheduler/base/BaseDriver.java new file mode 100644 index 0000000000..c1397b6cb5 --- /dev/null +++ b/e2e/src/test/java/org/apache/dolphinscheduler/base/BaseDriver.java @@ -0,0 +1,133 @@ +/* + * 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.base; + + +import org.apache.dolphinscheduler.constant.TestConstant; +import org.apache.dolphinscheduler.util.PropertiesReader; +import org.openqa.selenium.JavascriptExecutor; +import org.openqa.selenium.WebDriver; +import org.openqa.selenium.chrome.ChromeDriver; +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +/** + * base driver class + */ +public class BaseDriver { + /** + * driver + */ + private WebDriver driver; + + /** + * chrome driver path + */ + private String chromeDriverPath; + + + /** + * implicitly wait time(s) + */ + private long implicitlyWait; + + /** + * page load timeout(s) + */ + private long pageLoadTimeout; + + /** + * script Timeout(s) + */ + private long setScriptTimeout; + + + /** + * Local thread storage is used to store the driver + */ + public static ThreadLocal threadLocal = new ThreadLocal<>(); + + /** + *Initialization parameters + */ + public BaseDriver() throws IOException { + /* driver test class path */ + chromeDriverPath = PropertiesReader.getKey("driver.chromeDriver"); + + /* wait time */ + implicitlyWait = Long.valueOf(PropertiesReader.getKey("driver.timeouts.implicitlyWait")); + pageLoadTimeout = Long.valueOf(PropertiesReader.getKey("driver.timeouts.pageLoadTimeout")); + setScriptTimeout = Long.valueOf(PropertiesReader.getKey("driver.timeouts.setScriptTimeout")); + } + + + /** + * start chrome browser + */ + public void startBrowser() throws Exception { + // set chrome driver + System.setProperty("webdriver.chrome.driver", chromeDriverPath); + driver = new ChromeDriver(); + + /* driver setting wait time */ + // implicitly wait time + driver.manage().timeouts().implicitlyWait(implicitlyWait, TimeUnit.SECONDS); + // page load timeout + driver.manage().timeouts().pageLoadTimeout(pageLoadTimeout, TimeUnit.SECONDS); + // script timeout + driver.manage().timeouts().setScriptTimeout(setScriptTimeout, TimeUnit.SECONDS); + + // window maximize + driver.manage().window().maximize(); + + // set threadLocal + threadLocal.set(driver); + } + + /** + * get webDriver + * + * @return driver + */ + public WebDriver getDriver() { + return driver; + } + + /** + * set webDriver + * + * @param driver driver + */ + public void setDriver(WebDriver driver) { + this.driver = driver; + // Thread local storage + threadLocal.set(driver); + } + + /** + * close browser + */ + public void closeBrowser() throws InterruptedException { + // JS Show a pop-up box to indicate the end of the test + Thread.sleep(TestConstant.ONE_THOUSANG); + ((JavascriptExecutor) driver).executeScript("alert('Test completed, browser closes after 3s')"); + Thread.sleep(TestConstant.THREE_THOUSANG); + if (driver != null) { + driver.quit(); + } + } +} diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/base/BaseTest.java b/e2e/src/test/java/org/apache/dolphinscheduler/base/BaseTest.java new file mode 100644 index 0000000000..c12c19fc1d --- /dev/null +++ b/e2e/src/test/java/org/apache/dolphinscheduler/base/BaseTest.java @@ -0,0 +1,107 @@ +/* + * 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.base; + + +import org.apache.dolphinscheduler.page.LoginPage; +import org.apache.dolphinscheduler.util.PropertiesReader; +import org.openqa.selenium.WebDriver; +import org.testng.annotations.*; + +import java.io.IOException; +import java.util.Properties; + +/** + * base test class + */ +public class BaseTest { + /** + * properties + */ + private static Properties properties; + + + /** + * baseDriver + */ + private BaseDriver baseDriver; + + /** + * driver + */ + public WebDriver driver; + + /** + * Executed before executing a test suite  + * Read the test configuration file + * + * @param propertiesPath properties path + * @throws IOException IOException + */ + @BeforeSuite(alwaysRun = true) + @Parameters({"propertiesPath"}) + public void beforeSuite(@Optional("src/test/resources/config/config.properties") String propertiesPath) throws IOException { + // read properties + properties = PropertiesReader.readProperties(propertiesPath); + } + + /** + * Executed before executing a testcase + */ + @BeforeTest(alwaysRun = true) + public void beforeTest() throws Exception { + //base driver + baseDriver = new BaseDriver(); + baseDriver.startBrowser(); + driver = baseDriver.getDriver(); + } + + /** + * Executed before executing a class method in a test case + */ + @BeforeClass(alwaysRun = true) + public void setUp() throws IOException, InterruptedException { + LoginPage loginPage = new LoginPage(driver); + loginPage.jumpPage(); + loginPage.login(); + } + + + /** + * Execute after executing a class method in a test case + */ + @AfterClass(alwaysRun = true) + public void afterClass() { + // logout + } + + /** + * Execute after executing a testcase + */ + @AfterTest(alwaysRun = true) + public void afterTest() throws InterruptedException { + // close browser + baseDriver.closeBrowser(); + } + + /** + * Execute after executing a testsuite + */ + @AfterSuite(alwaysRun = true) + public void afterSuite() { + } +} \ No newline at end of file diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/common/BrowserCommon.java b/e2e/src/test/java/org/apache/dolphinscheduler/common/BrowserCommon.java new file mode 100644 index 0000000000..fb16162d55 --- /dev/null +++ b/e2e/src/test/java/org/apache/dolphinscheduler/common/BrowserCommon.java @@ -0,0 +1,353 @@ +/* + * 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.common; + +import org.apache.dolphinscheduler.util.PropertiesReader; +import org.apache.dolphinscheduler.util.RedisUtil; +import org.openqa.selenium.*; +import org.openqa.selenium.interactions.Actions; +import org.openqa.selenium.support.ui.ExpectedConditions; +import org.openqa.selenium.support.ui.WebDriverWait; +import redis.clients.jedis.Jedis; + +import java.util.ArrayList; +import java.util.List; +import java.util.Set; + +/** + * Encapsulate the most basic operations on the interface in the browser + */ +public class BrowserCommon { + /** + * driver + */ + protected WebDriver driver; + + /** + * actions + */ + protected Actions actions; + + /** + * Javascript + */ + protected JavascriptExecutor je; + + /** + * Show wait + */ + protected WebDriverWait wait; + + /** + * Jedis + */ + protected Jedis jedis; + + /** + * redis util + */ + protected RedisUtil redisUtil; + + /** + * @param driver driver + */ + public BrowserCommon(WebDriver driver) { + this.driver = driver; + this.actions = new Actions(driver); + this.je = ((JavascriptExecutor) driver); + // show wait timeout + long timeout = Long.valueOf(PropertiesReader.getKey("driver.timeouts.webDriverWait")); + wait = new WebDriverWait(driver, timeout); + } + + /** + * @param driver driver + * @param jedis jedis + */ + public BrowserCommon(WebDriver driver, Jedis jedis) { + this.driver = driver; + this.actions = new Actions(driver); + this.je = ((JavascriptExecutor) driver); + // show wait timeout + long timeout = Long.valueOf(PropertiesReader.getKey("driver.timeouts.webDriverWait")); + wait = new WebDriverWait(driver, timeout); + this.jedis = jedis; + } + + /** + * @param driver driver + * @param redisUtil redisUtil + */ + public BrowserCommon(WebDriver driver, RedisUtil redisUtil) { + this.driver = driver; + this.actions = new Actions(driver); + this.je = ((JavascriptExecutor) driver); + // show wait timeout + long timeout = Long.valueOf(PropertiesReader.getKey("driver.timeouts.webDriverWait")); + wait = new WebDriverWait(driver, timeout); +// this.redisUtil = redisUtil; +// this.jedis = redisUtil.getJedis(); + } + + + /** + * Get WebElement element object through element positioning + * + * @param locator By + * @return WebElement + */ + + public WebElement locateElement(By locator) { + return wait.until(ExpectedConditions.presenceOfElementLocated(locator)); + } + + /** + * Click button element + * @param locator By + * @return clickButton + */ + public WebElement clickButton(By locator) { + WebElement buttonElement = locateElement(locator); + wait.until(ExpectedConditions.elementToBeClickable(locator)); + ExpectedConditions.elementToBeClickable(locator); + buttonElement.click(); + return buttonElement; + } + + /** + * Click element + * + * @param locator By + * @return inputElement + */ + public WebElement clickElement(By locator) { + WebElement clickElement = locateElement(locator); + clickElement.click(); + return clickElement; + } + + /** + * input element + * + * @param locator By + * @param content Input content + * @return inputElement + */ + public WebElement sendInput(By locator, String content) { + WebElement inputElement = locateElement(locator); + inputElement.clear(); + inputElement.sendKeys(content); + return inputElement; + } + /** + * clear element + * + * @param locator By + */ + public void clearInput(By locator) { + WebElement clearElement = locateElement(locator); + clearElement.click(); + clearElement.clear(); + clearElement.sendKeys(Keys.chord(Keys.CONTROL, "a")); + clearElement.sendKeys(Keys.DELETE); + } + + + /** + * move to element + * + * @param locator BY + * @return actions + */ + public Actions moveToElement(By locator) { + return actions.moveToElement(locateElement(locator)); + } + + /** + * mouse drag element + * + * @param source_locator BY + * @param target_locator BY + * @param X X-axis + * @param Y Y-axis + */ + public void dragAndDropBy(By source_locator, By target_locator, int X, int Y) { + WebElement sourcetElement = locateElement(source_locator); + WebElement targetElement = locateElement(target_locator); + actions.dragAndDrop(sourcetElement, targetElement).moveToElement(targetElement, X, Y).perform(); + actions.release(); + } + + + /** + * jump page + * + * @param url url + */ + public void jumpPage(String url) { + driver.get(url); + } + + + /** + * Find the next handle, recommended for two windows + * + * @return driver + */ + public WebDriver switchNextHandle() { + // Current window handle + String currentHandle = driver.getWindowHandle(); + // All window handle + Set allHandles = driver.getWindowHandles(); + // Finding the next handle + for (String handle : allHandles) { + if (!handle.equals(currentHandle)) { + return driver.switchTo().window(handle); + } + } + return driver; + } + + /** + * Multi-window switch handle, according to the handle number passed in + * + * @param num Number starts from 1 + * @return driver + */ + public WebDriver switchHandle(int num) { + // current handle + String currentHandle = driver.getWindowHandle(); + // all handle + Set allHandlesSet = driver.getWindowHandles(); + List allHandlesList = new ArrayList<>(allHandlesSet); + // switch handle + return driver.switchTo().window(allHandlesList.get(num - 1)); + } + + /** + * Switch frame structure + * + * @param locator frame + * @return driver + */ + public WebDriver switchFrame(By locator) { + return driver.switchTo().frame(locateElement(locator)); + } + + /** + * Switch parent frame structure + * + * @return driver + */ + public WebDriver switchParentFrame() { + return driver.switchTo().parentFrame(); + } + + /** + * Switch out of frame structure + * + * @return driver + */ + public WebDriver switchOutOfFrame() { + return driver.switchTo().defaultContent(); + } + + + /** + * execute JS Script + * + * @param script JS script + */ + public void executeScript(String script) { + je.executeScript(script); + } + + /** + * execute JS Script + * + * @param script JS script + * @param args Object element array + */ + public void executeScript(String script, Object... args) { + je.executeScript(script, args); + } + + /** + * Page slide to top + */ + public void scrollToTop() { + executeScript("window.scrollTo(0, 0)"); + } + + /** + * Page slides to the bottom + */ + public void scrollToBottom() { + executeScript("window.scrollTo(0, document.body.scrollHeight)"); + } + + /** + * Page swipe makes the top of the element align with the top of the page + * + * @param by Elements that need to be aligned with the top of the page + */ + public void scrollElementTopToTop(By by) { + executeScript("arguments[0].scrollIntoView(true);", driver.findElement(by)); + } + + /** + * Page sliding makes the bottom of the element aligned with the bottom of the page + * + * @param by Elements that need to be aligned with the bottom of the page + */ + public void scrollElementBottomToBottom(By by) { + executeScript("arguments[0].scrollIntoView(false);", driver.findElement(by)); + } + + + /** + * Determine if the current page title is the specified title + * + * @param title title + * @return boolean + */ + + public boolean ifTitleIs(String title) { + return wait.until(ExpectedConditions.titleIs(title)); + } + + /** + * Determines whether the current page title contains the specified text + * + * @param text text + * @return boolean + */ + public boolean ifTitleContains(String text) { + return wait.until(ExpectedConditions.titleContains(text)); + } + + /** + * Determines whether the text value of an element on the current page is the specified text + * + * @param locator By + * @param text text + * @return boolean + */ + public boolean ifTextExists(By locator, String text) { + return wait.until(ExpectedConditions.textToBePresentInElementLocated(locator, text)); + } +} diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/common/PageCommon.java b/e2e/src/test/java/org/apache/dolphinscheduler/common/PageCommon.java new file mode 100644 index 0000000000..72dc03a77c --- /dev/null +++ b/e2e/src/test/java/org/apache/dolphinscheduler/common/PageCommon.java @@ -0,0 +1,50 @@ +/* + * 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.common; + +import org.apache.dolphinscheduler.util.RedisUtil; +import org.openqa.selenium.WebDriver; +import redis.clients.jedis.Jedis; + + +/** + * Encapsulate the operation methods that can be used for each module page + */ +public class PageCommon extends BrowserCommon { + /** + * @param driver driver + */ + public PageCommon(WebDriver driver) { + super(driver); + } + + /** + * @param driver driver + * @param jedis jedis + */ + public PageCommon(WebDriver driver, Jedis jedis) { + super(driver, jedis); + } + + /** + * @param driver driver + * @param redisUtil redisUtil + */ + public PageCommon(WebDriver driver, RedisUtil redisUtil) { + super(driver, redisUtil); + } +} diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/data/LoginData.java b/e2e/src/test/java/org/apache/dolphinscheduler/data/LoginData.java new file mode 100644 index 0000000000..e56df5a448 --- /dev/null +++ b/e2e/src/test/java/org/apache/dolphinscheduler/data/LoginData.java @@ -0,0 +1,43 @@ +/* + * 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.data; + +import org.apache.dolphinscheduler.util.PropertiesReader; + +/** + * Landing page object: data + */ + +public class LoginData { + /** + * Login URL + */ + public static final String URL = PropertiesReader.getKey("LOGIN_URL"); + + + /** + * Login username + */ + public static final String USER = PropertiesReader.getKey("USER_NAME"); + + /** + * Login password + */ + public static final String PASSWORD = PropertiesReader.getKey("PASSWORD"); + + public static final String TENANT = "租户管理 - DolphinScheduler"; +} diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/data/project/CreatProjectData.java b/e2e/src/test/java/org/apache/dolphinscheduler/data/project/CreatProjectData.java new file mode 100644 index 0000000000..8f6c9c8e97 --- /dev/null +++ b/e2e/src/test/java/org/apache/dolphinscheduler/data/project/CreatProjectData.java @@ -0,0 +1,26 @@ +/* + * 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.data.project; + +public class CreatProjectData { + // create project name + public static final String PROJECT_NAME = "selenium_project"; + // create project description + public static final String DESCRIPTION = "test create project description"; + // project page title + public static final String PROJECT_TITLE = "项目 - DolphinScheduler"; +} diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/data/project/CreatWorkflowData.java b/e2e/src/test/java/org/apache/dolphinscheduler/data/project/CreatWorkflowData.java new file mode 100644 index 0000000000..1e45d47a92 --- /dev/null +++ b/e2e/src/test/java/org/apache/dolphinscheduler/data/project/CreatWorkflowData.java @@ -0,0 +1,36 @@ +/* + * 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.data.project; + +public class CreatWorkflowData { + //input shell task name + public static final String SHELL_TASK_NAME = "shell task description test"; + + //input shell task description + public static final String SHELL_TASK_DESCRIPTION = "shell task description test"; + + //input timeout + public static final String INPUT_TIMEOUT = "60"; + + //input shell script + public static final String SHELL_SCRIPT = "echo 1111111"; + + public static final String WORKFLOW_TITLE = "创建流程定义 - DolphinScheduler"; + + + +} diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/data/security/TenantManageData.java b/e2e/src/test/java/org/apache/dolphinscheduler/data/security/TenantManageData.java new file mode 100644 index 0000000000..b791c073e0 --- /dev/null +++ b/e2e/src/test/java/org/apache/dolphinscheduler/data/security/TenantManageData.java @@ -0,0 +1,55 @@ +/* + * 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. + */ + +/** + * Tenant page object: data + */ +package org.apache.dolphinscheduler.data.security; + +import org.apache.dolphinscheduler.data.LoginData; + +public class TenantManageData { + /** + * Tenant URL + */ + public static final String TENANAT_URL = LoginData.URL + "/ui/#/security/tenant"; + + /** + * Tenant Code + */ + public static final String TENANAT_CODE = "dolphinscheduler_tenant_code2"; + + /** + * Tenant Name + */ + public static final String TENANAT_NAME = "dolphinscheduler_tenant_Name"; + + /** + * Queue + */ + public static final String QUEUE = "default"; + + /** + * Description + */ + public static final String DESCRIPTION = "creat tenant test"; + + public static final String TENANAT_MANAGE = "租户管理 - DolphinScheduler"; + + + +} diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/data/security/UserManageData.java b/e2e/src/test/java/org/apache/dolphinscheduler/data/security/UserManageData.java new file mode 100644 index 0000000000..03c985fd81 --- /dev/null +++ b/e2e/src/test/java/org/apache/dolphinscheduler/data/security/UserManageData.java @@ -0,0 +1,34 @@ +/* + * 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.data.security; + +import org.apache.dolphinscheduler.data.LoginData; + +public class UserManageData { + public static final String USER_URL = LoginData.URL + "/ui/#/security/users"; + + public static final String USERNAME = "selenium111"; + + public static final String PASSWORD = "123456qwe"; + + public static final String EMAIL = "123456789@qq.com"; + + public static final String PHONE = "15811112222"; + + public static final String USER_MANAGE = "用户管理 - DolphinScheduler"; + +} diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/locator/LoginLocator.java b/e2e/src/test/java/org/apache/dolphinscheduler/locator/LoginLocator.java new file mode 100644 index 0000000000..32a82bbbc8 --- /dev/null +++ b/e2e/src/test/java/org/apache/dolphinscheduler/locator/LoginLocator.java @@ -0,0 +1,33 @@ +/* + * 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.locator; + +import org.openqa.selenium.By; + +/** + * Page object: element positioning + */ + +public class LoginLocator { + public static final By LOGIN_INPUT_USER = By.xpath("//input[@class='input-element suffix']"); + + public static final By LOGIN_INPUT_PASSWORD = By.xpath("//input[@class='input-element suffix']"); + + public static final By LOGIN_BUTTON = By.xpath("//button"); + + public static final By LOGIN_BUTTON_MOVE = By.xpath("//button[contains(.,' Loading...')]"); +} diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/locator/project/CreateProjectLocator.java b/e2e/src/test/java/org/apache/dolphinscheduler/locator/project/CreateProjectLocator.java new file mode 100644 index 0000000000..d2c0d8412c --- /dev/null +++ b/e2e/src/test/java/org/apache/dolphinscheduler/locator/project/CreateProjectLocator.java @@ -0,0 +1,36 @@ +/* + * 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.locator.project; + +import org.openqa.selenium.By; + +public class CreateProjectLocator { + //click project manage + public static final By PROJECT_MANAGE = By.xpath("//div[2]/div/a/span"); + + //click create project button + public static final By CREATE_PROJECT_BUTTON = By.xpath("//button/span"); + + //input project name + public static final By PROJECT_NAME = By.xpath("//div[2]/div/div/div[2]/div/input"); + + //input project description + public static final By PROJECT_DESCRIPTION = By.xpath("//textarea"); + + //submit button + public static final By SUBMIT_BUTTON = By.xpath("//div[3]/button[2]/span"); +} diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/locator/project/CreateWorkflowLocator.java b/e2e/src/test/java/org/apache/dolphinscheduler/locator/project/CreateWorkflowLocator.java new file mode 100644 index 0000000000..3bea648d57 --- /dev/null +++ b/e2e/src/test/java/org/apache/dolphinscheduler/locator/project/CreateWorkflowLocator.java @@ -0,0 +1,97 @@ +/* + * 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.locator.project; + +import org.openqa.selenium.By; + +public class CreateWorkflowLocator { + // click project manage + public static final By CLICK_PROJECT_MANAGE = By.xpath("//div[2]/div/a/span"); + + // click project name + public static final By CLICK_PROJECT_NAME = By.xpath("//span/a"); + + // click workflow define + public static final By CLICK_WORKFLOW_DEFINE = By.xpath("//li/span"); + + // click create workflow button + public static final By CLICK_CREATE_WORKFLOW_BUTTON = By.xpath("//button/span"); + + //mouse down at shell + public static final By MOUSE_DOWN_AT_SHELL = By.xpath("//*[@id='SHELL']/div/div"); + + //mouse down at spark + public static final By MOUSE_DOWN_AT_SPARK = By.xpath("//div[5]/div/div"); + + //mouse move at DAG + public static final By MOUSE_MOVE_SHELL_AT_DAG = By.xpath("//div[2]/div/div[2]/div[2]/div/div"); + +// //click shell task +// public static final By CLICK_SHELL_TASK = By.xpath("//div[2]/div/div[2]/div[2]/div/div"); + + //input shell task _name + public static final By INPUT_SHELL_TASK_NAME = By.xpath("//input"); + + //click stop run type + public static final By CLICK_STOP_RUN_TYPE = By.xpath("//label[2]/span/input"); + + //click normal run type + public static final By CLICK_NORMAL_RUN_TYPE = By.xpath("//span/input"); + + //input shell task description + public static final By INPUT_SHELL_TASK_DESCRIPTION = By.xpath("//label/div/textarea"); + + //click task priority + public static final By CLICK_TASK_PRIORITY = By.xpath("//span/div/div/div/div/div"); + + //select task priority + public static final By SELECT_TASK_PRIORITY = By.xpath("//li[2]/li/span"); + + //click work group + public static final By CLICK_WORK_GROUP = By.xpath("//div/div/input"); + + //select work group + public static final By SELECT_WORK_GROUP = By.xpath("//div[4]/div[2]/div/div[1]/div/input"); + + //select number of failed retries + public static final By SELECT_FAIL_RETRIES_NUMBER = By.xpath("//div[5]/div[2]/div[1]/div[1]/div/input"); + + //select failed retry interval + public static final By SELECT_FAIL_RETRIES_INTERVAL = By.xpath("//div[5]/div[2]/div[2]/div[1]/div/input"); + + //click timeout alarm + public static final By CLICK_TIMEOUT_ALARM = By.xpath("//label/div/span/span"); + + //select timeout fail + public static final By SELECT_TIMEOUT_FAIL = By.xpath("//div/div/label[2]/span/input"); + + //cancel timeout alarm + public static final By CANCEL_TIMEOUT_ALARM = By.xpath("//div/div/label/span/input"); + + //select timeout alarm + public static final By SELECT_TIMEOUT_ALARM = By.xpath("//div/div/label/span/input"); + + //input timeout + public static final By SELECT_TIMEOUT = By.xpath("//div[3]/div[2]/label/div/input"); + + //input script + public static final By INPUT_SCRIPT = By.xpath("//div[2]/div/div/div/div/div/textarea"); + + + +} diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/locator/security/TenantManageLocator.java b/e2e/src/test/java/org/apache/dolphinscheduler/locator/security/TenantManageLocator.java new file mode 100644 index 0000000000..7d9c8a57ce --- /dev/null +++ b/e2e/src/test/java/org/apache/dolphinscheduler/locator/security/TenantManageLocator.java @@ -0,0 +1,33 @@ +/* + * 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.locator.security; + +import org.openqa.selenium.By; + +public class TenantManageLocator{ + public static final By CREATE_TENANT_BUTTON = By.xpath("//button[@class='ans-btn ans-btn-ghost ans-btn-small']"); + + public static final By TENANT_INPUT_CODE = By.xpath("//div[2]/div/div/div[2]/div/input"); + + public static final By TENANT_INPUT_NAME = By.xpath("//div[2]/div[2]/div/input"); + + public static final By QUEUE = By.xpath("//textarea"); + + public static final By DESCRIPTION = By.xpath("//textarea"); + + public static final By SUBMIT_BUTTON = By.xpath("//div[3]/button[2]/span"); +} diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/locator/security/UserManageLocator.java b/e2e/src/test/java/org/apache/dolphinscheduler/locator/security/UserManageLocator.java new file mode 100644 index 0000000000..0d84692cb2 --- /dev/null +++ b/e2e/src/test/java/org/apache/dolphinscheduler/locator/security/UserManageLocator.java @@ -0,0 +1,45 @@ +/* + * 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.locator.security; + +import org.openqa.selenium.By; + +public class UserManageLocator { + + public static final By CLICK_USER_MANAGE = By.xpath("//div[3]/div/a/div/a/span"); + + public static final By CLICK_CREATE_USER_BUTTON = By.xpath("//span[contains(.,'创建用户')]"); + + public static final By INPUT_USERNAME = By.xpath("//div[2]/div/div/div[2]/div/input"); + + public static final By INPUT_PASSWORD = By.xpath("//div[2]/div[2]/div/input"); + + public static final By CLICK_TENANT = By.xpath("//div[3]/div[2]/div/div/div/input"); + + public static final By SELECT_TENANT = By.xpath("//div[3]/div[2]/div/div[2]/div/div/div/ul/li/span"); + + public static final By CLICK_QUEUE = By.xpath("//div[4]/div[2]/div/div/div/input"); + + public static final By SELECT_QUEUE = By.xpath("//div[4]/div[2]/div/div[2]/div/div/div/ul/li/span"); + + public static final By TENANT_INPUT_EMAIL = By.xpath("//div[5]/div[2]/div/input"); + + public static final By TENANT_INPUT_PHONE = By.xpath("//div[6]/div[2]/div/input"); + + public static final By SUBMIT = By.xpath("//div[3]/button[2]/span"); + +} diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/page/LoginPage.java b/e2e/src/test/java/org/apache/dolphinscheduler/page/LoginPage.java new file mode 100644 index 0000000000..cd6b318651 --- /dev/null +++ b/e2e/src/test/java/org/apache/dolphinscheduler/page/LoginPage.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.page; + +import org.apache.dolphinscheduler.common.PageCommon; +import org.apache.dolphinscheduler.constant.TestConstant; +import org.apache.dolphinscheduler.data.LoginData; +import org.apache.dolphinscheduler.locator.LoginLocator; +import org.apache.dolphinscheduler.util.RedisUtil; +import org.openqa.selenium.WebDriver; + + + +public class LoginPage extends PageCommon { + /** + * Unique constructor + * @param driver driver + */ + public LoginPage(WebDriver driver) { + super(driver); + } + + + /** + * jump page + */ + public void jumpPage() { + System.out.println("jump login page"); + super.jumpPage(LoginData.URL); + } + + /** + * login + * + * @return Whether to enter the specified page after searching + */ + public boolean login() throws InterruptedException { + System.out.println("LoginPage"); + // login data + sendInput(LoginLocator.LOGIN_INPUT_USER, LoginData.USER); + sendInput(LoginLocator.LOGIN_INPUT_PASSWORD, LoginData.PASSWORD); + + // click login button + clickButton(LoginLocator.LOGIN_BUTTON); + + moveToElement(LoginLocator.LOGIN_BUTTON_MOVE); + + // Whether to enter the specified page after login + return ifTitleContains(LoginData.TENANT); + } +} diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/page/project/CreateProjectPage.java b/e2e/src/test/java/org/apache/dolphinscheduler/page/project/CreateProjectPage.java new file mode 100644 index 0000000000..796171542b --- /dev/null +++ b/e2e/src/test/java/org/apache/dolphinscheduler/page/project/CreateProjectPage.java @@ -0,0 +1,58 @@ +/* + * 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.page.project; + +import org.apache.dolphinscheduler.common.PageCommon; +import org.apache.dolphinscheduler.constant.TestConstant; +import org.apache.dolphinscheduler.data.project.CreatProjectData; +import org.apache.dolphinscheduler.locator.project.CreateProjectLocator; +import org.openqa.selenium.WebDriver; + +public class CreateProjectPage extends PageCommon { + public CreateProjectPage(WebDriver driver) { + super(driver); + } + /** + * jump page + */ + public void jumpPage() throws InterruptedException { + Thread.sleep(TestConstant.ONE_THOUSANG); + clickElement(CreateProjectLocator.PROJECT_MANAGE); + } + + /** + * creatTenant + * + * @return Whether to enter the specified page after creat tenant + */ + public boolean createProject() throws InterruptedException { + Thread.sleep(TestConstant.ONE_THOUSANG); + //click create project + clickElement(CreateProjectLocator.CREATE_PROJECT_BUTTON); + Thread.sleep(TestConstant.ONE_THOUSANG); + + // input create project data + sendInput(CreateProjectLocator.PROJECT_NAME, CreatProjectData.PROJECT_NAME); + sendInput(CreateProjectLocator.PROJECT_DESCRIPTION, CreatProjectData.DESCRIPTION); + + // click submit button + clickButton(CreateProjectLocator.SUBMIT_BUTTON); + + // Whether to enter the specified page after submit + return ifTitleContains(CreatProjectData.PROJECT_TITLE); + } +} diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/page/project/CreateWorkflowPage.java b/e2e/src/test/java/org/apache/dolphinscheduler/page/project/CreateWorkflowPage.java new file mode 100644 index 0000000000..827198bd86 --- /dev/null +++ b/e2e/src/test/java/org/apache/dolphinscheduler/page/project/CreateWorkflowPage.java @@ -0,0 +1,108 @@ +/* + * 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.page.project; + +import org.apache.dolphinscheduler.common.PageCommon; +import org.apache.dolphinscheduler.constant.TestConstant; +import org.apache.dolphinscheduler.data.project.CreatWorkflowData; +import org.apache.dolphinscheduler.locator.project.CreateProjectLocator; +import org.apache.dolphinscheduler.locator.project.CreateWorkflowLocator; +import org.openqa.selenium.WebDriver; + +public class CreateWorkflowPage extends PageCommon { + public CreateWorkflowPage(WebDriver driver) { + super(driver); + } + /** + * jump page + */ + public boolean CreateWorkflow() throws InterruptedException { + Thread.sleep(TestConstant.ONE_THOUSANG); + // click project manage + clickElement(CreateProjectLocator.PROJECT_MANAGE); + Thread.sleep(TestConstant.ONE_THOUSANG); + + // click project name + clickElement(CreateWorkflowLocator.CLICK_PROJECT_NAME); + Thread.sleep(TestConstant.ONE_THOUSANG); + + // click workflow define + clickElement(CreateWorkflowLocator.CLICK_WORKFLOW_DEFINE); + + // click create workflow button + clickElement(CreateWorkflowLocator.CLICK_CREATE_WORKFLOW_BUTTON); + + //drag shell_task + dragAndDropBy(CreateWorkflowLocator.MOUSE_DOWN_AT_SHELL,CreateWorkflowLocator.MOUSE_MOVE_SHELL_AT_DAG, 3 ,6); + + //input shell task _name + sendInput(CreateWorkflowLocator.INPUT_SHELL_TASK_NAME , CreatWorkflowData.SHELL_TASK_NAME); + + //click stop run type + clickElement(CreateWorkflowLocator.CLICK_STOP_RUN_TYPE); + + + //click normal run type + clickElement(CreateWorkflowLocator.CLICK_NORMAL_RUN_TYPE); + + + //input shell task description + sendInput(CreateWorkflowLocator.INPUT_SHELL_TASK_DESCRIPTION , CreatWorkflowData.SHELL_TASK_DESCRIPTION); + + //select task priority + clickElement(CreateWorkflowLocator.CLICK_TASK_PRIORITY); + clickElement(CreateWorkflowLocator.SELECT_TASK_PRIORITY); + + //select work group + clickElement(CreateWorkflowLocator.CLICK_WORK_GROUP); + clickElement(CreateWorkflowLocator.SELECT_WORK_GROUP); + + //select number of failed retries + clickElement(CreateWorkflowLocator.SELECT_FAIL_RETRIES_NUMBER); + + //select failed retry interval + clickElement(CreateWorkflowLocator.SELECT_FAIL_RETRIES_INTERVAL); + + + //click timeout alarm + clickElement(CreateWorkflowLocator.CLICK_TIMEOUT_ALARM); + + + //select timeout fail + clickElement(CreateWorkflowLocator.SELECT_TIMEOUT_FAIL); + + + //cancel timeout alarm + clickElement(CreateWorkflowLocator.CANCEL_TIMEOUT_ALARM); + + + //select timeout alarm + clickElement(CreateWorkflowLocator.SELECT_TIMEOUT_ALARM); + + //clear input + clearInput(CreateWorkflowLocator.SELECT_TIMEOUT); + Thread.sleep(TestConstant.ONE_THOUSANG); + + //input timeout + sendInput(CreateWorkflowLocator.SELECT_TIMEOUT,CreatWorkflowData.INPUT_TIMEOUT); + + //input script + sendInput(CreateWorkflowLocator.INPUT_SCRIPT, CreatWorkflowData.SHELL_SCRIPT); + + return ifTitleContains(CreatWorkflowData.WORKFLOW_TITLE); + } +} diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/page/security/TenantManagePage.java b/e2e/src/test/java/org/apache/dolphinscheduler/page/security/TenantManagePage.java new file mode 100644 index 0000000000..4c88f6575e --- /dev/null +++ b/e2e/src/test/java/org/apache/dolphinscheduler/page/security/TenantManagePage.java @@ -0,0 +1,69 @@ +/* + * 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.page.security; + +import org.apache.dolphinscheduler.common.PageCommon; +import org.apache.dolphinscheduler.constant.TestConstant; +import org.apache.dolphinscheduler.data.LoginData; +import org.apache.dolphinscheduler.data.security.TenantManageData; +import org.apache.dolphinscheduler.locator.LoginLocator; +import org.apache.dolphinscheduler.locator.security.TenantManageLocator; +import org.apache.dolphinscheduler.util.RedisUtil; +import org.openqa.selenium.WebDriver; + +public class TenantManagePage extends PageCommon { + /** + * Unique constructor + * @param driver driver + */ + public TenantManagePage(WebDriver driver) { + super(driver); + } + + + /** + * jump page + */ + public void jumpPage() { + System.out.println("jump tenant page"); + super.jumpPage(TenantManageData.TENANAT_URL); + } + + /** + * createTenant + * + * @return Whether to enter the specified page after creat tenant + */ + public boolean createTenant() throws InterruptedException { + Thread.sleep(TestConstant.ONE_THOUSANG); + + //create tenant + clickButton(TenantManageLocator.CREATE_TENANT_BUTTON); + + // tenant data + sendInput(TenantManageLocator.TENANT_INPUT_CODE, TenantManageData.TENANAT_CODE); + sendInput(TenantManageLocator.TENANT_INPUT_NAME, TenantManageData.TENANAT_NAME); + sendInput(TenantManageLocator.QUEUE, TenantManageData.QUEUE); + sendInput(TenantManageLocator.DESCRIPTION, TenantManageData.DESCRIPTION); + + // click button + clickButton(TenantManageLocator.SUBMIT_BUTTON); + + // Whether to enter the specified page after submit + return ifTitleContains(TenantManageData.TENANAT_MANAGE); + } +} diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/page/security/UserManagePage.java b/e2e/src/test/java/org/apache/dolphinscheduler/page/security/UserManagePage.java new file mode 100644 index 0000000000..3e2be2d126 --- /dev/null +++ b/e2e/src/test/java/org/apache/dolphinscheduler/page/security/UserManagePage.java @@ -0,0 +1,67 @@ +/* + * 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.page.security; + +import org.apache.dolphinscheduler.common.PageCommon; +import org.apache.dolphinscheduler.constant.TestConstant; +import org.apache.dolphinscheduler.data.security.UserManageData; +import org.apache.dolphinscheduler.locator.security.UserManageLocator; +import org.openqa.selenium.WebDriver; + +public class UserManagePage extends PageCommon { + public UserManagePage(WebDriver driver) { + super(driver); + } + /** + * jump page + */ + public void jumpPage() { + System.out.println("jump tenant page"); + super.jumpPage(UserManageData.USER_URL); + } + + /** + * creatTenant + * + * @return Whether to enter the specified page after creat tenant + */ + public boolean createUser() throws InterruptedException { + Thread.sleep(TestConstant.ONE_THOUSANG); + // click user manage + clickElement(UserManageLocator.CLICK_USER_MANAGE); + Thread.sleep(TestConstant.ONE_THOUSANG); + + // click create user button + clickButton(UserManageLocator.CLICK_CREATE_USER_BUTTON); + + // input user data + sendInput(UserManageLocator.INPUT_USERNAME, UserManageData.USERNAME); + sendInput(UserManageLocator.INPUT_PASSWORD, UserManageData.PASSWORD); + clickButton(UserManageLocator.CLICK_TENANT); + clickButton(UserManageLocator.SELECT_TENANT); + clickButton(UserManageLocator.CLICK_QUEUE); + clickButton(UserManageLocator.SELECT_QUEUE); + sendInput(UserManageLocator.TENANT_INPUT_EMAIL, UserManageData.EMAIL); + sendInput(UserManageLocator.TENANT_INPUT_PHONE, UserManageData.PHONE); + + // click button + clickButton(UserManageLocator.SUBMIT); + + // Whether to enter the specified page after submit + return ifTitleContains(UserManageData.USER_MANAGE); + } +} diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/testcase/LoginTest.java b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/LoginTest.java new file mode 100644 index 0000000000..bd3c31d38b --- /dev/null +++ b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/LoginTest.java @@ -0,0 +1,37 @@ +/* + * 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.testcase; + +import org.apache.dolphinscheduler.base.BaseTest; +import org.apache.dolphinscheduler.page.LoginPage; +import org.testng.annotations.Test; + + +public class LoginTest extends BaseTest { + private LoginPage loginPage; + @Test(description = "LoginTest", priority = 1) + public void testLogin() throws InterruptedException { + // init login page + loginPage = new LoginPage(driver); + + // enter login page + loginPage.jumpPage(); + + //assert login page + assert loginPage.login(); + } +} diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/testcase/project/CreateProjectTest.java b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/project/CreateProjectTest.java new file mode 100644 index 0000000000..66af10543b --- /dev/null +++ b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/project/CreateProjectTest.java @@ -0,0 +1,36 @@ +/* + * 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.testcase.project; + +import org.apache.dolphinscheduler.base.BaseTest; +import org.apache.dolphinscheduler.page.project.CreateProjectPage; +import org.testng.annotations.Test; + +public class CreateProjectTest extends BaseTest { + private CreateProjectPage createProjectPage; + + @Test(description = "TenantTest", priority = 1) + public void testUserManage() throws InterruptedException { + createProjectPage = new CreateProjectPage(driver); + // enter user manage page + createProjectPage.jumpPage(); + //assert user manage page + assert createProjectPage.createProject(); + } + +} + diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/testcase/project/CreateWorkflowTest.java b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/project/CreateWorkflowTest.java new file mode 100644 index 0000000000..ac1bfec69f --- /dev/null +++ b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/project/CreateWorkflowTest.java @@ -0,0 +1,34 @@ +/* + * 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.testcase.project; + +import org.apache.dolphinscheduler.base.BaseTest; +import org.apache.dolphinscheduler.page.project.CreateWorkflowPage; +import org.testng.annotations.Test; + +public class CreateWorkflowTest extends BaseTest { + private CreateWorkflowPage createWorkflowPage; + + @Test(description = "CreateWorkflowTest", priority = 1) + public void CreateWorkflowTest() throws InterruptedException { + createWorkflowPage = new CreateWorkflowPage(driver); + // enter user manage page +// createWorkflowPage.jumpPage(); + //assert user manage page + assert createWorkflowPage.CreateWorkflow(); + } +} diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/testcase/security/TenantManageTest.java b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/security/TenantManageTest.java new file mode 100644 index 0000000000..7124b4e094 --- /dev/null +++ b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/security/TenantManageTest.java @@ -0,0 +1,34 @@ +/* + * 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.testcase.security; + +import org.apache.dolphinscheduler.base.BaseTest; +import org.apache.dolphinscheduler.page.security.TenantManagePage; +import org.testng.annotations.Test; + +public class TenantManageTest extends BaseTest { + private TenantManagePage tenantManagePage; + + @Test(description = "TenantTest", priority = 1) + public void testTenantManage() throws InterruptedException { + tenantManagePage = new TenantManagePage(driver); + // enter tenant manage page + tenantManagePage.jumpPage(); + //assert tenant manage page + assert tenantManagePage.createTenant(); + } +} diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/testcase/security/UserManageTest.java b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/security/UserManageTest.java new file mode 100644 index 0000000000..834ebdbbf0 --- /dev/null +++ b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/security/UserManageTest.java @@ -0,0 +1,34 @@ +/* + * 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.testcase.security; + +import org.apache.dolphinscheduler.base.BaseTest; +import org.apache.dolphinscheduler.page.security.UserManagePage; +import org.testng.annotations.Test; + +public class UserManageTest extends BaseTest { + private UserManagePage userManagePage; + + @Test(description = "TenantTest", priority = 1) + public void testUserManage() throws InterruptedException { + userManagePage = new UserManagePage(driver); + // enter user manage page + userManagePage.jumpPage(); + //assert user manage page + assert userManagePage.createUser(); + } +} diff --git a/e2e/src/test/resources/config/config.properties b/e2e/src/test/resources/config/config.properties new file mode 100644 index 0000000000..e3ee61c1c1 --- /dev/null +++ b/e2e/src/test/resources/config/config.properties @@ -0,0 +1,58 @@ +# +# 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. +# + +############### project ############## +# login url +LOGIN_URL=http://ark1:12345/dolphinscheduler +#login username +USER_NAME=admin +#login password +PASSWORD=123456 + +############### web driver ############## +# driver path +driver.chromeDriver=/Users/chenxingchun/Documents/easyscheduler/selenium/chromeDriver +# implicitly wait(s) +driver.timeouts.implicitlyWait=10 +# show wait(s) +driver.timeouts.webDriverWait=10 +# page load timeout(s) +driver.timeouts.pageLoadTimeout=10 +# JS wait timeouts(s) +driver.timeouts.setScriptTimeout=10 + + +############### redis ############## +# redis ip +redis.ip=127.0.0.1 +# redis port +redis.port=6379 +# redis password +redis.pwd= +############### redis pool ############## +# jedis expireTime(s) +jedis.expireTime=3600 +# jedis maxTotal +jedis.pool.maxTotal=3000 +# jedis maxIdle +jedis.pool.maxIdle=1000 +# jedis maxWaitMillis +jedis.pool.maxWaitMillis=10000 +# jedis Whether to perform a valid check when calling the borrowObject method +jedis.pool.testOnBorrow=true +# jedis Whether to perform a valid check when calling the returnObject method +jedis.pool.testOnReturn=true \ No newline at end of file diff --git a/e2e/suite.xml b/e2e/suite.xml new file mode 100644 index 0000000000..d9d7ae6845 --- /dev/null +++ b/e2e/suite.xml @@ -0,0 +1,18 @@ + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/e2e/testng.xml b/e2e/testng.xml new file mode 100644 index 0000000000..60d2541e73 --- /dev/null +++ b/e2e/testng.xml @@ -0,0 +1,22 @@ + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/pom.xml b/pom.xml index ee2d808111..076b85611f 100644 --- a/pom.xml +++ b/pom.xml @@ -242,7 +242,6 @@ dolphinscheduler-alert ${project.version} - org.apache.curator curator-framework From 6b62501301b84c3f7f1e138bdb35448da182a47b Mon Sep 17 00:00:00 2001 From: dailidong Date: Wed, 26 Feb 2020 13:03:01 +0800 Subject: [PATCH 039/221] correct repeat images in README (#2017) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit some images not right,correct --- README.md | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/README.md b/README.md index 883f3fdd14..75f0877528 100644 --- a/README.md +++ b/README.md @@ -48,13 +48,13 @@ Overload processing: Task queue mechanism, the number of schedulable tasks on a ### System partial screenshot ![home page](https://user-images.githubusercontent.com/15833811/75218288-bf286400-57d4-11ea-8263-d639c6511d5f.jpg) -![dag](https://user-images.githubusercontent.com/15833811/75216789-25f74e80-57d0-11ea-8d9f-ad64a3db473c.png) +![dag](https://user-images.githubusercontent.com/15833811/75236750-3374fe80-57f9-11ea-857d-62a66a5a559d.png) ![process definition list page](https://user-images.githubusercontent.com/15833811/75216886-6f479e00-57d0-11ea-92dd-66e7640a186f.png) ![view task log online](https://user-images.githubusercontent.com/15833811/75216924-9900c500-57d0-11ea-91dc-3522a76bdbbe.png) ![resource management](https://user-images.githubusercontent.com/15833811/75216984-be8dce80-57d0-11ea-840d-58546edc8788.png) ![datasource management](https://user-images.githubusercontent.com/15833811/75217041-cfd6db00-57d0-11ea-8451-8b376d0dc8f1.png) ![monitor](https://user-images.githubusercontent.com/15833811/75217091-fac12f00-57d0-11ea-9665-d8dc19813707.png) -![security](https://user-images.githubusercontent.com/15833811/75217157-29d7a080-57d1-11ea-91ca-1fa8af936428.png) +![security](https://user-images.githubusercontent.com/15833811/75236441-bfd2f180-57f8-11ea-88bd-f24311e01b7e.png) ![treeview](https://user-images.githubusercontent.com/15833811/75217191-3fe56100-57d1-11ea-8856-f19180d9a879.png) ### Document @@ -106,10 +106,3 @@ It is because of the shoulders of these open source projects that the birth of t Please refer to [LICENSE](https://github.com/apache/incubator-dolphinscheduler/blob/dev/LICENSE) file. - - - - - - - From ee3c0aedc86611c25b9d52355a20832408f9ce16 Mon Sep 17 00:00:00 2001 From: khadgarmage Date: Thu, 27 Feb 2020 09:51:25 +0800 Subject: [PATCH 040/221] docker and testcase config for e2e test (#2028) * github action for e2e test * github action for e2e test * github action for e2e test * chromedriver * e2e * driver option * t * args bug * check * restore docker --- .github/workflows/ci_e2e.yml | 12 +++++++++++- .github/workflows/ci_ut.yml | 2 +- e2e/pom.xml | 4 ++-- .../apache/dolphinscheduler/base/BaseDriver.java | 13 ++++++++++++- .../org/apache/dolphinscheduler/data/LoginData.java | 2 +- .../data/security/TenantManageData.java | 2 +- e2e/src/test/resources/config/config.properties | 8 ++++---- e2e/testng.xml | 8 ++++---- 8 files changed, 36 insertions(+), 15 deletions(-) diff --git a/.github/workflows/ci_e2e.yml b/.github/workflows/ci_e2e.yml index 616f62d049..13f12641fe 100644 --- a/.github/workflows/ci_e2e.yml +++ b/.github/workflows/ci_e2e.yml @@ -48,8 +48,18 @@ jobs: docker run -dit -e POSTGRESQL_USERNAME=test -e POSTGRESQL_PASSWORD=test -p 8888:8888 dolphinscheduler:$VERSION all - name: Check Server Status run: sh ./dockerfile/hooks/check + - name: Prepare e2e env + run: | + sudo apt-get install -y libxss1 libappindicator1 libindicator7 xvfb unzip + wget https://dl.google.com/linux/direct/google-chrome-stable_current_amd64.deb + sudo dpkg -i google-chrome*.deb + sudo apt-get install -f -y + wget -N https://chromedriver.storage.googleapis.com/80.0.3987.106/chromedriver_linux64.zip + unzip chromedriver_linux64.zip + sudo mv -f chromedriver /usr/local/share/chromedriver + sudo ln -s /usr/local/share/chromedriver /usr/local/bin/chromedriver - name: Run e2e Test - run: echo "Run e2e Test" + run: cd ./e2e && mvn -B clean test - name: Collect logs run: | mkdir -p ${LOG_DIR} diff --git a/.github/workflows/ci_ut.yml b/.github/workflows/ci_ut.yml index 8013d40f31..6f3eda0362 100644 --- a/.github/workflows/ci_ut.yml +++ b/.github/workflows/ci_ut.yml @@ -20,7 +20,7 @@ env: DOCKER_DIR: ./docker LOG_DIR: /tmp/dolphinscheduler -name: Test Coveralls Parallel +name: Unit Test jobs: diff --git a/e2e/pom.xml b/e2e/pom.xml index ac9cea94c3..ea67c0ca16 100644 --- a/e2e/pom.xml +++ b/e2e/pom.xml @@ -64,7 +64,7 @@ org.apache.servicemix.bundles org.apache.servicemix.bundles.jedis - 2.6.0_2-SNAPSHOT + 2.6.2_1 org.apache.commons @@ -134,4 +134,4 @@ - \ No newline at end of file + diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/base/BaseDriver.java b/e2e/src/test/java/org/apache/dolphinscheduler/base/BaseDriver.java index c1397b6cb5..86a77ffcfd 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/base/BaseDriver.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/base/BaseDriver.java @@ -22,8 +22,10 @@ import org.apache.dolphinscheduler.util.PropertiesReader; import org.openqa.selenium.JavascriptExecutor; import org.openqa.selenium.WebDriver; import org.openqa.selenium.chrome.ChromeDriver; +import org.openqa.selenium.chrome.ChromeOptions; import java.io.IOException; import java.util.concurrent.TimeUnit; +import org.openqa.selenium.PageLoadStrategy; /** * base driver class @@ -81,7 +83,16 @@ public class BaseDriver { public void startBrowser() throws Exception { // set chrome driver System.setProperty("webdriver.chrome.driver", chromeDriverPath); - driver = new ChromeDriver(); + ChromeOptions chromeOptions = new ChromeOptions(); + chromeOptions.setPageLoadStrategy(PageLoadStrategy.NONE); + chromeOptions.addArguments("--no-sandbox"); + chromeOptions.addArguments("--disable-dev-shm-usage"); + chromeOptions.addArguments("--headless"); + chromeOptions.addArguments("--disable-gpu"); + chromeOptions.addArguments("--whitelisted-ips"); + chromeOptions.addArguments("--disable-infobars"); + chromeOptions.addArguments("--disable-browser-side-navigation"); + driver = new ChromeDriver(chromeOptions); /* driver setting wait time */ // implicitly wait time diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/data/LoginData.java b/e2e/src/test/java/org/apache/dolphinscheduler/data/LoginData.java index e56df5a448..532849565c 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/data/LoginData.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/data/LoginData.java @@ -39,5 +39,5 @@ public class LoginData { */ public static final String PASSWORD = PropertiesReader.getKey("PASSWORD"); - public static final String TENANT = "租户管理 - DolphinScheduler"; + public static final String TENANT = "Tenant Manage - DolphinScheduler"; } diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/data/security/TenantManageData.java b/e2e/src/test/java/org/apache/dolphinscheduler/data/security/TenantManageData.java index b791c073e0..e6f6ee6b86 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/data/security/TenantManageData.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/data/security/TenantManageData.java @@ -48,7 +48,7 @@ public class TenantManageData { */ public static final String DESCRIPTION = "creat tenant test"; - public static final String TENANAT_MANAGE = "租户管理 - DolphinScheduler"; + public static final String TENANAT_MANAGE = "Tenant Manage - DolphinScheduler"; diff --git a/e2e/src/test/resources/config/config.properties b/e2e/src/test/resources/config/config.properties index e3ee61c1c1..6a01234d01 100644 --- a/e2e/src/test/resources/config/config.properties +++ b/e2e/src/test/resources/config/config.properties @@ -17,15 +17,15 @@ ############### project ############## # login url -LOGIN_URL=http://ark1:12345/dolphinscheduler +LOGIN_URL=http://127.0.0.1:8888/dolphinscheduler/ #login username USER_NAME=admin #login password -PASSWORD=123456 +PASSWORD=dolphinscheduler123 ############### web driver ############## # driver path -driver.chromeDriver=/Users/chenxingchun/Documents/easyscheduler/selenium/chromeDriver +driver.chromeDriver=/usr/local/bin/chromedriver # implicitly wait(s) driver.timeouts.implicitlyWait=10 # show wait(s) @@ -55,4 +55,4 @@ jedis.pool.maxWaitMillis=10000 # jedis Whether to perform a valid check when calling the borrowObject method jedis.pool.testOnBorrow=true # jedis Whether to perform a valid check when calling the returnObject method -jedis.pool.testOnReturn=true \ No newline at end of file +jedis.pool.testOnReturn=true diff --git a/e2e/testng.xml b/e2e/testng.xml index 60d2541e73..757ffab248 100644 --- a/e2e/testng.xml +++ b/e2e/testng.xml @@ -6,9 +6,9 @@ - - - + + + @@ -19,4 +19,4 @@ - \ No newline at end of file + From f407adcb71be734cede10fbcecc6d7bbe1f2e7fd Mon Sep 17 00:00:00 2001 From: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> Date: Thu, 27 Feb 2020 11:43:09 +0800 Subject: [PATCH 041/221] =?UTF-8?q?Add=20e2e=20to=20create=20workflow=20ca?= =?UTF-8?q?se=20=C2=A0=20(#2027)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * add e2e * add ui-test * add e2e license and notice * add e2e license * add license * add ui-test * add ui-test * add ui-test * e2e delete dolphinscheduler parent project dependency * add ui-test * dolphinscheduler_dist delete e2e license/notice * dolphinscheduler_dist delete e2e license/notice * dolphinscheduler_dist delete e2e license/notice * dolphinscheduler_dist delete e2e license/notice * pom.xml delete e2e dependency * add e2e create workflow * Merge remote-tracking branch 'upstream/dev' into dev # Conflicts: # e2e/src/test/java/org/apache/dolphinscheduler/base/BaseDriver.java * Merge remote-tracking branch 'upstream/dev' into dev # Conflicts: # e2e/src/test/java/org/apache/dolphinscheduler/base/BaseDriver.java * Merge remote-tracking branch 'upstream/dev' into dev # Conflicts: # e2e/src/test/java/org/apache/dolphinscheduler/base/BaseDriver.java * Merge remote-tracking branch 'upstream/dev' into dev # Conflicts: # e2e/src/test/java/org/apache/dolphinscheduler/base/BaseDriver.java * modify workflow case Co-authored-by: chenxingchun <50446296+chenxingchun@users.noreply.github.com> --- .../dolphinscheduler/base/BaseDriver.java | 6 ++ .../common/BrowserCommon.java | 45 +++++++++++---- .../data/project/CreatWorkflowData.java | 12 +++- .../project/CreateWorkflowLocator.java | 34 +++++++++--- .../page/project/CreateProjectPage.java | 4 +- .../page/project/CreateWorkflowPage.java | 55 ++++++++++++++----- .../testcase/project/CreateProjectTest.java | 2 +- .../testcase/project/CreateWorkflowTest.java | 11 ++-- 8 files changed, 128 insertions(+), 41 deletions(-) diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/base/BaseDriver.java b/e2e/src/test/java/org/apache/dolphinscheduler/base/BaseDriver.java index 86a77ffcfd..7d3ab9b837 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/base/BaseDriver.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/base/BaseDriver.java @@ -27,6 +27,7 @@ import java.io.IOException; import java.util.concurrent.TimeUnit; import org.openqa.selenium.PageLoadStrategy; + /** * base driver class */ @@ -97,8 +98,13 @@ public class BaseDriver { /* driver setting wait time */ // implicitly wait time driver.manage().timeouts().implicitlyWait(implicitlyWait, TimeUnit.SECONDS); + + // page load timeout + driver.manage().timeouts().pageLoadTimeout(pageLoadTimeout, TimeUnit.SECONDS); + // page load timeout driver.manage().timeouts().pageLoadTimeout(pageLoadTimeout, TimeUnit.SECONDS); + // script timeout driver.manage().timeouts().setScriptTimeout(setScriptTimeout, TimeUnit.SECONDS); diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/common/BrowserCommon.java b/e2e/src/test/java/org/apache/dolphinscheduler/common/BrowserCommon.java index fb16162d55..072ccb652e 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/common/BrowserCommon.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/common/BrowserCommon.java @@ -99,8 +99,6 @@ public class BrowserCommon { // show wait timeout long timeout = Long.valueOf(PropertiesReader.getKey("driver.timeouts.webDriverWait")); wait = new WebDriverWait(driver, timeout); -// this.redisUtil = redisUtil; -// this.jedis = redisUtil.getJedis(); } @@ -158,22 +156,34 @@ public class BrowserCommon { * * @param locator By */ - public void clearInput(By locator) { + public WebElement clearInput(By locator) { WebElement clearElement = locateElement(locator); clearElement.click(); - clearElement.clear(); clearElement.sendKeys(Keys.chord(Keys.CONTROL, "a")); - clearElement.sendKeys(Keys.DELETE); + clearElement.sendKeys(Keys.BACK_SPACE); + return clearElement; } + /** + * input codeMirror + * + * @param codeMirrorLocator By codeMirror + * @param codeMirrorLineLocator By codeMirrorLine + + */ + public void inputCodeMirror(By codeMirrorLocator,By codeMirrorLineLocator,String content) { + WebElement codeMirrorElement = locateElement(codeMirrorLocator); + WebElement codeMirrorLineElement = locateElement(codeMirrorLineLocator); + codeMirrorElement.click(); + codeMirrorLineElement.sendKeys(content); + } /** * move to element - * * @param locator BY * @return actions */ - public Actions moveToElement(By locator) { + public Actions moveToElement(By locator){ return actions.moveToElement(locateElement(locator)); } @@ -182,13 +192,17 @@ public class BrowserCommon { * * @param source_locator BY * @param target_locator BY - * @param X X-axis - * @param Y Y-axis */ - public void dragAndDropBy(By source_locator, By target_locator, int X, int Y) { - WebElement sourcetElement = locateElement(source_locator); + public void dragAndDrop(By source_locator, By target_locator){ + WebElement sourceElement = locateElement(source_locator); + WebElement targetElement = locateElement(target_locator); + actions.dragAndDrop(sourceElement, targetElement).perform(); + actions.release(); + } + + public void moveToDragElement(By target_locator, int X, int Y){ WebElement targetElement = locateElement(target_locator); - actions.dragAndDrop(sourcetElement, targetElement).moveToElement(targetElement, X, Y).perform(); + actions.dragAndDropBy(targetElement, X, Y).perform(); actions.release(); } @@ -300,6 +314,13 @@ public class BrowserCommon { executeScript("window.scrollTo(0, document.body.scrollHeight)"); } + public void scrollToElementBottom() { + + WebElement webElement = driver.findElement(By.xpath("/html/body/div[4]/div/div[2]/div/div[2]/div/div[7]/div[3]")); + ((JavascriptExecutor) driver).executeScript("arguments[0].scrollIntoView(true);", webElement); + } + + /** * Page swipe makes the top of the element align with the top of the page * diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/data/project/CreatWorkflowData.java b/e2e/src/test/java/org/apache/dolphinscheduler/data/project/CreatWorkflowData.java index 1e45d47a92..765a54f406 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/data/project/CreatWorkflowData.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/data/project/CreatWorkflowData.java @@ -29,8 +29,18 @@ public class CreatWorkflowData { //input shell script public static final String SHELL_SCRIPT = "echo 1111111"; - public static final String WORKFLOW_TITLE = "创建流程定义 - DolphinScheduler"; + //input custom parameters + public static final String INPUT_CUSTOM_PARAMETERS = "selenium_parameter"; + + //input custom parameters value + public static final String INPUT_CUSTOM_PARAMETERS_VALUE = "selenium_parameter_123"; + //input add custom parameters + public static final String INPUT_ADD_CUSTOM_PARAMETERS = "selenium_parameter_delete"; + //input add custom parameters value + public static final String INPUT_ADD_CUSTOM_PARAMETERS_VALUE = "selenium_parameter_delete_456"; + //create workflow title + public static final String WORKFLOW_TITLE = "创建流程定义 - DolphinScheduler"; } diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/locator/project/CreateWorkflowLocator.java b/e2e/src/test/java/org/apache/dolphinscheduler/locator/project/CreateWorkflowLocator.java index 3bea648d57..f063d6ef61 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/locator/project/CreateWorkflowLocator.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/locator/project/CreateWorkflowLocator.java @@ -20,9 +20,6 @@ package org.apache.dolphinscheduler.locator.project; import org.openqa.selenium.By; public class CreateWorkflowLocator { - // click project manage - public static final By CLICK_PROJECT_MANAGE = By.xpath("//div[2]/div/a/span"); - // click project name public static final By CLICK_PROJECT_NAME = By.xpath("//span/a"); @@ -33,16 +30,13 @@ public class CreateWorkflowLocator { public static final By CLICK_CREATE_WORKFLOW_BUTTON = By.xpath("//button/span"); //mouse down at shell - public static final By MOUSE_DOWN_AT_SHELL = By.xpath("//*[@id='SHELL']/div/div"); + public static final By MOUSE_DOWN_AT_SHELL = By.xpath("//div[@id='SHELL']/div/div"); //mouse down at spark public static final By MOUSE_DOWN_AT_SPARK = By.xpath("//div[5]/div/div"); //mouse move at DAG - public static final By MOUSE_MOVE_SHELL_AT_DAG = By.xpath("//div[2]/div/div[2]/div[2]/div/div"); - -// //click shell task -// public static final By CLICK_SHELL_TASK = By.xpath("//div[2]/div/div[2]/div[2]/div/div"); + public static final By MOUSE_MOVE_SHELL_AT_DAG = By.xpath("//div[@id='canvas']"); //input shell task _name public static final By INPUT_SHELL_TASK_NAME = By.xpath("//input"); @@ -89,9 +83,33 @@ public class CreateWorkflowLocator { //input timeout public static final By SELECT_TIMEOUT = By.xpath("//div[3]/div[2]/label/div/input"); + //click codeMirror + public static final By CLICK_CODE_MIRROR = By.xpath("//div[5]/div/pre"); + //input script public static final By INPUT_SCRIPT = By.xpath("//div[2]/div/div/div/div/div/textarea"); + //click custom parameters + public static final By CLICK_CUSTOM_PARAMETERS = By.xpath("//span/a/em"); + + //input custom parameters + public static final By INPUT_CUSTOM_PARAMETERS = By.xpath("//div[2]/div/div/div/div/div/input"); + + //input custom parameters value + public static final By INPUT_CUSTOM_PARAMETERS_VALUE = By.xpath("//div[2]/input"); + + //click add custom parameters + public static final By CLICK_ADD_CUSTOM_PARAMETERS = By.xpath("//span[2]/a/em"); + + //input add custom parameters + public static final By INPUT_ADD_CUSTOM_PARAMETERS = By.xpath("//div[2]/div/div/div/div[2]/div/input"); + + //input add custom parameters value + public static final By INPUT_ADD_CUSTOM_PARAMETERS_VALUE = By.xpath("//div[2]/div[2]/input"); + //delete custom parameters + public static final By CLICK_DELETE_CUSTOM_PARAMETERS = By.xpath("//div[2]/span/a/em"); + //click submit button + public static final By CLICK_SUBMIT_BUTTON = By.xpath("//button[2]/span"); } diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/page/project/CreateProjectPage.java b/e2e/src/test/java/org/apache/dolphinscheduler/page/project/CreateProjectPage.java index 796171542b..8dd1010a82 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/page/project/CreateProjectPage.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/page/project/CreateProjectPage.java @@ -29,9 +29,10 @@ public class CreateProjectPage extends PageCommon { /** * jump page */ - public void jumpPage() throws InterruptedException { + public void jumpProjectManagePage() throws InterruptedException { Thread.sleep(TestConstant.ONE_THOUSANG); clickElement(CreateProjectLocator.PROJECT_MANAGE); + Thread.sleep(TestConstant.ONE_THOUSANG); } /** @@ -40,7 +41,6 @@ public class CreateProjectPage extends PageCommon { * @return Whether to enter the specified page after creat tenant */ public boolean createProject() throws InterruptedException { - Thread.sleep(TestConstant.ONE_THOUSANG); //click create project clickElement(CreateProjectLocator.CREATE_PROJECT_BUTTON); Thread.sleep(TestConstant.ONE_THOUSANG); diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/page/project/CreateWorkflowPage.java b/e2e/src/test/java/org/apache/dolphinscheduler/page/project/CreateWorkflowPage.java index 827198bd86..aeec83cd19 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/page/project/CreateWorkflowPage.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/page/project/CreateWorkflowPage.java @@ -19,7 +19,6 @@ package org.apache.dolphinscheduler.page.project; import org.apache.dolphinscheduler.common.PageCommon; import org.apache.dolphinscheduler.constant.TestConstant; import org.apache.dolphinscheduler.data.project.CreatWorkflowData; -import org.apache.dolphinscheduler.locator.project.CreateProjectLocator; import org.apache.dolphinscheduler.locator.project.CreateWorkflowLocator; import org.openqa.selenium.WebDriver; @@ -30,24 +29,23 @@ public class CreateWorkflowPage extends PageCommon { /** * jump page */ - public boolean CreateWorkflow() throws InterruptedException { - Thread.sleep(TestConstant.ONE_THOUSANG); - // click project manage - clickElement(CreateProjectLocator.PROJECT_MANAGE); - Thread.sleep(TestConstant.ONE_THOUSANG); + public boolean createWorkflow() throws InterruptedException { // click project name clickElement(CreateWorkflowLocator.CLICK_PROJECT_NAME); Thread.sleep(TestConstant.ONE_THOUSANG); + // click workflow define clickElement(CreateWorkflowLocator.CLICK_WORKFLOW_DEFINE); + Thread.sleep(TestConstant.ONE_THOUSANG); // click create workflow button clickElement(CreateWorkflowLocator.CLICK_CREATE_WORKFLOW_BUTTON); + Thread.sleep(TestConstant.ONE_THOUSANG); //drag shell_task - dragAndDropBy(CreateWorkflowLocator.MOUSE_DOWN_AT_SHELL,CreateWorkflowLocator.MOUSE_MOVE_SHELL_AT_DAG, 3 ,6); + dragAndDrop(CreateWorkflowLocator.MOUSE_DOWN_AT_SHELL,CreateWorkflowLocator.MOUSE_MOVE_SHELL_AT_DAG); //input shell task _name sendInput(CreateWorkflowLocator.INPUT_SHELL_TASK_NAME , CreatWorkflowData.SHELL_TASK_NAME); @@ -55,11 +53,9 @@ public class CreateWorkflowPage extends PageCommon { //click stop run type clickElement(CreateWorkflowLocator.CLICK_STOP_RUN_TYPE); - //click normal run type clickElement(CreateWorkflowLocator.CLICK_NORMAL_RUN_TYPE); - //input shell task description sendInput(CreateWorkflowLocator.INPUT_SHELL_TASK_DESCRIPTION , CreatWorkflowData.SHELL_TASK_DESCRIPTION); @@ -93,15 +89,48 @@ public class CreateWorkflowPage extends PageCommon { //select timeout alarm clickElement(CreateWorkflowLocator.SELECT_TIMEOUT_ALARM); - //clear input + //clear timeout + clearInput(CreateWorkflowLocator.SELECT_TIMEOUT); clearInput(CreateWorkflowLocator.SELECT_TIMEOUT); - Thread.sleep(TestConstant.ONE_THOUSANG); //input timeout sendInput(CreateWorkflowLocator.SELECT_TIMEOUT,CreatWorkflowData.INPUT_TIMEOUT); - //input script - sendInput(CreateWorkflowLocator.INPUT_SCRIPT, CreatWorkflowData.SHELL_SCRIPT); + //click codeMirror and input script + inputCodeMirror(CreateWorkflowLocator.CLICK_CODE_MIRROR, CreateWorkflowLocator.INPUT_SCRIPT,CreatWorkflowData.SHELL_SCRIPT); + scrollToElementBottom(); + Thread.sleep(TestConstant.ONE_THOUSANG); + + //click custom parameters + clickElement(CreateWorkflowLocator.CLICK_CUSTOM_PARAMETERS); + + //input custom parameters + sendInput(CreateWorkflowLocator.INPUT_CUSTOM_PARAMETERS, CreatWorkflowData.INPUT_CUSTOM_PARAMETERS); + + //input custom parameters value + sendInput(CreateWorkflowLocator.INPUT_CUSTOM_PARAMETERS_VALUE, CreatWorkflowData.INPUT_CUSTOM_PARAMETERS_VALUE); + + //click add custom parameters + clickElement(CreateWorkflowLocator.CLICK_ADD_CUSTOM_PARAMETERS); + + scrollToElementBottom(); + Thread.sleep(TestConstant.ONE_THOUSANG); + + //input add custom parameters + sendInput(CreateWorkflowLocator.INPUT_ADD_CUSTOM_PARAMETERS,CreatWorkflowData.INPUT_ADD_CUSTOM_PARAMETERS); + + //input add custom parameters value + sendInput(CreateWorkflowLocator.INPUT_ADD_CUSTOM_PARAMETERS_VALUE,CreatWorkflowData.INPUT_ADD_CUSTOM_PARAMETERS_VALUE); + + //click delete custom parameters + clickElement(CreateWorkflowLocator.CLICK_DELETE_CUSTOM_PARAMETERS); + Thread.sleep(TestConstant.ONE_THOUSANG); + + //click submit button + clickElement(CreateWorkflowLocator.CLICK_SUBMIT_BUTTON); + Thread.sleep(TestConstant.ONE_THOUSANG); + + moveToDragElement(CreateWorkflowLocator.MOUSE_MOVE_SHELL_AT_DAG,-300,-100); return ifTitleContains(CreatWorkflowData.WORKFLOW_TITLE); } diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/testcase/project/CreateProjectTest.java b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/project/CreateProjectTest.java index 66af10543b..8abd09d37c 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/testcase/project/CreateProjectTest.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/project/CreateProjectTest.java @@ -27,7 +27,7 @@ public class CreateProjectTest extends BaseTest { public void testUserManage() throws InterruptedException { createProjectPage = new CreateProjectPage(driver); // enter user manage page - createProjectPage.jumpPage(); + createProjectPage.jumpProjectManagePage(); //assert user manage page assert createProjectPage.createProject(); } diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/testcase/project/CreateWorkflowTest.java b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/project/CreateWorkflowTest.java index ac1bfec69f..6ac13f8124 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/testcase/project/CreateWorkflowTest.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/project/CreateWorkflowTest.java @@ -17,18 +17,21 @@ package org.apache.dolphinscheduler.testcase.project; import org.apache.dolphinscheduler.base.BaseTest; +import org.apache.dolphinscheduler.page.project.CreateProjectPage; import org.apache.dolphinscheduler.page.project.CreateWorkflowPage; import org.testng.annotations.Test; public class CreateWorkflowTest extends BaseTest { private CreateWorkflowPage createWorkflowPage; + private CreateProjectPage createProjectPage; + @Test(description = "CreateWorkflowTest", priority = 1) public void CreateWorkflowTest() throws InterruptedException { + createProjectPage = new CreateProjectPage(driver); + createProjectPage.jumpProjectManagePage(); createWorkflowPage = new CreateWorkflowPage(driver); - // enter user manage page -// createWorkflowPage.jumpPage(); - //assert user manage page - assert createWorkflowPage.CreateWorkflow(); + //assert create workflow + assert createWorkflowPage.createWorkflow(); } } From 84409b57ee47cb8300976fd2d7cc1022bf786b20 Mon Sep 17 00:00:00 2001 From: Yelli Date: Fri, 28 Feb 2020 16:15:21 +0800 Subject: [PATCH 042/221] fix taskinstance.isSubprocess misjudgment (#2007) * add processInstanceService UT * fix taskInstance.isSubProcess add processInstanceService UT * add taskInstance.isSubProcess UT * modify taskInstance.isSubProcess UT * add license * remove author&date --- .../service/ProcessInstanceServiceTest.java | 345 +++++++++++++++++- .../dao/entity/TaskInstance.java | 2 +- .../dao/entity/TaskInstanceTest.java | 39 ++ pom.xml | 1 + 4 files changed, 374 insertions(+), 13 deletions(-) create mode 100644 dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/entity/TaskInstanceTest.java diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessInstanceServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessInstanceServiceTest.java index 18f1d6947f..959dca21bf 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessInstanceServiceTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/ProcessInstanceServiceTest.java @@ -19,14 +19,11 @@ package org.apache.dolphinscheduler.api.service; import com.baomidou.mybatisplus.extension.plugins.pagination.Page; import org.apache.dolphinscheduler.api.ApiApplicationServer; import org.apache.dolphinscheduler.api.enums.Status; +import org.apache.dolphinscheduler.api.utils.Result; import org.apache.dolphinscheduler.common.Constants; -import org.apache.dolphinscheduler.common.enums.DependResult; -import org.apache.dolphinscheduler.common.enums.ExecutionStatus; -import org.apache.dolphinscheduler.common.enums.UserType; +import org.apache.dolphinscheduler.common.enums.*; import org.apache.dolphinscheduler.common.utils.DateUtils; -import org.apache.dolphinscheduler.dao.entity.ProcessInstance; -import org.apache.dolphinscheduler.dao.entity.Project; -import org.apache.dolphinscheduler.dao.entity.User; +import org.apache.dolphinscheduler.dao.entity.*; import org.apache.dolphinscheduler.dao.mapper.*; import org.apache.dolphinscheduler.service.process.ProcessService; import org.junit.Assert; @@ -42,6 +39,7 @@ import org.springframework.boot.test.context.SpringBootTest; import java.io.IOException; import java.text.MessageFormat; +import java.text.ParseException; import java.util.*; import static org.mockito.ArgumentMatchers.eq; @@ -88,6 +86,13 @@ public class ProcessInstanceServiceTest { @Mock UsersService usersService; + private String shellJson = "{\"globalParams\":[],\"tasks\":[{\"type\":\"SHELL\",\"id\":\"tasks-9527\",\"name\":\"shell-1\"," + + "\"params\":{\"resourceList\":[],\"localParams\":[],\"rawScript\":\"#!/bin/bash\\necho \\\"shell-1\\\"\"}," + + "\"description\":\"\",\"runFlag\":\"NORMAL\",\"dependence\":{},\"maxRetryTimes\":\"0\",\"retryInterval\":\"1\"," + + "\"timeout\":{\"strategy\":\"\",\"interval\":1,\"enable\":false},\"taskInstancePriority\":\"MEDIUM\"," + + "\"workerGroupId\":-1,\"preTasks\":[]}],\"tenantId\":1,\"timeout\":0}"; + + @Test public void testQueryProcessInstanceList() { String projectName = "project_test1"; @@ -97,7 +102,7 @@ public class ProcessInstanceServiceTest { //project auth fail when(projectMapper.queryByName(projectName)).thenReturn(null); - when(projectService.checkProjectAndAuth(loginUser,null,projectName)).thenReturn(result); + when(projectService.checkProjectAndAuth(loginUser, null, projectName)).thenReturn(result); Map proejctAuthFailRes = processInstanceService.queryProcessInstanceList(loginUser, projectName, 46, "2020-01-01 00:00:00", "2020-01-02 00:00:00", "", "test_user", ExecutionStatus.SUBMITTED_SUCCESS, "192.168.xx.xx", 1, 10); @@ -114,7 +119,7 @@ public class ProcessInstanceServiceTest { processInstanceList.add(processInstance); pageReturn.setRecords(processInstanceList); when(projectMapper.queryByName(projectName)).thenReturn(project); - when(projectService.checkProjectAndAuth(loginUser,project,projectName)).thenReturn(result); + when(projectService.checkProjectAndAuth(loginUser, project, projectName)).thenReturn(result); when(usersService.queryUser(loginUser.getId())).thenReturn(loginUser); when(usersService.getUserIdByName(loginUser.getUserName())).thenReturn(loginUser.getId()); when(processInstanceMapper.queryProcessInstanceListPaging(Mockito.any(Page.class), eq(project.getId()), eq(1), eq(""), eq(-1), Mockito.any(), @@ -144,21 +149,293 @@ public class ProcessInstanceServiceTest { } @Test - public void testDependResult(){ + public void testQueryProcessInstanceById() { + String projectName = "project_test1"; + User loginUser = getAdminUser(); + Map result = new HashMap<>(5); + putMsg(result, Status.PROJECT_NOT_FOUNT, projectName); + + //project auth fail + when(projectMapper.queryByName(projectName)).thenReturn(null); + when(projectService.checkProjectAndAuth(loginUser, null, projectName)).thenReturn(result); + Map proejctAuthFailRes = processInstanceService.queryProcessInstanceById(loginUser, projectName, 1); + Assert.assertEquals(Status.PROJECT_NOT_FOUNT, proejctAuthFailRes.get(Constants.STATUS)); + + //project auth success + ProcessInstance processInstance = getProcessInstance(); + processInstance.setWorkerGroupId(-1); + processInstance.setReceivers("xxx@qq.com"); + processInstance.setReceiversCc("xxx@qq.com"); + processInstance.setProcessDefinitionId(46); + putMsg(result, Status.SUCCESS, projectName); + Project project = getProject(projectName); + ProcessDefinition processDefinition = getProcessDefinition(); + when(projectMapper.queryByName(projectName)).thenReturn(project); + when(projectService.checkProjectAndAuth(loginUser, project, projectName)).thenReturn(result); + when(processService.findProcessInstanceDetailById(processInstance.getId())).thenReturn(processInstance); + when(processService.findProcessDefineById(processInstance.getProcessDefinitionId())).thenReturn(processDefinition); + Map successRes = processInstanceService.queryProcessInstanceById(loginUser, projectName, 1); + Assert.assertEquals(Status.SUCCESS, successRes.get(Constants.STATUS)); + + //worker group null + processInstance.setWorkerGroupId(1); + when(workerGroupMapper.selectById(processInstance.getWorkerGroupId())).thenReturn(null); + Map workerNullRes = processInstanceService.queryProcessInstanceById(loginUser, projectName, 1); + Assert.assertEquals(Status.SUCCESS, workerNullRes.get(Constants.STATUS)); + + //worker group exist + WorkerGroup workerGroup = getWorkGroup(); + when(workerGroupMapper.selectById(processInstance.getWorkerGroupId())).thenReturn(workerGroup); + processInstance.setWorkerGroupId(1); + when(workerGroupMapper.selectById(processInstance.getWorkerGroupId())).thenReturn(null); + Map workerExistRes = processInstanceService.queryProcessInstanceById(loginUser, projectName, 1); + Assert.assertEquals(Status.SUCCESS, workerExistRes.get(Constants.STATUS)); + } + + @Test + public void testQueryTaskListByProcessId() throws IOException { + String projectName = "project_test1"; + User loginUser = getAdminUser(); + Map result = new HashMap<>(5); + putMsg(result, Status.PROJECT_NOT_FOUNT, projectName); + + //project auth fail + when(projectMapper.queryByName(projectName)).thenReturn(null); + when(projectService.checkProjectAndAuth(loginUser, null, projectName)).thenReturn(result); + Map proejctAuthFailRes = processInstanceService.queryTaskListByProcessId(loginUser, projectName, 1); + Assert.assertEquals(Status.PROJECT_NOT_FOUNT, proejctAuthFailRes.get(Constants.STATUS)); + + //project auth success + putMsg(result, Status.SUCCESS, projectName); + Project project = getProject(projectName); + ProcessInstance processInstance = getProcessInstance(); + processInstance.setState(ExecutionStatus.SUCCESS); + TaskInstance taskInstance = new TaskInstance(); + taskInstance.setTaskType(TaskType.SHELL.getDescp()); + List taskInstanceList = new ArrayList<>(); + taskInstanceList.add(taskInstance); + Result res = new Result(); + res.setCode(Status.SUCCESS.ordinal()); + res.setData("xxx"); + when(projectMapper.queryByName(projectName)).thenReturn(project); + when(projectService.checkProjectAndAuth(loginUser, project, projectName)).thenReturn(result); + when(processService.findProcessInstanceDetailById(processInstance.getId())).thenReturn(processInstance); + when(processService.findValidTaskListByProcessId(processInstance.getId())).thenReturn(taskInstanceList); + when(loggerService.queryLog(taskInstance.getId(), 0, 4098)).thenReturn(res); + Map successRes = processInstanceService.queryTaskListByProcessId(loginUser, projectName, 1); + Assert.assertEquals(Status.SUCCESS, successRes.get(Constants.STATUS)); + } + + + @Test + public void testParseLogForDependentResult() { String logString = "[INFO] 2019-03-19 17:11:08.475 org.apache.dolphinscheduler.server.worker.log.TaskLogger:[172] - [taskAppId=TASK_223_10739_452334] dependent item complete :|| 223-ALL-day-last1Day,SUCCESS\n" + "[INFO] 2019-03-19 17:11:08.476 org.apache.dolphinscheduler.server.worker.runner.TaskScheduleThread:[172] - task : 223_10739_452334 exit status code : 0\n" + "[root@node2 current]# "; try { Map resultMap = processInstanceService.parseLogForDependentResult(logString); - Assert.assertEquals(1 , resultMap.size()); + Assert.assertEquals(1, resultMap.size()); } catch (IOException e) { } } + @Test + public void testQuerySubProcessInstanceByTaskId() { + String projectName = "project_test1"; + User loginUser = getAdminUser(); + Map result = new HashMap<>(5); + putMsg(result, Status.PROJECT_NOT_FOUNT, projectName); + + //project auth fail + when(projectMapper.queryByName(projectName)).thenReturn(null); + when(projectService.checkProjectAndAuth(loginUser, null, projectName)).thenReturn(result); + Map proejctAuthFailRes = processInstanceService.querySubProcessInstanceByTaskId(loginUser, projectName, 1); + Assert.assertEquals(Status.PROJECT_NOT_FOUNT, proejctAuthFailRes.get(Constants.STATUS)); + + //task null + Project project = getProject(projectName); + putMsg(result, Status.SUCCESS, projectName); + when(projectMapper.queryByName(projectName)).thenReturn(project); + when(projectService.checkProjectAndAuth(loginUser, project, projectName)).thenReturn(result); + when(processService.findTaskInstanceById(1)).thenReturn(null); + Map taskNullRes = processInstanceService.querySubProcessInstanceByTaskId(loginUser, projectName, 1); + Assert.assertEquals(Status.TASK_INSTANCE_NOT_EXISTS, taskNullRes.get(Constants.STATUS)); + + //task not sub process + TaskInstance taskInstance = getTaskInstance(); + taskInstance.setTaskType(TaskType.HTTP.getDescp()); + taskInstance.setProcessInstanceId(1); + when(processService.findTaskInstanceById(1)).thenReturn(taskInstance); + Map notSubprocessRes = processInstanceService.querySubProcessInstanceByTaskId(loginUser, projectName, 1); + Assert.assertEquals(Status.TASK_INSTANCE_NOT_SUB_WORKFLOW_INSTANCE, notSubprocessRes.get(Constants.STATUS)); + + //sub process not exist + TaskInstance subTask = getTaskInstance(); + subTask.setTaskType(TaskType.SUB_PROCESS.getDescp()); + subTask.setProcessInstanceId(1); + when(processService.findTaskInstanceById(subTask.getId())).thenReturn(subTask); + when(processService.findSubProcessInstance(subTask.getProcessInstanceId(), subTask.getId())).thenReturn(null); + Map subprocessNotExistRes = processInstanceService.querySubProcessInstanceByTaskId(loginUser, projectName, 1); + Assert.assertEquals(Status.SUB_PROCESS_INSTANCE_NOT_EXIST, subprocessNotExistRes.get(Constants.STATUS)); + + //sub process exist + ProcessInstance processInstance = getProcessInstance(); + when(processService.findSubProcessInstance(taskInstance.getProcessInstanceId(), taskInstance.getId())).thenReturn(processInstance); + Map subprocessExistRes = processInstanceService.querySubProcessInstanceByTaskId(loginUser, projectName, 1); + Assert.assertEquals(Status.SUCCESS, subprocessExistRes.get(Constants.STATUS)); + } + + @Test + public void testUpdateProcessInstance() throws ParseException { + String projectName = "project_test1"; + User loginUser = getAdminUser(); + Map result = new HashMap<>(5); + putMsg(result, Status.PROJECT_NOT_FOUNT, projectName); + + //project auth fail + when(projectMapper.queryByName(projectName)).thenReturn(null); + when(projectService.checkProjectAndAuth(loginUser, null, projectName)).thenReturn(result); + Map proejctAuthFailRes = processInstanceService.updateProcessInstance(loginUser, projectName, 1, + shellJson, "2020-02-21 00:00:00", true, Flag.YES, "", ""); + Assert.assertEquals(Status.PROJECT_NOT_FOUNT, proejctAuthFailRes.get(Constants.STATUS)); + + //process instance null + Project project = getProject(projectName); + putMsg(result, Status.SUCCESS, projectName); + ProcessInstance processInstance = getProcessInstance(); + when(projectMapper.queryByName(projectName)).thenReturn(project); + when(projectService.checkProjectAndAuth(loginUser, project, projectName)).thenReturn(result); + when(processService.findProcessInstanceDetailById(1)).thenReturn(null); + Map processInstanceNullRes = processInstanceService.updateProcessInstance(loginUser, projectName, 1, + shellJson, "2020-02-21 00:00:00", true, Flag.YES, "", ""); + Assert.assertEquals(Status.PROCESS_INSTANCE_NOT_EXIST, processInstanceNullRes.get(Constants.STATUS)); + + //process instance not finish + when(processService.findProcessInstanceDetailById(1)).thenReturn(processInstance); + processInstance.setState(ExecutionStatus.RUNNING_EXEUTION); + Map processInstanceNotFinishRes = processInstanceService.updateProcessInstance(loginUser, projectName, 1, + shellJson, "2020-02-21 00:00:00", true, Flag.YES, "", ""); + Assert.assertEquals(Status.PROCESS_INSTANCE_STATE_OPERATION_ERROR, processInstanceNotFinishRes.get(Constants.STATUS)); + + //process instance finish + processInstance.setState(ExecutionStatus.SUCCESS); + processInstance.setTimeout(3000); + processInstance.setCommandType(CommandType.STOP); + ProcessDefinition processDefinition = getProcessDefinition(); + processDefinition.setUserId(1); + Tenant tenant = new Tenant(); + tenant.setId(1); + tenant.setTenantCode("test_tenant"); + when(processService.findProcessDefineById(processInstance.getProcessDefinitionId())).thenReturn(processDefinition); + when(processService.getTenantForProcess(Mockito.anyInt(), Mockito.anyInt())).thenReturn(tenant); + when(processService.updateProcessInstance(processInstance)).thenReturn(1); + when(processDefinitionService.checkProcessNodeList(Mockito.any(), eq(shellJson))).thenReturn(result); + Map processInstanceFinishRes = processInstanceService.updateProcessInstance(loginUser, projectName, 1, + shellJson, "2020-02-21 00:00:00", true, Flag.YES, "", ""); + Assert.assertEquals(Status.UPDATE_PROCESS_INSTANCE_ERROR, processInstanceFinishRes.get(Constants.STATUS)); + + //success + when(processDefineMapper.updateById(processDefinition)).thenReturn(1); + Map successRes = processInstanceService.updateProcessInstance(loginUser, projectName, 1, + shellJson, "2020-02-21 00:00:00", true, Flag.YES, "", ""); + Assert.assertEquals(Status.SUCCESS, successRes.get(Constants.STATUS)); + } + + @Test + public void testQueryParentInstanceBySubId() { + String projectName = "project_test1"; + User loginUser = getAdminUser(); + Map result = new HashMap<>(5); + putMsg(result, Status.PROJECT_NOT_FOUNT, projectName); + + //project auth fail + when(projectMapper.queryByName(projectName)).thenReturn(null); + when(projectService.checkProjectAndAuth(loginUser, null, projectName)).thenReturn(result); + Map proejctAuthFailRes = processInstanceService.queryParentInstanceBySubId(loginUser, projectName, 1); + Assert.assertEquals(Status.PROJECT_NOT_FOUNT, proejctAuthFailRes.get(Constants.STATUS)); + + //process instance null + Project project = getProject(projectName); + putMsg(result, Status.SUCCESS, projectName); + when(projectMapper.queryByName(projectName)).thenReturn(project); + when(projectService.checkProjectAndAuth(loginUser, project, projectName)).thenReturn(result); + when(processService.findProcessInstanceDetailById(1)).thenReturn(null); + Map processInstanceNullRes = processInstanceService.queryParentInstanceBySubId(loginUser, projectName, 1); + Assert.assertEquals(Status.PROCESS_INSTANCE_NOT_EXIST, processInstanceNullRes.get(Constants.STATUS)); + + //not sub process + ProcessInstance processInstance = getProcessInstance(); + processInstance.setIsSubProcess(Flag.NO); + when(processService.findProcessInstanceDetailById(1)).thenReturn(processInstance); + Map notSubProcessRes = processInstanceService.queryParentInstanceBySubId(loginUser, projectName, 1); + Assert.assertEquals(Status.PROCESS_INSTANCE_NOT_SUB_PROCESS_INSTANCE, notSubProcessRes.get(Constants.STATUS)); + + //sub process + processInstance.setIsSubProcess(Flag.YES); + when(processService.findParentProcessInstance(1)).thenReturn(null); + Map subProcessNullRes = processInstanceService.queryParentInstanceBySubId(loginUser, projectName, 1); + Assert.assertEquals(Status.SUB_PROCESS_INSTANCE_NOT_EXIST, subProcessNullRes.get(Constants.STATUS)); + + //success + when(processService.findParentProcessInstance(1)).thenReturn(processInstance); + Map successRes = processInstanceService.queryParentInstanceBySubId(loginUser, projectName, 1); + Assert.assertEquals(Status.SUCCESS, successRes.get(Constants.STATUS)); + } + + @Test + public void testDeleteProcessInstanceById() { + String projectName = "project_test1"; + User loginUser = getAdminUser(); + Map result = new HashMap<>(5); + putMsg(result, Status.PROJECT_NOT_FOUNT, projectName); + + //project auth fail + when(projectMapper.queryByName(projectName)).thenReturn(null); + when(projectService.checkProjectAndAuth(loginUser, null, projectName)).thenReturn(result); + Map proejctAuthFailRes = processInstanceService.deleteProcessInstanceById(loginUser, projectName, 1, Mockito.any()); + Assert.assertEquals(Status.PROJECT_NOT_FOUNT, proejctAuthFailRes.get(Constants.STATUS)); + + //process instance null + Project project = getProject(projectName); + putMsg(result, Status.SUCCESS, projectName); + when(projectMapper.queryByName(projectName)).thenReturn(project); + when(projectService.checkProjectAndAuth(loginUser, project, projectName)).thenReturn(result); + when(processService.findProcessInstanceDetailById(1)).thenReturn(null); + Map processInstanceNullRes = processInstanceService.deleteProcessInstanceById(loginUser, projectName, 1, Mockito.any()); + Assert.assertEquals(Status.PROCESS_INSTANCE_NOT_EXIST, processInstanceNullRes.get(Constants.STATUS)); + } + + @Test + public void testViewVariables() throws Exception { + //process instance not null + ProcessInstance processInstance = getProcessInstance(); + processInstance.setCommandType(CommandType.SCHEDULER); + processInstance.setScheduleTime(new Date()); + processInstance.setProcessInstanceJson(shellJson); + processInstance.setGlobalParams(""); + when(processInstanceMapper.queryDetailById(1)).thenReturn(processInstance); + Map successRes = processInstanceService.viewVariables(1); + Assert.assertEquals(Status.SUCCESS, successRes.get(Constants.STATUS)); + } + + @Test + public void testViewGantt() throws Exception { + ProcessInstance processInstance = getProcessInstance(); + processInstance.setProcessInstanceJson(shellJson); + TaskInstance taskInstance = getTaskInstance(); + taskInstance.setState(ExecutionStatus.RUNNING_EXEUTION); + taskInstance.setStartTime(new Date()); + when(processInstanceMapper.queryDetailById(1)).thenReturn(processInstance); + when(taskInstanceMapper.queryByInstanceIdAndName(Mockito.anyInt(), Mockito.any())).thenReturn(taskInstance); + Map successRes = processInstanceService.viewGantt(1); + Assert.assertEquals(Status.SUCCESS, successRes.get(Constants.STATUS)); + } + /** * get Mock Admin User + * * @return admin user */ private User getAdminUser() { @@ -171,19 +448,21 @@ public class ProcessInstanceServiceTest { /** * get mock Project + * * @param projectName projectName * @return Project */ - private Project getProject(String projectName){ + private Project getProject(String projectName) { Project project = new Project(); project.setId(1); project.setName(projectName); project.setUserId(1); - return project; + return project; } /** * get Mock process instance + * * @return process instance */ private ProcessInstance getProcessInstance() { @@ -195,6 +474,48 @@ public class ProcessInstanceServiceTest { return processInstance; } + /** + * get mock processDefinition + * + * @return ProcessDefinition + */ + private ProcessDefinition getProcessDefinition() { + ProcessDefinition processDefinition = new ProcessDefinition(); + processDefinition.setId(46); + processDefinition.setName("test_pdf"); + processDefinition.setProjectId(2); + processDefinition.setTenantId(1); + processDefinition.setDescription(""); + return processDefinition; + } + + /** + * get Mock worker group + * + * @return worker group + */ + private WorkerGroup getWorkGroup() { + WorkerGroup workerGroup = new WorkerGroup(); + workerGroup.setId(1); + workerGroup.setName("test_workergroup"); + return workerGroup; + } + + /** + * get Mock task instance + * + * @return task instance + */ + private TaskInstance getTaskInstance() { + TaskInstance taskInstance = new TaskInstance(); + taskInstance.setId(1); + taskInstance.setName("test_task_instance"); + taskInstance.setStartTime(new Date()); + taskInstance.setEndTime(new Date()); + taskInstance.setExecutorId(-1); + return taskInstance; + } + private void putMsg(Map result, Status status, Object... statusParams) { result.put(Constants.STATUS, status); if (statusParams != null && statusParams.length > 0) { diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java index 785b67eb28..c692575e3a 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java @@ -374,7 +374,7 @@ public class TaskInstance { public Boolean isSubProcess(){ - return TaskType.SUB_PROCESS.toString().equals(this.taskType.toUpperCase()); + return TaskType.SUB_PROCESS.getDescp().equals(this.taskType); } public String getDependency(){ diff --git a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/entity/TaskInstanceTest.java b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/entity/TaskInstanceTest.java new file mode 100644 index 0000000000..e165da1e88 --- /dev/null +++ b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/entity/TaskInstanceTest.java @@ -0,0 +1,39 @@ +/* + * 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.dao.entity; + +import org.junit.Assert; +import org.junit.Test; + +public class TaskInstanceTest { + + /** + * task instance sub process + */ + @Test + public void testTaskInstanceIsSubProcess() { + TaskInstance taskInstance = new TaskInstance(); + + //sub process + taskInstance.setTaskType("sub process"); + Assert.assertTrue(taskInstance.isSubProcess()); + + //not sub process + taskInstance.setTaskType("http"); + Assert.assertFalse(taskInstance.isSubProcess()); + } +} diff --git a/pom.xml b/pom.xml index 076b85611f..8a1e9b98f2 100644 --- a/pom.xml +++ b/pom.xml @@ -722,6 +722,7 @@ **/dao/mapper/AlertGroupMapperTest.java **/dao/mapper/AlertMapperTest.java **/dao/mapper/CommandMapperTest.java + **/dao/entity/TaskInstanceTest.java **/dao/cron/CronUtilsTest.java **/dao/utils/DagHelperTest.java **/alert/template/AlertTemplateFactoryTest.java From 0df7c6719d4073c15be5a432d802d35867c4ee0c Mon Sep 17 00:00:00 2001 From: bao liang <29528966+lenboo@users.noreply.github.com> Date: Fri, 28 Feb 2020 17:08:34 +0800 Subject: [PATCH 043/221] [New Feature] add conditions task #205 (#2003) * add funtion conditions task * update conditions tasks * update conditions for ui * update conditions * update * revert * update --- .../common/enums/TaskType.java | 6 +- .../common/model/DependentItem.java | 10 + .../common/model/TaskNode.java | 19 ++ .../task/conditions/ConditionsParameters.java | 79 ++++++ .../common/utils/TaskParametersUtils.java | 3 + .../master/runner/MasterExecThread.java | 207 ++++++++++---- .../worker/runner/TaskScheduleThread.java | 37 ++- .../server/worker/task/AbstractTask.java | 4 + .../server/worker/task/TaskManager.java | 3 + .../task/conditions/ConditionsTask.java | 145 ++++++++++ .../server/master/MasterExecThreadTest.java | 1 + dolphinscheduler-ui/.env | 2 +- .../js/conf/home/pages/dag/_source/config.js | 4 + .../js/conf/home/pages/dag/_source/dag.scss | 3 + .../js/conf/home/pages/dag/_source/dag.vue | 32 ++- .../pages/dag/_source/formModel/formModel.vue | 88 +++++- .../formModel/tasks/_source/commcon.js | 13 +- .../formModel/tasks/_source/nodeStatus.vue | 231 +++++++++++++++ .../_source/formModel/tasks/conditions.vue | 265 ++++++++++++++++++ .../pages/dag/_source/plugIn/jsPlumbHandle.js | 14 +- .../home/pages/dag/_source/plugIn/util.js | 5 +- .../home/pages/dag/img/toobar_CONDITIONS.png | Bin 0 -> 1367 bytes .../src/js/conf/home/store/dag/mutations.js | 1 + .../src/js/module/i18n/locale/en_US.js | 8 +- .../src/js/module/i18n/locale/zh_CN.js | 5 +- 25 files changed, 1098 insertions(+), 87 deletions(-) create mode 100644 dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/conditions/ConditionsParameters.java create mode 100644 dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/conditions/ConditionsTask.java mode change 100644 => 100755 dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/config.js mode change 100644 => 100755 dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.scss mode change 100644 => 100755 dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.vue mode change 100644 => 100755 dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue mode change 100644 => 100755 dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/commcon.js create mode 100644 dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/nodeStatus.vue create mode 100644 dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/conditions.vue mode change 100644 => 100755 dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/plugIn/jsPlumbHandle.js mode change 100644 => 100755 dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/plugIn/util.js create mode 100644 dolphinscheduler-ui/src/js/conf/home/pages/dag/img/toobar_CONDITIONS.png mode change 100644 => 100755 dolphinscheduler-ui/src/js/conf/home/store/dag/mutations.js mode change 100644 => 100755 dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js mode change 100644 => 100755 dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/TaskType.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/TaskType.java index b56e72560c..1f85432bd2 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/TaskType.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/enums/TaskType.java @@ -34,7 +34,8 @@ public enum TaskType { * 8 FLINK * 9 HTTP * 10 DATAX - * 11 SQOOP + * 11 CONDITIONS + * 12 SQOOP */ SHELL(0, "shell"), SQL(1, "sql"), @@ -47,7 +48,8 @@ public enum TaskType { FLINK(8, "flink"), HTTP(9, "http"), DATAX(10, "datax"), - SQOOP(11, "sqoop"); + CONDITIONS(11, "conditions"), + SQOOP(12, "sqoop"); TaskType(int code, String descp){ this.code = code; diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/DependentItem.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/DependentItem.java index 484a2f7ac8..6c09064eae 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/DependentItem.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/DependentItem.java @@ -17,6 +17,7 @@ package org.apache.dolphinscheduler.common.model; import org.apache.dolphinscheduler.common.enums.DependResult; +import org.apache.dolphinscheduler.common.enums.ExecutionStatus; /** * dependent item @@ -28,6 +29,7 @@ public class DependentItem { private String cycle; private String dateValue; private DependResult dependResult; + private ExecutionStatus status; public String getKey(){ @@ -77,4 +79,12 @@ public class DependentItem { public void setDependResult(DependResult dependResult) { this.dependResult = dependResult; } + + public ExecutionStatus getStatus() { + return status; + } + + public void setStatus(ExecutionStatus status) { + this.status = status; + } } diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/TaskNode.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/TaskNode.java index 40efd0a24f..b45bd8aeb8 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/TaskNode.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/TaskNode.java @@ -19,6 +19,7 @@ package org.apache.dolphinscheduler.common.model; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.Priority; import org.apache.dolphinscheduler.common.enums.TaskTimeoutStrategy; +import org.apache.dolphinscheduler.common.enums.TaskType; import org.apache.dolphinscheduler.common.task.TaskTimeoutParameter; import org.apache.dolphinscheduler.common.utils.CollectionUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils; @@ -108,6 +109,11 @@ public class TaskNode { @JsonSerialize(using = JSONUtils.JsonDataSerializer.class) private String dependence; + + @JsonDeserialize(using = JSONUtils.JsonDataDeserializer.class) + @JsonSerialize(using = JSONUtils.JsonDataSerializer.class) + private String conditionResult; + /** * task instance priority */ @@ -230,6 +236,7 @@ public class TaskNode { Objects.equals(extras, taskNode.extras) && Objects.equals(runFlag, taskNode.runFlag) && Objects.equals(dependence, taskNode.dependence) && + Objects.equals(conditionResult, taskNode.conditionResult) && Objects.equals(workerGroupId, taskNode.workerGroupId) && CollectionUtils.equalLists(depList, taskNode.depList); } @@ -292,6 +299,10 @@ public class TaskNode { return new TaskTimeoutParameter(false); } + public boolean isConditionsTask(){ + return this.getType().toUpperCase().equals(TaskType.CONDITIONS.toString()); + } + @Override public String toString() { return "TaskNode{" + @@ -321,4 +332,12 @@ public class TaskNode { public void setWorkerGroupId(int workerGroupId) { this.workerGroupId = workerGroupId; } + + public String getConditionResult() { + return conditionResult; + } + + public void setConditionResult(String conditionResult) { + this.conditionResult = conditionResult; + } } diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/conditions/ConditionsParameters.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/conditions/ConditionsParameters.java new file mode 100644 index 0000000000..5714b5ef3e --- /dev/null +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/conditions/ConditionsParameters.java @@ -0,0 +1,79 @@ +/* + * 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.common.task.conditions; + +import org.apache.dolphinscheduler.common.enums.DependentRelation; +import org.apache.dolphinscheduler.common.model.DependentTaskModel; +import org.apache.dolphinscheduler.common.task.AbstractParameters; + +import java.util.List; + +public class ConditionsParameters extends AbstractParameters { + + //depend node list and state, only need task name + private List dependTaskList; + private DependentRelation dependRelation; + + // node list to run when success + private List successNode; + + // node list to run when failed + private List failedNode; + + + @Override + public boolean checkParameters() { + return true; + } + + @Override + public List getResourceFilesList() { + return null; + } + + public List getDependTaskList() { + return dependTaskList; + } + + public void setDependTaskList(List dependTaskList) { + this.dependTaskList = dependTaskList; + } + + public DependentRelation getDependRelation() { + return dependRelation; + } + + public void setDependRelation(DependentRelation dependRelation) { + this.dependRelation = dependRelation; + } + + public List getSuccessNode() { + return successNode; + } + + public void setSuccessNode(List successNode) { + this.successNode = successNode; + } + + public List getFailedNode() { + return failedNode; + } + + public void setFailedNode(List failedNode) { + this.failedNode = failedNode; + } +} diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/TaskParametersUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/TaskParametersUtils.java index 7a0e069a9a..a3492f49fa 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/TaskParametersUtils.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/TaskParametersUtils.java @@ -18,6 +18,7 @@ package org.apache.dolphinscheduler.common.utils; import org.apache.dolphinscheduler.common.enums.TaskType; import org.apache.dolphinscheduler.common.task.AbstractParameters; +import org.apache.dolphinscheduler.common.task.conditions.ConditionsParameters; import org.apache.dolphinscheduler.common.task.dependent.DependentParameters; import org.apache.dolphinscheduler.common.task.datax.DataxParameters; import org.apache.dolphinscheduler.common.task.flink.FlinkParameters; @@ -72,6 +73,8 @@ public class TaskParametersUtils { return JSONUtils.parseObject(parameter, HttpParameters.class); case DATAX: return JSONUtils.parseObject(parameter, DataxParameters.class); + case CONDITIONS: + return JSONUtils.parseObject(parameter, ConditionsParameters.class); case SQOOP: return JSONUtils.parseObject(parameter, SqoopParameters.class); default: diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java index f5e31210a0..4b22b27bec 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java @@ -25,6 +25,7 @@ import org.apache.dolphinscheduler.common.graph.DAG; import org.apache.dolphinscheduler.common.model.TaskNode; import org.apache.dolphinscheduler.common.model.TaskNodeRelation; import org.apache.dolphinscheduler.common.process.ProcessDag; +import org.apache.dolphinscheduler.common.task.conditions.ConditionsParameters; import org.apache.dolphinscheduler.common.thread.Stopper; import org.apache.dolphinscheduler.common.thread.ThreadUtils; import org.apache.dolphinscheduler.common.utils.*; @@ -109,6 +110,11 @@ public class MasterExecThread implements Runnable { */ private Map forbiddenTaskList = new ConcurrentHashMap<>(); + /** + * skip task map + */ + private Map skipTaskNodeList = new ConcurrentHashMap<>(); + /** * recover tolerance fault task list */ @@ -434,7 +440,7 @@ public class MasterExecThread implements Runnable { * @return TaskInstance */ private TaskInstance createTaskInstance(ProcessInstance processInstance, String nodeName, - TaskNode taskNode, String parentNodeName) { + TaskNode taskNode) { TaskInstance taskInstance = findTaskIfExists(nodeName); if(taskInstance == null){ @@ -484,58 +490,140 @@ public class MasterExecThread implements Runnable { } /** - * get post task instance by node - * @param dag dag - * @param parentNodeName parent node name - * @return task instance list + * is there have conditions after the parent node + * @param parentNodeName + * @return */ - private List getPostTaskInstanceByNode(DAG dag, String parentNodeName){ + private boolean haveConditionsAfterNode(String parentNodeName){ - List postTaskList = new ArrayList<>(); + boolean result = false; Collection startVertex = DagHelper.getStartVertex(parentNodeName, dag, completeTaskList); if(startVertex == null){ - return postTaskList; + return result; + } + for(String nodeName : startVertex){ + TaskNode taskNode = dag.getNode(nodeName); + if(taskNode.getType().equals(TaskType.CONDITIONS.toString())){ + result = true; + break; + } } + return result; + } - for (String nodeName : startVertex){ - // encapsulation task instance - TaskInstance taskInstance = createTaskInstance(processInstance, nodeName , - dag.getNode(nodeName),parentNodeName); - postTaskList.add(taskInstance); + /** + * if all of the task dependence are skip, skip it too. + * @param taskNode + * @return + */ + private boolean isTaskNodeNeedSkip(TaskNode taskNode){ + if(CollectionUtils.isEmpty(taskNode.getDepList())){ + return false; } - return postTaskList; + for(String depNode : taskNode.getDepList()){ + if(!skipTaskNodeList.containsKey(depNode)){ + return false; + } + } + return true; } /** - * return start task node list - * @return task instance list + * set task node skip if dependence all skip + * @param taskNodesSkipList */ - private List getStartSubmitTaskList(){ + private void setTaskNodeSkip(List taskNodesSkipList){ + for(String skipNode : taskNodesSkipList){ + skipTaskNodeList.putIfAbsent(skipNode, dag.getNode(skipNode)); + Collection postNodeList = DagHelper.getStartVertex(skipNode, dag, completeTaskList); + List postSkipList = new ArrayList<>(); + for(String post : postNodeList){ + TaskNode postNode = dag.getNode(post); + if(isTaskNodeNeedSkip(postNode)){ + postSkipList.add(post); + } + } + setTaskNodeSkip(postSkipList); + } + } - List startTaskList = getPostTaskInstanceByNode(dag, null); - HashMap successTaskMaps = new HashMap<>(); - List resultList = new ArrayList<>(); - while(Stopper.isRunning()){ - for(TaskInstance task : startTaskList){ - if(task.getState().typeIsSuccess()){ - successTaskMaps.put(task.getName(), task); - }else if(!completeTaskList.containsKey(task.getName()) && !errorTaskList.containsKey(task.getName())){ - resultList.add(task); + /** + * parse condition task find the branch process + * set skip flag for another one. + * @param nodeName + * @return + */ + private List parseConditionTask(String nodeName){ + List conditionTaskList = new ArrayList<>(); + TaskNode taskNode = dag.getNode(nodeName); + if(!taskNode.isConditionsTask()){ + return conditionTaskList; + } + ConditionsParameters conditionsParameters = + JSONUtils.parseObject(taskNode.getConditionResult(), ConditionsParameters.class); + + TaskInstance taskInstance = completeTaskList.get(nodeName); + if(taskInstance == null){ + logger.error("task instance cannot find, please check it!", nodeName); + return conditionTaskList; + } + + if(taskInstance.getState().typeIsSuccess()){ + conditionTaskList = conditionsParameters.getSuccessNode(); + setTaskNodeSkip(conditionsParameters.getFailedNode()); + }else if(taskInstance.getState().typeIsFailure()){ + conditionTaskList = conditionsParameters.getFailedNode(); + setTaskNodeSkip(conditionsParameters.getSuccessNode()); + }else{ + conditionTaskList.add(nodeName); + } + return conditionTaskList; + } + + /** + * parse post node list of previous node + * if condition node: return process according to the settings + * if post node completed, return post nodes of the completed node + * @param previousNodeName + * @return + */ + private List parsePostNodeList(String previousNodeName){ + List postNodeList = new ArrayList<>(); + + TaskNode taskNode = dag.getNode(previousNodeName); + if(taskNode != null && taskNode.isConditionsTask()){ + return parseConditionTask(previousNodeName); + } + Collection postNodeCollection = DagHelper.getStartVertex(previousNodeName, dag, completeTaskList); + List postSkipList = new ArrayList<>(); + // delete success node, parse the past nodes + // if conditions node, + // 1. parse the branch process according the conditions setting + // 2. set skip flag on anther branch process + for(String postNode : postNodeCollection){ + if(completeTaskList.containsKey(postNode)){ + TaskInstance postTaskInstance = completeTaskList.get(postNode); + if(dag.getNode(postNode).isConditionsTask()){ + List conditionTaskNodeList = parseConditionTask(postNode); + for(String conditions : conditionTaskNodeList){ + postNodeList.addAll(parsePostNodeList(conditions)); + } + }else if(postTaskInstance.getState().typeIsSuccess()){ + postNodeList.addAll(parsePostNodeList(postNode)); + }else{ + postNodeList.add(postNode); } - } - startTaskList.clear(); - if(successTaskMaps.size() == 0){ - break; - } - Set taskNameKeys = successTaskMaps.keySet(); - for(String taskName : taskNameKeys){ - startTaskList.addAll(getPostTaskInstanceByNode(dag, taskName)); + }else if(isTaskNodeNeedSkip(dag.getNode(postNode))){ + postSkipList.add(postNode); + setTaskNodeSkip(postSkipList); + postSkipList.clear(); + }else{ + postNodeList.add(postNode); } - successTaskMaps.clear(); } - return resultList; + return postNodeList; } /** @@ -544,14 +632,17 @@ public class MasterExecThread implements Runnable { */ private void submitPostNode(String parentNodeName){ - List submitTaskList = null; - if(parentNodeName == null){ - submitTaskList = getStartSubmitTaskList(); - }else{ - submitTaskList = getPostTaskInstanceByNode(dag, parentNodeName); + List submitTaskNodeList = parsePostNodeList(parentNodeName); + + List taskInstances = new ArrayList<>(); + for(String taskNode : submitTaskNodeList){ + taskInstances.add(createTaskInstance(processInstance, taskNode, + dag.getNode(taskNode))); } + // if previous node success , post node submit - for(TaskInstance task : submitTaskList){ + for(TaskInstance task : taskInstances){ + if(readyToSubmitTaskList.containsKey(task.getName())){ continue; } @@ -575,27 +666,31 @@ public class MasterExecThread implements Runnable { private DependResult isTaskDepsComplete(String taskName) { Collection startNodes = dag.getBeginNode(); - // if the vertex returns true directly + // if vertex,returns true directly if(startNodes.contains(taskName)){ return DependResult.SUCCESS; } TaskNode taskNode = dag.getNode(taskName); - List depsNameList = taskNode.getDepList(); - for(String depsNode : depsNameList ){ + List depNameList = taskNode.getDepList(); + for(String depsNode : depNameList ){ - if(forbiddenTaskList.containsKey(depsNode)){ + if(forbiddenTaskList.containsKey(depsNode) || + skipTaskNodeList.containsKey(depsNode)){ continue; } // dependencies must be fully completed if(!completeTaskList.containsKey(depsNode)){ return DependResult.WAITING; } - ExecutionStatus taskState = completeTaskList.get(depsNode).getState(); - if(taskState.typeIsFailure()){ - return DependResult.FAILED; + ExecutionStatus depTaskState = completeTaskList.get(depsNode).getState(); + // conditions task would not return failed. + if(depTaskState.typeIsFailure()){ + if(!haveConditionsAfterNode(depsNode) && !dag.getNode(depsNode).isConditionsTask()){ + return DependResult.FAILED; + } } - if(taskState.typeIsPause() || taskState.typeIsCancel()){ + if(depTaskState.typeIsPause() || depTaskState.typeIsCancel()){ return DependResult.WAITING; } } @@ -878,11 +973,15 @@ public class MasterExecThread implements Runnable { if(task.taskCanRetry()){ addTaskToStandByList(task); }else{ - // node failure, based on failure strategy - errorTaskList.put(task.getName(), task); completeTaskList.put(task.getName(), task); - if(processInstance.getFailureStrategy() == FailureStrategy.END){ - killTheOtherTasks(); + if( task.getTaskType().equals(TaskType.CONDITIONS.toString()) || + haveConditionsAfterNode(task.getName())) { + submitPostNode(task.getName()); + }else{ + errorTaskList.put(task.getName(), task); + if(processInstance.getFailureStrategy() == FailureStrategy.END){ + killTheOtherTasks(); + } } } continue; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/TaskScheduleThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/TaskScheduleThread.java index a69cffd58d..5e68acf94e 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/TaskScheduleThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/TaskScheduleThread.java @@ -165,7 +165,6 @@ public class TaskScheduleThread implements Runnable { new Date(), taskInstance.getId()); } - /** * get global paras map * @return @@ -212,21 +211,29 @@ public class TaskScheduleThread implements Runnable { * @return log path */ private String getTaskLogPath() { - String baseLog = ((TaskLogDiscriminator) ((SiftingAppender) ((LoggerContext) LoggerFactory.getILoggerFactory()) - .getLogger("ROOT") - .getAppender("TASKLOGFILE")) - .getDiscriminator()).getLogBase(); - if (baseLog.startsWith(Constants.SINGLE_SLASH)){ - return baseLog + Constants.SINGLE_SLASH + - taskInstance.getProcessDefinitionId() + Constants.SINGLE_SLASH + - taskInstance.getProcessInstanceId() + Constants.SINGLE_SLASH + - taskInstance.getId() + ".log"; + String logPath; + try{ + String baseLog = ((TaskLogDiscriminator) ((SiftingAppender) ((LoggerContext) LoggerFactory.getILoggerFactory()) + .getLogger("ROOT") + .getAppender("TASKLOGFILE")) + .getDiscriminator()).getLogBase(); + if (baseLog.startsWith(Constants.SINGLE_SLASH)){ + logPath = baseLog + Constants.SINGLE_SLASH + + taskInstance.getProcessDefinitionId() + Constants.SINGLE_SLASH + + taskInstance.getProcessInstanceId() + Constants.SINGLE_SLASH + + taskInstance.getId() + ".log"; + }else{ + logPath = System.getProperty("user.dir") + Constants.SINGLE_SLASH + + baseLog + Constants.SINGLE_SLASH + + taskInstance.getProcessDefinitionId() + Constants.SINGLE_SLASH + + taskInstance.getProcessInstanceId() + Constants.SINGLE_SLASH + + taskInstance.getId() + ".log"; + } + }catch (Exception e){ + logger.error("logger" + e); + logPath = ""; } - return System.getProperty("user.dir") + Constants.SINGLE_SLASH + - baseLog + Constants.SINGLE_SLASH + - taskInstance.getProcessDefinitionId() + Constants.SINGLE_SLASH + - taskInstance.getProcessInstanceId() + Constants.SINGLE_SLASH + - taskInstance.getId() + ".log"; + return logPath; } /** diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractTask.java index 5c50df2265..3795506b78 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractTask.java @@ -22,6 +22,7 @@ import org.apache.dolphinscheduler.common.enums.TaskRecordStatus; import org.apache.dolphinscheduler.common.enums.TaskType; import org.apache.dolphinscheduler.common.process.Property; import org.apache.dolphinscheduler.common.task.AbstractParameters; +import org.apache.dolphinscheduler.common.task.conditions.ConditionsParameters; import org.apache.dolphinscheduler.common.task.datax.DataxParameters; import org.apache.dolphinscheduler.common.task.flink.FlinkParameters; import org.apache.dolphinscheduler.common.task.mr.MapreduceParameters; @@ -202,6 +203,9 @@ public abstract class AbstractTask { case SQOOP: paramsClass = SqoopParameters.class; break; + case CONDITIONS: + paramsClass = ConditionsParameters.class; + break; default: logger.error("not support this task type: {}", taskType); throw new IllegalArgumentException("not support this task type"); diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/TaskManager.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/TaskManager.java index 1ab6f2f3da..ad62b77655 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/TaskManager.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/TaskManager.java @@ -19,6 +19,7 @@ package org.apache.dolphinscheduler.server.worker.task; import org.apache.dolphinscheduler.common.enums.TaskType; import org.apache.dolphinscheduler.common.utils.EnumUtils; +import org.apache.dolphinscheduler.server.worker.task.conditions.ConditionsTask; import org.apache.dolphinscheduler.server.worker.task.dependent.DependentTask; import org.apache.dolphinscheduler.server.worker.task.datax.DataxTask; import org.apache.dolphinscheduler.server.worker.task.flink.FlinkTask; @@ -71,6 +72,8 @@ public class TaskManager { return new DataxTask(props, logger); case SQOOP: return new SqoopTask(props, logger); + case CONDITIONS: + return new ConditionsTask(props, logger); default: logger.error("unsupport task type: {}", taskType); throw new IllegalArgumentException("not support task type"); diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/conditions/ConditionsTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/conditions/ConditionsTask.java new file mode 100644 index 0000000000..cbe82ce20a --- /dev/null +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/conditions/ConditionsTask.java @@ -0,0 +1,145 @@ +/* + * 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.server.worker.task.conditions; + +import org.apache.dolphinscheduler.common.Constants; +import org.apache.dolphinscheduler.common.enums.DependResult; +import org.apache.dolphinscheduler.common.enums.ExecutionStatus; +import org.apache.dolphinscheduler.common.model.DependentItem; +import org.apache.dolphinscheduler.common.model.DependentTaskModel; +import org.apache.dolphinscheduler.common.task.AbstractParameters; +import org.apache.dolphinscheduler.common.task.dependent.DependentParameters; +import org.apache.dolphinscheduler.common.utils.DependentUtils; +import org.apache.dolphinscheduler.common.utils.JSONUtils; +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; +import org.apache.dolphinscheduler.dao.entity.TaskInstance; +import org.apache.dolphinscheduler.server.worker.task.AbstractTask; +import org.apache.dolphinscheduler.server.worker.task.TaskProps; +import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; +import org.apache.dolphinscheduler.service.process.ProcessService; +import org.slf4j.Logger; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +public class ConditionsTask extends AbstractTask { + + + /** + * dependent parameters + */ + private DependentParameters dependentParameters; + + /** + * process dao + */ + private ProcessService processService; + + /** + * taskInstance + */ + private TaskInstance taskInstance; + + /** + * processInstance + */ + private ProcessInstance processInstance; + + /** + * + */ + private Map completeTaskList = new ConcurrentHashMap<>(); + + /** + * constructor + * + * @param taskProps task props + * @param logger logger + */ + public ConditionsTask(TaskProps taskProps, Logger logger) { + super(taskProps, logger); + } + + @Override + public void init() throws Exception { + logger.info("conditions task initialize"); + + this.processService = SpringApplicationContext.getBean(ProcessService.class); + + this.dependentParameters = JSONUtils.parseObject(this.taskProps.getDependence(), DependentParameters.class); + + this.taskInstance = processService.findTaskInstanceById(taskProps.getTaskInstId()); + + if(taskInstance == null){ + throw new Exception("cannot find the task instance!"); + } + + List taskInstanceList = processService.findValidTaskListByProcessId(taskInstance.getProcessInstanceId()); + for(TaskInstance task : taskInstanceList){ + this.completeTaskList.putIfAbsent(task.getName(), task.getState()); + } + } + + @Override + public void handle() throws Exception { + + String threadLoggerInfoName = String.format(Constants.TASK_LOG_INFO_FORMAT, taskProps.getTaskAppId()); + Thread.currentThread().setName(threadLoggerInfoName); + + List modelResultList = new ArrayList<>(); + for(DependentTaskModel dependentTaskModel : dependentParameters.getDependTaskList()){ + + List itemDependResult = new ArrayList<>(); + for(DependentItem item : dependentTaskModel.getDependItemList()){ + itemDependResult.add(getDependResultForItem(item)); + } + DependResult modelResult = DependentUtils.getDependResultForRelation(dependentTaskModel.getRelation(), itemDependResult); + modelResultList.add(modelResult); + } + DependResult result = DependentUtils.getDependResultForRelation( + dependentParameters.getRelation(), modelResultList + ); + logger.info("the conditions task depend result : {}", result); + exitStatusCode = (result == DependResult.SUCCESS) ? + Constants.EXIT_CODE_SUCCESS : Constants.EXIT_CODE_FAILURE; + } + + private DependResult getDependResultForItem(DependentItem item){ + + DependResult dependResult = DependResult.SUCCESS; + if(!completeTaskList.containsKey(item.getDepTasks())){ + logger.info("depend item: {} have not completed yet.", item.getDepTasks()); + dependResult = DependResult.FAILED; + return dependResult; + } + ExecutionStatus executionStatus = completeTaskList.get(item.getDepTasks()); + if(executionStatus != item.getStatus()){ + logger.info("depend item : {} expect status: {}, actual status: {}" ,item.getDepTasks(), item.getStatus().toString(), executionStatus.toString()); + dependResult = DependResult.FAILED; + } + logger.info("depend item: {}, depend result: {}", + item.getDepTasks(), dependResult); + return dependResult; + } + + @Override + public AbstractParameters getParameters() { + return null; + } +} \ No newline at end of file diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterExecThreadTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterExecThreadTest.java index 770ab3cff6..d2a0fb2407 100644 --- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterExecThreadTest.java +++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterExecThreadTest.java @@ -151,4 +151,5 @@ public class MasterExecThreadTest { schedulerList.add(schedule); return schedulerList; } + } \ No newline at end of file diff --git a/dolphinscheduler-ui/.env b/dolphinscheduler-ui/.env index 4c7e96e795..e676be6059 100644 --- a/dolphinscheduler-ui/.env +++ b/dolphinscheduler-ui/.env @@ -17,4 +17,4 @@ API_BASE = http://192.168.xx.xx:12345 # If IP access is required for local development, remove the "#" -#DEV_HOST = 192.168.xx.xx \ No newline at end of file +#DEV_HOST = 192.168.xx.xx diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/config.js b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/config.js old mode 100644 new mode 100755 index be4daa6595..a9a51aa2b1 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/config.js +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/config.js @@ -287,6 +287,10 @@ let tasksType = { 'SQOOP': { desc: 'SQOOP', color: '#E46F13' + }, + 'CONDITIONS': { + desc: 'CONDITIONS', + color: '#E46F13' } } diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.scss b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.scss old mode 100644 new mode 100755 index 08918c969f..6d97856960 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.scss +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.scss @@ -107,6 +107,9 @@ .icos-SQOOP { background: url("../img/toolbar_SQOOP.png") no-repeat 50% 50%; } + .icos-CONDITIONS { + background: url("../img/toobar_CONDITIONS.png") no-repeat 50% 50%; + } .toolbar { width: 60px; height: 100%; diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.vue old mode 100644 new mode 100755 index a21889f4ac..d912a9a884 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.vue @@ -473,7 +473,35 @@ */ _createNodes ({ id, type }) { let self = this + let preNode = [] + let rearNode = [] + let rearList = [] + $('div[data-targetarr*="' + id + '"]').each(function(){ + rearNode.push($(this).attr("id")) + }) + if (rearNode.length>0) { + rearNode.forEach(v => { + let rearobj = {} + rearobj.value = $(`#${v}`).find('.name-p').text() + rearobj.label = $(`#${v}`).find('.name-p').text() + rearList.push(rearobj) + }) + } else { + rearList = [] + } + let targetarr = $(`#${id}`).attr('data-targetarr') + if (targetarr) { + let nodearr = targetarr.split(',') + nodearr.forEach(v => { + let nodeobj = {} + nodeobj.value = $(`#${v}`).find('.name-p').text() + nodeobj.label = $(`#${v}`).find('.name-p').text() + preNode.push(nodeobj) + }) + } else { + preNode = [] + } if (eventModel) { eventModel.remove() } @@ -524,7 +552,9 @@ props: { id: id, taskType: type, - self: self + self: self, + preNode: preNode, + rearList: rearList } }) }) diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue old mode 100644 new mode 100755 index 177b252693..2500ce5772 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue @@ -109,6 +109,43 @@ ({{$t('Minute')}})
    +
    +
    + {{$t('State')}} +
    +
    + + + + + + + {{$t('Branch flow')}} + + + + +
    +
    + +
    +
    + {{$t('State')}} +
    +
    + + + + + + + {{$t('Branch flow')}} + + + + +
    +
    + +
    @@ -236,6 +280,7 @@ import mDependent from './tasks/dependent' import mHttp from './tasks/http' import mDatax from './tasks/datax' + import mConditions from './tasks/CONDITIONS' import mSqoop from './tasks/sqoop' import mSubProcess from './tasks/sub_process' import mSelectInput from './_source/selectInput' @@ -253,13 +298,21 @@ // loading spinnerLoading: false, // node name - name: ``, + name: '', // description description: '', // Node echo data backfillItem: {}, // Resource(list) resourcesList: [], + successNode: 'success', + failedNode: 'failed', + successBranch: '', + failedBranch: '', + conditionResult: { + 'successNode': [], + 'failedNode': [] + }, // dependence dependence: {}, // cache dependence @@ -279,7 +332,17 @@ // Task priority taskInstancePriority: 'MEDIUM', // worker group id - workerGroupId: -1 + workerGroupId: -1, + stateList:[ + { + value: 'success', + label: `${i18n.$t('success')}` + }, + { + value: 'failed', + label: `${i18n.$t('failed')}` + } + ] } }, /** @@ -290,7 +353,9 @@ props: { id: Number, taskType: String, - self: Object + self: Object, + preNode: Array, + rearList: Array }, methods: { /** @@ -399,6 +464,10 @@ this.$message.warning(`${i18n.$t('Please enter name (required)')}`) return false } + if (this.successBranch !='' && this.successBranch == this.failedBranch) { + this.$message.warning(`${i18n.$t('Cannot select the same node for successful branch flow and failed branch flow')}`) + return false + } if (this.name === this.backfillItem.name) { return true } @@ -427,6 +496,8 @@ } $(`#${this.id}`).find('span').text(this.name) + this.conditionResult.successNode[0] = this.successBranch + this.conditionResult.failedNode[0] = this.failedBranch // Store the corresponding node data structure this.$emit('addTaskInfo', { item: { @@ -436,12 +507,15 @@ params: this.params, description: this.description, runFlag: this.runFlag, + conditionResult: this.conditionResult, dependence: this.dependence, maxRetryTimes: this.maxRetryTimes, retryInterval: this.retryInterval, timeout: this.timeout, taskInstancePriority: this.taskInstancePriority, - workerGroupId: this.workerGroupId + workerGroupId: this.workerGroupId, + status: this.status, + branch: this.branch }, fromThis: this }) @@ -526,7 +600,10 @@ this.description = o.description this.maxRetryTimes = o.maxRetryTimes this.retryInterval = o.retryInterval - + if(o.conditionResult) { + this.successBranch = o.conditionResult.successNode[0] + this.failedBranch = o.conditionResult.failedNode[0] + } // If the workergroup has been deleted, set the default workergroup var hasMatch = false; for (let i = 0; i < this.store.state.security.workerGroupsListAll.length; i++) { @@ -598,6 +675,7 @@ mHttp, mDatax, mSqoop, + mConditions, mSelectInput, mTimeoutAlarm, mPriority, diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/commcon.js b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/commcon.js old mode 100644 new mode 100755 index fc8fe654d2..cdf632f13d --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/commcon.js +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/commcon.js @@ -232,6 +232,16 @@ const positionList = [ code: "Headers" } ] +const nodeStatusList = [ + { + value: 'SUCCESS', + label: `${i18n.$t('success')}` + }, + { + value: 'FAILURE', + label: `${i18n.$t('failed')}` + } +] export { cycleList, @@ -239,5 +249,6 @@ export { typeList, directList, sqlTypeList, - positionList + positionList, + nodeStatusList } diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/nodeStatus.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/nodeStatus.vue new file mode 100644 index 0000000000..4afb8b46c5 --- /dev/null +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/nodeStatus.vue @@ -0,0 +1,231 @@ +/* + * 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. + */ + + + + \ No newline at end of file diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/conditions.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/conditions.vue new file mode 100644 index 0000000000..4ac04d91a6 --- /dev/null +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/conditions.vue @@ -0,0 +1,265 @@ +/* + * 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. + */ + + + + \ No newline at end of file diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/plugIn/jsPlumbHandle.js b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/plugIn/jsPlumbHandle.js old mode 100644 new mode 100755 index b0a7a64a47..598c94209e --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/plugIn/jsPlumbHandle.js +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/plugIn/jsPlumbHandle.js @@ -198,7 +198,8 @@ JSP.prototype.jsonHandle = function ({ largeJson, locations }) { targetarr: locations[v.id]['targetarr'], isAttachment: this.config.isAttachment, taskType: v.type, - runFlag: v.runFlag + runFlag: v.runFlag, + nodenumber: locations[v.id]['nodenumber'], })) // contextmenu event @@ -517,6 +518,9 @@ JSP.prototype.removeConnect = function ($connect) { targetarr = _.filter(targetarr, v => v !== sourceId) $(`#${targetId}`).attr('data-targetarr', targetarr.toString()) } + if ($(`#${sourceId}`).attr('data-tasks-type')=='CONDITIONS') { + $(`#${sourceId}`).attr('data-nodenumber',Number($(`#${sourceId}`).attr('data-nodenumber'))-1) + } this.JspInstance.deleteConnection($connect) this.selectedElement = {} @@ -572,6 +576,7 @@ JSP.prototype.copyNodes = function ($id) { [newId]: { name: newName, targetarr: '', + nodenumber: 0, x: newX, y: newY } @@ -658,6 +663,7 @@ JSP.prototype.saveStore = function () { locations[v.id] = { name: v.name, targetarr: v.targetarr, + nodenumber: v.nodenumber, x: v.x, y: v.y } @@ -711,6 +717,12 @@ JSP.prototype.handleEvent = function () { return false } + if ($(`#${sourceId}`).attr('data-tasks-type')=='CONDITIONS' && $(`#${sourceId}`).attr('data-nodenumber')==2) { + return false + } else { + $(`#${sourceId}`).attr('data-nodenumber',Number($(`#${sourceId}`).attr('data-nodenumber'))+1) + } + // Storage node dependency information saveTargetarr(sourceId, targetId) diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/plugIn/util.js b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/plugIn/util.js old mode 100644 new mode 100755 index c10dfda5d6..4b485fec0b --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/plugIn/util.js +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/plugIn/util.js @@ -43,9 +43,9 @@ const rtBantpl = () => { /** * return node html */ -const rtTasksTpl = ({ id, name, x, y, targetarr, isAttachment, taskType, runFlag }) => { +const rtTasksTpl = ({ id, name, x, y, targetarr, isAttachment, taskType, runFlag, nodenumber }) => { let tpl = `` - tpl += `
    ` + tpl += `
    ` tpl += `
    ` tpl += `
    ` tpl += `
    ` @@ -73,6 +73,7 @@ const tasksAll = () => { id: e.attr('id'), name: e.find('.name-p').text(), targetarr: e.attr('data-targetarr') || '', + nodenumber: e.attr('data-nodenumber'), x: parseInt(e.css('left'), 10), y: parseInt(e.css('top'), 10) }) diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/img/toobar_CONDITIONS.png b/dolphinscheduler-ui/src/js/conf/home/pages/dag/img/toobar_CONDITIONS.png new file mode 100644 index 0000000000000000000000000000000000000000..e8c5e383398b3829420fc17d8225d4fa255ea01b GIT binary patch literal 1367 zcmV-d1*rOoP)^v3$xSzya0p==m0_Q|p_E3IN$r<1&e9#o#Xm_2}-& z1fHmy9Zl~d`EucZ{u@L&}DB8H!j@b8+>p4Y1r}xfcZt890c3gk$TI| z#u_zNfY|5w2ey|A(_uih8a4Zjs&)ZB3hU8TUXDb%HS`bf0Jz>ekQ~Jgc5zkeXD@ly zG1Y}jGHYhm5%L~EE2u}pwdgI$CE2tF z;M)MEg3^mA*EezHkz76YmT4%xsSvEEHEJY5BtRYs&5k@(kAra#eesV0d6pfiiUCv2 zk99utB}ATtpr1L@I3;jyxcwnLx^hwgl5xere6|lhF&R6{G*mmTAaI(NXAi&2eWP;fcgrMPUg)A>8*g51mODr?91CUn$T&WL0vRPou)`}n+mjbv8 zw(R$ib^Enr8Gw3HAO!$R*g16E9Y}mqAbXk?L*p@j0PYm{6rm*JVNeF0fHoq4n9%qa zU~gvP&^Jy?P5>Z%@iu6(8n)aHP=^L*=m6RpCI)^ifiEjUzcwtL!l9m=0+Aa|9k_B^ z03aJnCzVpIRK-iOsT<^C1#-MrCNPjQGkesw zG(L)zV$L*r1X$}B8+SGMwv z zK8chKei6E5`X_8?_`6!I)CAx*R`Vhcwvg&Ef@vh~C=@IH1aXDn2m%-g&W!v~Pk}0( Ze*xmDmN}hJv@iew002ovPDHLkV1gLacJBZH literal 0 HcmV?d00001 diff --git a/dolphinscheduler-ui/src/js/conf/home/store/dag/mutations.js b/dolphinscheduler-ui/src/js/conf/home/store/dag/mutations.js old mode 100644 new mode 100755 index 6ceabed8c1..b914b86740 --- a/dolphinscheduler-ui/src/js/conf/home/store/dag/mutations.js +++ b/dolphinscheduler-ui/src/js/conf/home/store/dag/mutations.js @@ -134,6 +134,7 @@ export default { state.locations[payload.id] = _.assign(state.locations[payload.id], { name: dom.find('.name-p').text(), targetarr: dom.attr('data-targetarr'), + nodenumber: dom.attr('data-nodenumber'), x: parseInt(dom.css('left'), 10), y: parseInt(dom.css('top'), 10) }) diff --git a/dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js b/dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js old mode 100644 new mode 100755 index a96ee8e145..e8ac57adc0 --- a/dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js +++ b/dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js @@ -520,7 +520,6 @@ export default { '0 means unlimited by byte': '0 means unlimited', '0 means unlimited by count': '0 means unlimited', 'Modify User': 'Modify User', - 'Please enter Mysql Database(required)': 'Please enter Mysql Database(required)', 'Please enter Mysql Table(required)': 'Please enter Mysql Table(required)', 'Please enter Columns (Comma separated)': 'Please enter Columns (Comma separated)', @@ -566,7 +565,8 @@ export default { 'Data Source': 'Data Source', 'Data Target': 'Data Target', 'All Columns': 'All Columns', - 'Some Columns': 'Some Columns' - - + 'Some Columns': 'Some Columns', + 'Modify User': 'Modify User', + 'Branch flow': 'Branch flow', + 'Cannot select the same node for successful branch flow and failed branch flow': 'Cannot select the same node for successful branch flow and failed branch flow' } diff --git a/dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js b/dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js old mode 100644 new mode 100755 index 72c978d2e1..c72090657b --- a/dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js +++ b/dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js @@ -565,5 +565,8 @@ export default { 'Data Source': '数据来源', 'Data Target': '数据目的', 'All Columns': '全表导入', - 'Some Columns': '选择列' + 'Some Columns': '选择列', + 'Modify User': '修改用户', + 'Branch flow': '分支流转', + 'Cannot select the same node for successful branch flow and failed branch flow': '成功分支流转和失败分支流转不能选择同一个节点' } From 415d087dfc00cb664703af157d761d7d821302be Mon Sep 17 00:00:00 2001 From: DS Date: Sun, 1 Mar 2020 20:42:45 +0800 Subject: [PATCH 044/221] Update README.md (#2046) --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 75f0877528..0323d1093d 100644 --- a/README.md +++ b/README.md @@ -100,7 +100,7 @@ It is because of the shoulders of these open source projects that the birth of t ### Get Help 1. Submit an issue 1. Subscribe the mail list : https://dolphinscheduler.apache.org/en-us/docs/development/subscribe.html. then send mail to dev@dolphinscheduler.apache.org -1. Contact WeChat group manager, ID 510570367. This is for Mandarin(CN) discussion. +1. Contact WeChat(dailidong66). This is just for Mandarin(CN) discussion. ### License Please refer to [LICENSE](https://github.com/apache/incubator-dolphinscheduler/blob/dev/LICENSE) file. From 8b2a224716f124552e16b5b9b8c3305723448554 Mon Sep 17 00:00:00 2001 From: DS Date: Sun, 1 Mar 2020 20:43:09 +0800 Subject: [PATCH 045/221] image about monitor screenshot is wrong (#2047) image about monitor screenshot is wrong --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 0323d1093d..726432c1d3 100644 --- a/README.md +++ b/README.md @@ -53,7 +53,7 @@ Overload processing: Task queue mechanism, the number of schedulable tasks on a ![view task log online](https://user-images.githubusercontent.com/15833811/75216924-9900c500-57d0-11ea-91dc-3522a76bdbbe.png) ![resource management](https://user-images.githubusercontent.com/15833811/75216984-be8dce80-57d0-11ea-840d-58546edc8788.png) ![datasource management](https://user-images.githubusercontent.com/15833811/75217041-cfd6db00-57d0-11ea-8451-8b376d0dc8f1.png) -![monitor](https://user-images.githubusercontent.com/15833811/75217091-fac12f00-57d0-11ea-9665-d8dc19813707.png) +![monitor](https://user-images.githubusercontent.com/59273635/75625839-c698a480-5bfc-11ea-8bbe-895b561b337f.png) ![security](https://user-images.githubusercontent.com/15833811/75236441-bfd2f180-57f8-11ea-88bd-f24311e01b7e.png) ![treeview](https://user-images.githubusercontent.com/15833811/75217191-3fe56100-57d1-11ea-8856-f19180d9a879.png) ### Document From c8adf824e238a0f73672cb4ff6f8609343ab1774 Mon Sep 17 00:00:00 2001 From: DS Date: Sun, 1 Mar 2020 20:48:38 +0800 Subject: [PATCH 046/221] Update README.md (#2048) --- README.md | 1 - 1 file changed, 1 deletion(-) diff --git a/README.md b/README.md index 726432c1d3..0a9a164b85 100644 --- a/README.md +++ b/README.md @@ -52,7 +52,6 @@ Overload processing: Task queue mechanism, the number of schedulable tasks on a ![process definition list page](https://user-images.githubusercontent.com/15833811/75216886-6f479e00-57d0-11ea-92dd-66e7640a186f.png) ![view task log online](https://user-images.githubusercontent.com/15833811/75216924-9900c500-57d0-11ea-91dc-3522a76bdbbe.png) ![resource management](https://user-images.githubusercontent.com/15833811/75216984-be8dce80-57d0-11ea-840d-58546edc8788.png) -![datasource management](https://user-images.githubusercontent.com/15833811/75217041-cfd6db00-57d0-11ea-8451-8b376d0dc8f1.png) ![monitor](https://user-images.githubusercontent.com/59273635/75625839-c698a480-5bfc-11ea-8bbe-895b561b337f.png) ![security](https://user-images.githubusercontent.com/15833811/75236441-bfd2f180-57f8-11ea-88bd-f24311e01b7e.png) ![treeview](https://user-images.githubusercontent.com/15833811/75217191-3fe56100-57d1-11ea-8856-f19180d9a879.png) From 1ea48375f49e1b86b437faffbb19c1d6b6dec032 Mon Sep 17 00:00:00 2001 From: Rubik <39549317+wenhemin@users.noreply.github.com> Date: Mon, 2 Mar 2020 10:29:42 +0800 Subject: [PATCH 047/221] Fixbug i18n #2008 (#2035) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * fix:add chinese message * fix:add chinese message * fix:'getLocale()' can return not null * fix:optimize message content --- .../api/controller/DataSourceController.java | 4 +- .../dolphinscheduler/api/enums/Status.java | 422 +++++++++--------- .../api/enums/StatusTest.java | 12 +- 3 files changed, 228 insertions(+), 210 deletions(-) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/DataSourceController.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/DataSourceController.java index 9c04bf7d5c..881c93f2f7 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/DataSourceController.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/DataSourceController.java @@ -391,8 +391,8 @@ public class DataSourceController extends BaseController { try { return dataSourceService.verifyDataSourceName(loginUser, name); } catch (Exception e) { - logger.error(VERFIY_DATASOURCE_NAME_FAILURE.getMsg(),e); - return error(VERFIY_DATASOURCE_NAME_FAILURE.getCode(), VERFIY_DATASOURCE_NAME_FAILURE.getMsg()); + logger.error(VERIFY_DATASOURCE_NAME_FAILURE.getMsg(), e); + return error(VERIFY_DATASOURCE_NAME_FAILURE.getCode(), VERIFY_DATASOURCE_NAME_FAILURE.getMsg()); } } diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java index 7a87d552de..9955463f8e 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/enums/Status.java @@ -16,244 +16,248 @@ */ package org.apache.dolphinscheduler.api.enums; +import org.springframework.context.i18n.LocaleContextHolder; + +import java.util.Locale; + /** * status enum */ public enum Status { - SUCCESS(0, "success"), + SUCCESS(0, "success", "成功"), - REQUEST_PARAMS_NOT_VALID_ERROR(10001, "request parameter {0} is not valid"), - TASK_TIMEOUT_PARAMS_ERROR(10002, "task timeout parameter is not valid"), - USER_NAME_EXIST(10003, "user name already exists"), - USER_NAME_NULL(10004,"user name is null"), - HDFS_OPERATION_ERROR(10006, "hdfs operation error"), - TASK_INSTANCE_NOT_FOUND(10008, "task instance not found"), - TENANT_NAME_EXIST(10009, "tenant code already exists"), - USER_NOT_EXIST(10010, "user {0} not exists"), - ALERT_GROUP_NOT_EXIST(10011, "alarm group not found"), - ALERT_GROUP_EXIST(10012, "alarm group already exists"), - USER_NAME_PASSWD_ERROR(10013,"user name or password error"), - LOGIN_SESSION_FAILED(10014,"create session failed!"), - DATASOURCE_EXIST(10015, "data source name already exists"), - DATASOURCE_CONNECT_FAILED(10016, "data source connection failed"), - TENANT_NOT_EXIST(10017, "tenant not exists"), - PROJECT_NOT_FOUNT(10018, "project {0} not found "), - PROJECT_ALREADY_EXISTS(10019, "project {0} already exists"), - TASK_INSTANCE_NOT_EXISTS(10020, "task instance {0} does not exist"), - TASK_INSTANCE_NOT_SUB_WORKFLOW_INSTANCE(10021, "task instance {0} is not sub process instance"), - SCHEDULE_CRON_NOT_EXISTS(10022, "scheduler crontab {0} does not exist"), - SCHEDULE_CRON_ONLINE_FORBID_UPDATE(10023, "online status does not allow updateProcessInstance operations"), - SCHEDULE_CRON_CHECK_FAILED(10024, "scheduler crontab expression validation failure: {0}"), - MASTER_NOT_EXISTS(10025, "master does not exist"), - SCHEDULE_STATUS_UNKNOWN(10026, "unknown command: {0}"), - CREATE_ALERT_GROUP_ERROR(10027,"create alert group error"), - QUERY_ALL_ALERTGROUP_ERROR(10028,"query all alertgroup error"), - LIST_PAGING_ALERT_GROUP_ERROR(10029,"list paging alert group error"), - UPDATE_ALERT_GROUP_ERROR(10030,"updateProcessInstance alert group error"), - DELETE_ALERT_GROUP_ERROR(10031,"delete alert group error"), - ALERT_GROUP_GRANT_USER_ERROR(10032,"alert group grant user error"), - CREATE_DATASOURCE_ERROR(10033,"create datasource error"), - UPDATE_DATASOURCE_ERROR(10034,"updateProcessInstance datasource error"), - QUERY_DATASOURCE_ERROR(10035,"query datasource error"), - CONNECT_DATASOURCE_FAILURE(10036,"connect datasource failure"), - CONNECTION_TEST_FAILURE(10037,"connection test failure"), - DELETE_DATA_SOURCE_FAILURE(10038,"delete data source failure"), - VERFIY_DATASOURCE_NAME_FAILURE(10039,"verfiy datasource name failure"), - UNAUTHORIZED_DATASOURCE(10040,"unauthorized datasource"), - AUTHORIZED_DATA_SOURCE(10041,"authorized data source"), - LOGIN_SUCCESS(10042,"login success"), - USER_LOGIN_FAILURE(10043,"user login failure"), - LIST_WORKERS_ERROR(10044,"list workers error"), - LIST_MASTERS_ERROR(10045,"list masters error"), - UPDATE_PROJECT_ERROR(10046,"updateProcessInstance project error"), - QUERY_PROJECT_DETAILS_BY_ID_ERROR(10047,"query project details by id error"), - CREATE_PROJECT_ERROR(10048,"create project error"), - LOGIN_USER_QUERY_PROJECT_LIST_PAGING_ERROR(10049,"login user query project list paging error"), - DELETE_PROJECT_ERROR(10050,"delete project error"), - QUERY_UNAUTHORIZED_PROJECT_ERROR(10051,"query unauthorized project error"), - QUERY_AUTHORIZED_PROJECT(10052,"query authorized project"), - QUERY_QUEUE_LIST_ERROR(10053,"query queue list error"), - CREATE_RESOURCE_ERROR(10054,"create resource error"), - UPDATE_RESOURCE_ERROR(10055,"updateProcessInstance resource error"), - QUERY_RESOURCES_LIST_ERROR(10056,"query resources list error"), - QUERY_RESOURCES_LIST_PAGING(10057,"query resources list paging"), - DELETE_RESOURCE_ERROR(10058,"delete resource error"), - VERIFY_RESOURCE_BY_NAME_AND_TYPE_ERROR(10059,"verify resource by name and type error"), - VIEW_RESOURCE_FILE_ON_LINE_ERROR(10060,"view resource file online error"), - CREATE_RESOURCE_FILE_ON_LINE_ERROR(10061,"create resource file online error"), - RESOURCE_FILE_IS_EMPTY(10062,"resource file is empty"), - EDIT_RESOURCE_FILE_ON_LINE_ERROR(10063,"edit resource file online error"), - DOWNLOAD_RESOURCE_FILE_ERROR(10064,"download resource file error"), - CREATE_UDF_FUNCTION_ERROR(10065 ,"create udf function error"), - VIEW_UDF_FUNCTION_ERROR( 10066,"view udf function error"), - UPDATE_UDF_FUNCTION_ERROR(10067,"updateProcessInstance udf function error"), - QUERY_UDF_FUNCTION_LIST_PAGING_ERROR( 10068,"query udf function list paging error"), - QUERY_DATASOURCE_BY_TYPE_ERROR( 10069,"query datasource by type error"), - VERIFY_UDF_FUNCTION_NAME_ERROR( 10070,"verify udf function name error"), - DELETE_UDF_FUNCTION_ERROR( 10071,"delete udf function error"), - AUTHORIZED_FILE_RESOURCE_ERROR( 10072,"authorized file resource error"), - UNAUTHORIZED_FILE_RESOURCE_ERROR( 10073,"unauthorized file resource error"), - UNAUTHORIZED_UDF_FUNCTION_ERROR( 10074,"unauthorized udf function error"), - AUTHORIZED_UDF_FUNCTION_ERROR(10075,"authorized udf function error"), - CREATE_SCHEDULE_ERROR(10076,"create schedule error"), - UPDATE_SCHEDULE_ERROR(10077,"updateProcessInstance schedule error"), - PUBLISH_SCHEDULE_ONLINE_ERROR(10078,"publish schedule online error"), - OFFLINE_SCHEDULE_ERROR(10079,"offline schedule error"), - QUERY_SCHEDULE_LIST_PAGING_ERROR(10080,"query schedule list paging error"), - QUERY_SCHEDULE_LIST_ERROR(10081,"query schedule list error"), - QUERY_TASK_LIST_PAGING_ERROR(10082,"query task list paging error"), - QUERY_TASK_RECORD_LIST_PAGING_ERROR(10083,"query task record list paging error"), - CREATE_TENANT_ERROR(10084,"create tenant error"), - QUERY_TENANT_LIST_PAGING_ERROR(10085,"query tenant list paging error"), - QUERY_TENANT_LIST_ERROR(10086,"query tenant list error"), - UPDATE_TENANT_ERROR(10087,"updateProcessInstance tenant error"), - DELETE_TENANT_BY_ID_ERROR(10088,"delete tenant by id error"), - VERIFY_TENANT_CODE_ERROR(10089,"verify tenant code error"), - CREATE_USER_ERROR(10090,"create user error"), - QUERY_USER_LIST_PAGING_ERROR(10091,"query user list paging error"), - UPDATE_USER_ERROR(10092,"updateProcessInstance user error"), - DELETE_USER_BY_ID_ERROR(10093,"delete user by id error"), - GRANT_PROJECT_ERROR(10094,"grant project error"), - GRANT_RESOURCE_ERROR(10095,"grant resource error"), - GRANT_UDF_FUNCTION_ERROR(10096,"grant udf function error"), - GRANT_DATASOURCE_ERROR(10097,"grant datasource error"), - GET_USER_INFO_ERROR(10098,"get user info error"), - USER_LIST_ERROR(10099,"user list error"), - VERIFY_USERNAME_ERROR(10100,"verify username error"), - UNAUTHORIZED_USER_ERROR(10101,"unauthorized user error"), - AUTHORIZED_USER_ERROR(10102,"authorized user error"), - QUERY_TASK_INSTANCE_LOG_ERROR(10103,"view task instance log error"), - DOWNLOAD_TASK_INSTANCE_LOG_FILE_ERROR(10104,"download task instance log file error"), - CREATE_PROCESS_DEFINITION(10105,"create process definition"), - VERIFY_PROCESS_DEFINITION_NAME_UNIQUE_ERROR(10106,"verify process definition name unique error"), - UPDATE_PROCESS_DEFINITION_ERROR(10107,"updateProcessInstance process definition error"), - RELEASE_PROCESS_DEFINITION_ERROR(10108,"release process definition error"), - QUERY_DATAIL_OF_PROCESS_DEFINITION_ERROR(10109,"query datail of process definition error"), - QUERY_PROCCESS_DEFINITION_LIST(10110,"query proccess definition list"), - ENCAPSULATION_TREEVIEW_STRUCTURE_ERROR(10111,"encapsulation treeview structure error"), - GET_TASKS_LIST_BY_PROCESS_DEFINITION_ID_ERROR(10112,"get tasks list by process definition id error"), - QUERY_PROCESS_INSTANCE_LIST_PAGING_ERROR(10113,"query process instance list paging error"), - QUERY_TASK_LIST_BY_PROCESS_INSTANCE_ID_ERROR(10114,"query task list by process instance id error"), - UPDATE_PROCESS_INSTANCE_ERROR(10115,"updateProcessInstance process instance error"), - QUERY_PROCESS_INSTANCE_BY_ID_ERROR(10116,"query process instance by id error"), - DELETE_PROCESS_INSTANCE_BY_ID_ERROR(10117,"delete process instance by id error"), - QUERY_SUB_PROCESS_INSTANCE_DETAIL_INFO_BY_TASK_ID_ERROR(10118,"query sub process instance detail info by task id error"), - QUERY_PARENT_PROCESS_INSTANCE_DETAIL_INFO_BY_SUB_PROCESS_INSTANCE_ID_ERROR(10119,"query parent process instance detail info by sub process instance id error"), - QUERY_PROCESS_INSTANCE_ALL_VARIABLES_ERROR(10120,"query process instance all variables error"), - ENCAPSULATION_PROCESS_INSTANCE_GANTT_STRUCTURE_ERROR(10121,"encapsulation process instance gantt structure error"), - QUERY_PROCCESS_DEFINITION_LIST_PAGING_ERROR(10122,"query proccess definition list paging error"), - SIGN_OUT_ERROR(10123,"sign out error"), - TENANT_CODE_HAS_ALREADY_EXISTS(10124,"tenant code has already exists"), - IP_IS_EMPTY(10125,"ip is empty"), - SCHEDULE_CRON_REALEASE_NEED_NOT_CHANGE(10126, "schedule release is already {0}"), - CREATE_QUEUE_ERROR(10127, "create queue error"), - QUEUE_NOT_EXIST(10128, "queue {0} not exists"), - QUEUE_VALUE_EXIST(10129, "queue value {0} already exists"), - QUEUE_NAME_EXIST(10130, "queue name {0} already exists"), - UPDATE_QUEUE_ERROR(10131, "update queue error"), - NEED_NOT_UPDATE_QUEUE(10132, "no content changes, no updates are required"), - VERIFY_QUEUE_ERROR(10133,"verify queue error"), - NAME_NULL(10134,"name must be not null"), - NAME_EXIST(10135, "name {0} already exists"), - SAVE_ERROR(10136, "save error"), - DELETE_PROJECT_ERROR_DEFINES_NOT_NULL(10137, "please delete the process definitions in project first!"), - BATCH_DELETE_PROCESS_INSTANCE_BY_IDS_ERROR(10117,"batch delete process instance by ids {0} error"), - PREVIEW_SCHEDULE_ERROR(10139,"preview schedule error"), - PARSE_TO_CRON_EXPRESSION_ERROR(10140,"parse cron to cron expression error"), - SCHEDULE_START_TIME_END_TIME_SAME(10141,"The start time must not be the same as the end"), - DELETE_TENANT_BY_ID_FAIL(100142,"delete tenant by id fail, for there are {0} process instances in executing using it"), - DELETE_TENANT_BY_ID_FAIL_DEFINES(100143,"delete tenant by id fail, for there are {0} process definitions using it"), - DELETE_TENANT_BY_ID_FAIL_USERS(100144,"delete tenant by id fail, for there are {0} users using it"), + REQUEST_PARAMS_NOT_VALID_ERROR(10001, "request parameter {0} is not valid", "请求参数[{0}]无效"), + TASK_TIMEOUT_PARAMS_ERROR(10002, "task timeout parameter is not valid", "任务超时参数无效"), + USER_NAME_EXIST(10003, "user name already exists", "用户名已存在"), + USER_NAME_NULL(10004,"user name is null", "用户名不能为空"), + HDFS_OPERATION_ERROR(10006, "hdfs operation error", "hdfs操作错误"), + TASK_INSTANCE_NOT_FOUND(10008, "task instance not found", "任务实例不存在"), + TENANT_NAME_EXIST(10009, "tenant code already exists", "租户编码不能为空"), + USER_NOT_EXIST(10010, "user {0} not exists", "用户[{0}]不存在"), + ALERT_GROUP_NOT_EXIST(10011, "alarm group not found", "告警组不存在"), + ALERT_GROUP_EXIST(10012, "alarm group already exists", "告警组名称已存在"), + USER_NAME_PASSWD_ERROR(10013,"user name or password error", "用户名或密码错误"), + LOGIN_SESSION_FAILED(10014,"create session failed!", "创建session失败"), + DATASOURCE_EXIST(10015, "data source name already exists", "数据源名称已存在"), + DATASOURCE_CONNECT_FAILED(10016, "data source connection failed", "建立数据源连接失败"), + TENANT_NOT_EXIST(10017, "tenant not exists", "租户不存在"), + PROJECT_NOT_FOUNT(10018, "project {0} not found ", "项目[{0}]不存在"), + PROJECT_ALREADY_EXISTS(10019, "project {0} already exists", "项目名称[{0}]已存在"), + TASK_INSTANCE_NOT_EXISTS(10020, "task instance {0} does not exist", "任务实例[{0}]不存在"), + TASK_INSTANCE_NOT_SUB_WORKFLOW_INSTANCE(10021, "task instance {0} is not sub process instance", "任务实例[{0}]不是子流程实例"), + SCHEDULE_CRON_NOT_EXISTS(10022, "scheduler crontab {0} does not exist", "调度配置定时表达式[{0}]不存在"), + SCHEDULE_CRON_ONLINE_FORBID_UPDATE(10023, "online status does not allow update operations", "调度配置上线状态不允许修改"), + SCHEDULE_CRON_CHECK_FAILED(10024, "scheduler crontab expression validation failure: {0}", "调度配置定时表达式验证失败: {0}"), + MASTER_NOT_EXISTS(10025, "master does not exist", "无可用master节点"), + SCHEDULE_STATUS_UNKNOWN(10026, "unknown status: {0}", "未知状态: {0}"), + CREATE_ALERT_GROUP_ERROR(10027,"create alert group error", "创建告警组错误"), + QUERY_ALL_ALERTGROUP_ERROR(10028,"query all alertgroup error", "查询告警组错误"), + LIST_PAGING_ALERT_GROUP_ERROR(10029,"list paging alert group error", "分页查询告警组错误"), + UPDATE_ALERT_GROUP_ERROR(10030,"update alert group error", "更新告警组错误"), + DELETE_ALERT_GROUP_ERROR(10031,"delete alert group error", "删除告警组错误"), + ALERT_GROUP_GRANT_USER_ERROR(10032,"alert group grant user error", "告警组授权用户错误"), + CREATE_DATASOURCE_ERROR(10033,"create datasource error", "创建数据源错误"), + UPDATE_DATASOURCE_ERROR(10034,"update datasource error", "更新数据源错误"), + QUERY_DATASOURCE_ERROR(10035,"query datasource error", "查询数据源错误"), + CONNECT_DATASOURCE_FAILURE(10036,"connect datasource failure", "建立数据源连接失败"), + CONNECTION_TEST_FAILURE(10037,"connection test failure", "测试数据源连接失败"), + DELETE_DATA_SOURCE_FAILURE(10038,"delete data source failure", "删除数据源失败"), + VERIFY_DATASOURCE_NAME_FAILURE(10039,"verify datasource name failure", "验证数据源名称失败"), + UNAUTHORIZED_DATASOURCE(10040,"unauthorized datasource", "未经授权的数据源"), + AUTHORIZED_DATA_SOURCE(10041,"authorized data source", "授权数据源失败"), + LOGIN_SUCCESS(10042,"login success", "登录成功"), + USER_LOGIN_FAILURE(10043,"user login failure", "用户登录失败"), + LIST_WORKERS_ERROR(10044,"list workers error", "查询worker列表错误"), + LIST_MASTERS_ERROR(10045,"list masters error", "查询master列表错误"), + UPDATE_PROJECT_ERROR(10046,"update project error", "更新项目信息错误"), + QUERY_PROJECT_DETAILS_BY_ID_ERROR(10047,"query project details by id error", "查询项目详细信息错误"), + CREATE_PROJECT_ERROR(10048,"create project error", "创建项目错误"), + LOGIN_USER_QUERY_PROJECT_LIST_PAGING_ERROR(10049,"login user query project list paging error", "分页查询项目列表错误"), + DELETE_PROJECT_ERROR(10050,"delete project error", "删除项目错误"), + QUERY_UNAUTHORIZED_PROJECT_ERROR(10051,"query unauthorized project error", "查询未授权项目错误"), + QUERY_AUTHORIZED_PROJECT(10052,"query authorized project", "查询授权项目错误"), + QUERY_QUEUE_LIST_ERROR(10053,"query queue list error", "查询队列列表错误"), + CREATE_RESOURCE_ERROR(10054,"create resource error", "创建资源错误"), + UPDATE_RESOURCE_ERROR(10055,"update resource error", "更新资源错误"), + QUERY_RESOURCES_LIST_ERROR(10056,"query resources list error", "查询资源列表错误"), + QUERY_RESOURCES_LIST_PAGING(10057,"query resources list paging", "分页查询资源列表错误"), + DELETE_RESOURCE_ERROR(10058,"delete resource error", "删除资源错误"), + VERIFY_RESOURCE_BY_NAME_AND_TYPE_ERROR(10059,"verify resource by name and type error", "资源名称或类型验证错误"), + VIEW_RESOURCE_FILE_ON_LINE_ERROR(10060,"view resource file online error", "查看资源文件错误"), + CREATE_RESOURCE_FILE_ON_LINE_ERROR(10061,"create resource file online error", "创建资源文件错误"), + RESOURCE_FILE_IS_EMPTY(10062,"resource file is empty", "资源文件内容不能为空"), + EDIT_RESOURCE_FILE_ON_LINE_ERROR(10063,"edit resource file online error", "更新资源文件错误"), + DOWNLOAD_RESOURCE_FILE_ERROR(10064,"download resource file error", "下载资源文件错误"), + CREATE_UDF_FUNCTION_ERROR(10065 ,"create udf function error", "创建UDF函数错误"), + VIEW_UDF_FUNCTION_ERROR( 10066,"view udf function error", "查询UDF函数错误"), + UPDATE_UDF_FUNCTION_ERROR(10067,"update udf function error", "更新UDF函数错误"), + QUERY_UDF_FUNCTION_LIST_PAGING_ERROR( 10068,"query udf function list paging error", "分页查询UDF函数列表错误"), + QUERY_DATASOURCE_BY_TYPE_ERROR( 10069,"query datasource by type error", "查询数据源信息错误"), + VERIFY_UDF_FUNCTION_NAME_ERROR( 10070,"verify udf function name error", "UDF函数名称验证错误"), + DELETE_UDF_FUNCTION_ERROR( 10071,"delete udf function error", "删除UDF函数错误"), + AUTHORIZED_FILE_RESOURCE_ERROR( 10072,"authorized file resource error", "授权资源文件错误"), + UNAUTHORIZED_FILE_RESOURCE_ERROR( 10073,"unauthorized file resource error", "查询未授权资源错误"), + UNAUTHORIZED_UDF_FUNCTION_ERROR( 10074,"unauthorized udf function error", "查询未授权UDF函数错误"), + AUTHORIZED_UDF_FUNCTION_ERROR(10075,"authorized udf function error", "授权UDF函数错误"), + CREATE_SCHEDULE_ERROR(10076,"create schedule error", "创建调度配置错误"), + UPDATE_SCHEDULE_ERROR(10077,"update schedule error", "更新调度配置错误"), + PUBLISH_SCHEDULE_ONLINE_ERROR(10078,"publish schedule online error", "上线调度配置错误"), + OFFLINE_SCHEDULE_ERROR(10079,"offline schedule error", "下线调度配置错误"), + QUERY_SCHEDULE_LIST_PAGING_ERROR(10080,"query schedule list paging error", "分页查询调度配置列表错误"), + QUERY_SCHEDULE_LIST_ERROR(10081,"query schedule list error", "查询调度配置列表错误"), + QUERY_TASK_LIST_PAGING_ERROR(10082,"query task list paging error", "分页查询任务列表错误"), + QUERY_TASK_RECORD_LIST_PAGING_ERROR(10083,"query task record list paging error", "分页查询任务记录错误"), + CREATE_TENANT_ERROR(10084,"create tenant error", "创建租户错误"), + QUERY_TENANT_LIST_PAGING_ERROR(10085,"query tenant list paging error", "分页查询租户列表错误"), + QUERY_TENANT_LIST_ERROR(10086,"query tenant list error", "查询租户列表错误"), + UPDATE_TENANT_ERROR(10087,"update tenant error", "更新租户错误"), + DELETE_TENANT_BY_ID_ERROR(10088,"delete tenant by id error", "删除租户错误"), + VERIFY_TENANT_CODE_ERROR(10089,"verify tenant code error", "租户编码验证错误"), + CREATE_USER_ERROR(10090,"create user error", "创建用户错误"), + QUERY_USER_LIST_PAGING_ERROR(10091,"query user list paging error", "分页查询用户列表错误"), + UPDATE_USER_ERROR(10092,"update user error", "更新用户错误"), + DELETE_USER_BY_ID_ERROR(10093,"delete user by id error", "删除用户错误"), + GRANT_PROJECT_ERROR(10094,"grant project error", "授权项目错误"), + GRANT_RESOURCE_ERROR(10095,"grant resource error", "授权资源错误"), + GRANT_UDF_FUNCTION_ERROR(10096,"grant udf function error", "授权UDF函数错误"), + GRANT_DATASOURCE_ERROR(10097,"grant datasource error", "授权数据源错误"), + GET_USER_INFO_ERROR(10098,"get user info error", "获取用户信息错误"), + USER_LIST_ERROR(10099,"user list error", "查询用户列表错误"), + VERIFY_USERNAME_ERROR(10100,"verify username error", "用户名验证错误"), + UNAUTHORIZED_USER_ERROR(10101,"unauthorized user error", "查询未授权用户错误"), + AUTHORIZED_USER_ERROR(10102,"authorized user error", "查询授权用户错误"), + QUERY_TASK_INSTANCE_LOG_ERROR(10103,"view task instance log error", "查询任务实例日志错误"), + DOWNLOAD_TASK_INSTANCE_LOG_FILE_ERROR(10104,"download task instance log file error", "下载任务日志文件错误"), + CREATE_PROCESS_DEFINITION(10105,"create process definition", "创建工作流错误"), + VERIFY_PROCESS_DEFINITION_NAME_UNIQUE_ERROR(10106,"verify process definition name unique error", "工作流名称已存在"), + UPDATE_PROCESS_DEFINITION_ERROR(10107,"update process definition error", "更新工作流定义错误"), + RELEASE_PROCESS_DEFINITION_ERROR(10108,"release process definition error", "上线工作流错误"), + QUERY_DATAIL_OF_PROCESS_DEFINITION_ERROR(10109,"query datail of process definition error", "查询工作流详细信息错误"), + QUERY_PROCCESS_DEFINITION_LIST(10110,"query proccess definition list", "查询工作流列表错误"), + ENCAPSULATION_TREEVIEW_STRUCTURE_ERROR(10111,"encapsulation treeview structure error", "查询工作流树形图数据错误"), + GET_TASKS_LIST_BY_PROCESS_DEFINITION_ID_ERROR(10112,"get tasks list by process definition id error", "查询工作流定义节点信息错误"), + QUERY_PROCESS_INSTANCE_LIST_PAGING_ERROR(10113,"query process instance list paging error", "分页查询工作流实例列表错误"), + QUERY_TASK_LIST_BY_PROCESS_INSTANCE_ID_ERROR(10114,"query task list by process instance id error", "查询任务实例列表错误"), + UPDATE_PROCESS_INSTANCE_ERROR(10115,"update process instance error", "更新工作流实例错误"), + QUERY_PROCESS_INSTANCE_BY_ID_ERROR(10116,"query process instance by id error", "查询工作流实例错误"), + DELETE_PROCESS_INSTANCE_BY_ID_ERROR(10117,"delete process instance by id error", "删除工作流实例错误"), + QUERY_SUB_PROCESS_INSTANCE_DETAIL_INFO_BY_TASK_ID_ERROR(10118,"query sub process instance detail info by task id error", "查询子流程任务实例错误"), + QUERY_PARENT_PROCESS_INSTANCE_DETAIL_INFO_BY_SUB_PROCESS_INSTANCE_ID_ERROR(10119,"query parent process instance detail info by sub process instance id error", "查询子流程该工作流实例错误"), + QUERY_PROCESS_INSTANCE_ALL_VARIABLES_ERROR(10120,"query process instance all variables error", "查询工作流自定义变量信息错误"), + ENCAPSULATION_PROCESS_INSTANCE_GANTT_STRUCTURE_ERROR(10121,"encapsulation process instance gantt structure error", "查询工作流实例甘特图数据错误"), + QUERY_PROCCESS_DEFINITION_LIST_PAGING_ERROR(10122,"query proccess definition list paging error", "分页查询工作流定义列表错误"), + SIGN_OUT_ERROR(10123,"sign out error", "退出错误"), + TENANT_CODE_HAS_ALREADY_EXISTS(10124,"tenant code has already exists", "租户编码已存在"), + IP_IS_EMPTY(10125,"ip is empty", "IP地址不能为空"), + SCHEDULE_CRON_REALEASE_NEED_NOT_CHANGE(10126, "schedule release is already {0}", "调度配置上线错误[{0}]"), + CREATE_QUEUE_ERROR(10127, "create queue error", "创建队列错误"), + QUEUE_NOT_EXIST(10128, "queue {0} not exists", "队列ID[{0}]不存在"), + QUEUE_VALUE_EXIST(10129, "queue value {0} already exists", "队列值[{0}]已存在"), + QUEUE_NAME_EXIST(10130, "queue name {0} already exists", "队列名称[{0}]已存在"), + UPDATE_QUEUE_ERROR(10131, "update queue error", "更新队列信息错误"), + NEED_NOT_UPDATE_QUEUE(10132, "no content changes, no updates are required", "数据未变更,不需要更新队列信息"), + VERIFY_QUEUE_ERROR(10133,"verify queue error", "验证队列信息错误"), + NAME_NULL(10134,"name must be not null", "名称不能为空"), + NAME_EXIST(10135, "name {0} already exists", "名称[{0}]已存在"), + SAVE_ERROR(10136, "save error", "保存错误"), + DELETE_PROJECT_ERROR_DEFINES_NOT_NULL(10137, "please delete the process definitions in project first!", "请先删除全部工作流定义"), + BATCH_DELETE_PROCESS_INSTANCE_BY_IDS_ERROR(10117,"batch delete process instance by ids {0} error", "批量删除工作流实例错误"), + PREVIEW_SCHEDULE_ERROR(10139,"preview schedule error", "预览调度配置错误"), + PARSE_TO_CRON_EXPRESSION_ERROR(10140,"parse cron to cron expression error", "解析调度表达式错误"), + SCHEDULE_START_TIME_END_TIME_SAME(10141,"The start time must not be the same as the end", "开始时间不能和结束时间一样"), + DELETE_TENANT_BY_ID_FAIL(100142,"delete tenant by id fail, for there are {0} process instances in executing using it", "删除租户失败,有[{0}]个运行中的工作流实例正在使用"), + DELETE_TENANT_BY_ID_FAIL_DEFINES(100143,"delete tenant by id fail, for there are {0} process definitions using it", "删除租户失败,有[{0}]个工作流定义正在使用"), + DELETE_TENANT_BY_ID_FAIL_USERS(100144,"delete tenant by id fail, for there are {0} users using it", "删除租户失败,有[{0}]个用户正在使用"), - DELETE_WORKER_GROUP_BY_ID_FAIL(100145,"delete worker group by id fail, for there are {0} process instances in executing using it"), + DELETE_WORKER_GROUP_BY_ID_FAIL(100145,"delete worker group by id fail, for there are {0} process instances in executing using it", "删除Worker分组失败,有[{0}]个运行中的工作流实例正在使用"), - QUERY_WORKER_GROUP_FAIL(100146,"query worker group fail "), - DELETE_WORKER_GROUP_FAIL(100147,"delete worker group fail "), + QUERY_WORKER_GROUP_FAIL(100146,"query worker group fail ", "查询worker分组失败"), + DELETE_WORKER_GROUP_FAIL(100147,"delete worker group fail ", "删除worker分组失败"), - UDF_FUNCTION_NOT_EXIST(20001, "UDF function not found"), - UDF_FUNCTION_EXISTS(20002, "UDF function already exists"), - RESOURCE_NOT_EXIST(20004, "resource not exist"), - RESOURCE_EXIST(20005, "resource already exists"), - RESOURCE_SUFFIX_NOT_SUPPORT_VIEW(20006, "resource suffix do not support online viewing"), - RESOURCE_SIZE_EXCEED_LIMIT(20007, "upload resource file size exceeds limit"), - RESOURCE_SUFFIX_FORBID_CHANGE(20008, "resource suffix not allowed to be modified"), - UDF_RESOURCE_SUFFIX_NOT_JAR(20009, "UDF resource suffix name must be jar"), - HDFS_COPY_FAIL(20009, "hdfs copy {0} -> {1} fail"), - RESOURCE_FILE_EXIST(20010, "resource file {0} already exists in hdfs,please delete it or change name!"), - RESOURCE_FILE_NOT_EXIST(20011, "resource file {0} not exists in hdfs!"), + UDF_FUNCTION_NOT_EXIST(20001, "UDF function not found", "UDF函数不存在"), + UDF_FUNCTION_EXISTS(20002, "UDF function already exists", "UDF函数已存在"), + RESOURCE_NOT_EXIST(20004, "resource not exist", "资源不存在"), + RESOURCE_EXIST(20005, "resource already exists", "资源已存在"), + RESOURCE_SUFFIX_NOT_SUPPORT_VIEW(20006, "resource suffix do not support online viewing", "资源文件后缀不支持查看"), + RESOURCE_SIZE_EXCEED_LIMIT(20007, "upload resource file size exceeds limit", "上传资源文件大小超过限制"), + RESOURCE_SUFFIX_FORBID_CHANGE(20008, "resource suffix not allowed to be modified", "资源文件后缀不支持修改"), + UDF_RESOURCE_SUFFIX_NOT_JAR(20009, "UDF resource suffix name must be jar", "UDF资源文件后缀名只支持[jar]"), + HDFS_COPY_FAIL(20009, "hdfs copy {0} -> {1} fail", "hdfs复制失败:[{0}] -> [{1}]"), + RESOURCE_FILE_EXIST(20010, "resource file {0} already exists in hdfs,please delete it or change name!", "资源文件[{0}]在hdfs中已存在,请删除或修改资源名"), + RESOURCE_FILE_NOT_EXIST(20011, "resource file {0} not exists in hdfs!", "资源文件[{0}]在hdfs中不存在"), - USER_NO_OPERATION_PERM(30001, "user has no operation privilege"), - USER_NO_OPERATION_PROJECT_PERM(30002, "user {0} is not has project {1} permission"), + USER_NO_OPERATION_PERM(30001, "user has no operation privilege", "当前用户没有操作权限"), + USER_NO_OPERATION_PROJECT_PERM(30002, "user {0} is not has project {1} permission", "当前用户[{0}]没有[{1}]项目的操作权限"), - PROCESS_INSTANCE_NOT_EXIST(50001, "process instance {0} does not exist"), - PROCESS_INSTANCE_EXIST(50002, "process instance {0} already exists"), - PROCESS_DEFINE_NOT_EXIST(50003, "process definition {0} does not exist"), - PROCESS_DEFINE_NOT_RELEASE(50004, "process definition {0} not on line"), - PROCESS_INSTANCE_ALREADY_CHANGED(50005, "the status of process instance {0} is already {1}"), - PROCESS_INSTANCE_STATE_OPERATION_ERROR(50006, "the status of process instance {0} is {1},Cannot perform {2} operation"), - SUB_PROCESS_INSTANCE_NOT_EXIST(50007, "the task belong to process instance does not exist"), - PROCESS_DEFINE_NOT_ALLOWED_EDIT(50008, "process definition {0} does not allow edit"), - PROCESS_INSTANCE_EXECUTING_COMMAND(50009, "process instance {0} is executing the command, please wait ..."), - PROCESS_INSTANCE_NOT_SUB_PROCESS_INSTANCE(50010, "process instance {0} is not sub process instance"), - TASK_INSTANCE_STATE_COUNT_ERROR(50011,"task instance state count error"), - COUNT_PROCESS_INSTANCE_STATE_ERROR(50012,"count process instance state error"), - COUNT_PROCESS_DEFINITION_USER_ERROR(50013,"count process definition user error"), - START_PROCESS_INSTANCE_ERROR(50014,"start process instance error"), - EXECUTE_PROCESS_INSTANCE_ERROR(50015,"execute process instance error"), - CHECK_PROCESS_DEFINITION_ERROR(50016,"check process definition error"), - QUERY_RECIPIENTS_AND_COPYERS_BY_PROCESS_DEFINITION_ERROR(50017,"query recipients and copyers by process definition error"), - DATA_IS_NOT_VALID(50017,"data %s not valid"), - DATA_IS_NULL(50018,"data %s is null"), - PROCESS_NODE_HAS_CYCLE(50019,"process node has cycle"), - PROCESS_NODE_S_PARAMETER_INVALID(50020,"process node %s parameter invalid"), - PROCESS_DEFINE_STATE_ONLINE(50021, "process definition {0} is already on line"), - DELETE_PROCESS_DEFINE_BY_ID_ERROR(50022,"delete process definition by id error"), - SCHEDULE_CRON_STATE_ONLINE(50023,"the status of schedule {0} is already on line"), - DELETE_SCHEDULE_CRON_BY_ID_ERROR(50024,"delete schedule by id error"), - BATCH_DELETE_PROCESS_DEFINE_ERROR(50025,"batch delete process definition error"), - BATCH_DELETE_PROCESS_DEFINE_BY_IDS_ERROR(50026,"batch delete process definition by ids {0} error"), - TENANT_NOT_SUITABLE(50027,"there is not any tenant suitable, please choose a tenant available."), - EXPORT_PROCESS_DEFINE_BY_ID_ERROR(50028,"export process definition by id error"), - IMPORT_PROCESS_DEFINE_ERROR(50029,"import process definition error"), + PROCESS_INSTANCE_NOT_EXIST(50001, "process instance {0} does not exist", "工作流实例[{0}]不存在"), + PROCESS_INSTANCE_EXIST(50002, "process instance {0} already exists", "工作流实例[{0}]已存在"), + PROCESS_DEFINE_NOT_EXIST(50003, "process definition {0} does not exist", "工作流定义[{0}]不存在"), + PROCESS_DEFINE_NOT_RELEASE(50004, "process definition {0} not on line", "工作流定义[{0}]不是上线状态"), + PROCESS_INSTANCE_ALREADY_CHANGED(50005, "the status of process instance {0} is already {1}", "工作流实例[{0}]的状态已经是[{1}]"), + PROCESS_INSTANCE_STATE_OPERATION_ERROR(50006, "the status of process instance {0} is {1},Cannot perform {2} operation", "工作流实例[{0}]的状态是[{1}],无法执行[{2}]操作"), + SUB_PROCESS_INSTANCE_NOT_EXIST(50007, "the task belong to process instance does not exist", "子工作流实例不存在"), + PROCESS_DEFINE_NOT_ALLOWED_EDIT(50008, "process definition {0} does not allow edit", "工作流定义[{0}]不允许修改"), + PROCESS_INSTANCE_EXECUTING_COMMAND(50009, "process instance {0} is executing the command, please wait ...", "工作流实例[{0}]正在执行命令,请稍等..."), + PROCESS_INSTANCE_NOT_SUB_PROCESS_INSTANCE(50010, "process instance {0} is not sub process instance", "工作流实例[{0}]不是子工作流实例"), + TASK_INSTANCE_STATE_COUNT_ERROR(50011,"task instance state count error", "查询各状态任务实例数错误"), + COUNT_PROCESS_INSTANCE_STATE_ERROR(50012,"count process instance state error", "查询各状态流程实例数错误"), + COUNT_PROCESS_DEFINITION_USER_ERROR(50013,"count process definition user error", "查询各用户流程定义数错误"), + START_PROCESS_INSTANCE_ERROR(50014,"start process instance error", "运行工作流实例错误"), + EXECUTE_PROCESS_INSTANCE_ERROR(50015,"execute process instance error", "操作工作流实例错误"), + CHECK_PROCESS_DEFINITION_ERROR(50016,"check process definition error", "检查工作流实例错误"), + QUERY_RECIPIENTS_AND_COPYERS_BY_PROCESS_DEFINITION_ERROR(50017,"query recipients and copyers by process definition error", "查询收件人和抄送人错误"), + DATA_IS_NOT_VALID(50017,"data %s not valid", "数据[%s]无效"), + DATA_IS_NULL(50018,"data %s is null", "数据[%s]不能为空"), + PROCESS_NODE_HAS_CYCLE(50019,"process node has cycle", "流程节点间存在循环依赖"), + PROCESS_NODE_S_PARAMETER_INVALID(50020,"process node %s parameter invalid", "流程节点[%s]参数无效"), + PROCESS_DEFINE_STATE_ONLINE(50021, "process definition {0} is already on line", "工作流定义[{0}]已上线"), + DELETE_PROCESS_DEFINE_BY_ID_ERROR(50022,"delete process definition by id error", "删除工作流定义错误"), + SCHEDULE_CRON_STATE_ONLINE(50023,"the status of schedule {0} is already on line", "调度配置[{0}]已上线"), + DELETE_SCHEDULE_CRON_BY_ID_ERROR(50024,"delete schedule by id error", "删除调度配置错误"), + BATCH_DELETE_PROCESS_DEFINE_ERROR(50025,"batch delete process definition error", "批量删除工作流定义错误"), + BATCH_DELETE_PROCESS_DEFINE_BY_IDS_ERROR(50026,"batch delete process definition by ids {0} error", "批量删除工作流定义[{0}]错误"), + TENANT_NOT_SUITABLE(50027,"there is not any tenant suitable, please choose a tenant available.", "没有合适的租户,请选择可用的租户"), + EXPORT_PROCESS_DEFINE_BY_ID_ERROR(50028,"export process definition by id error", "导出工作流定义错误"), + IMPORT_PROCESS_DEFINE_ERROR(50029,"import process definition error", "导入工作流定义错误"), - HDFS_NOT_STARTUP(60001,"hdfs not startup"), - HDFS_TERANT_RESOURCES_FILE_EXISTS(60002,"resource file exists,please delete resource first"), - HDFS_TERANT_UDFS_FILE_EXISTS(60003,"udf file exists,please delete resource first"), + HDFS_NOT_STARTUP(60001,"hdfs not startup", "hdfs未启用"), /** * for monitor */ - QUERY_DATABASE_STATE_ERROR(70001,"query database state error"), - QUERY_ZOOKEEPER_STATE_ERROR(70002,"query zookeeper state error"), + QUERY_DATABASE_STATE_ERROR(70001,"query database state error", "查询数据库状态错误"), + QUERY_ZOOKEEPER_STATE_ERROR(70002,"query zookeeper state error", "查询zookeeper状态错误"), - CREATE_ACCESS_TOKEN_ERROR(70010,"create access token error"), - GENERATE_TOKEN_ERROR(70011,"generate token error"), - QUERY_ACCESSTOKEN_LIST_PAGING_ERROR(70012,"query access token list paging error"), - UPDATE_ACCESS_TOKEN_ERROR(70013,"update access token error"), - DELETE_ACCESS_TOKEN_ERROR(70014,"delete access token error"), - ACCESS_TOKEN_NOT_EXIST(70015, "access token not exist"), + CREATE_ACCESS_TOKEN_ERROR(70010,"create access token error", "创建访问token错误"), + GENERATE_TOKEN_ERROR(70011,"generate token error", "生成token错误"), + QUERY_ACCESSTOKEN_LIST_PAGING_ERROR(70012,"query access token list paging error", "分页查询访问token列表错误"), + UPDATE_ACCESS_TOKEN_ERROR(70013,"update access token error", "更新访问token错误"), + DELETE_ACCESS_TOKEN_ERROR(70014,"delete access token error", "删除访问token错误"), + ACCESS_TOKEN_NOT_EXIST(70015, "access token not exist", "访问token不存在"), - COMMAND_STATE_COUNT_ERROR(80001,"task instance state count error"), + COMMAND_STATE_COUNT_ERROR(80001,"task instance state count error", "查询各状态任务实例数错误"), - QUEUE_COUNT_ERROR(90001,"queue count error"), + QUEUE_COUNT_ERROR(90001,"queue count error", "查询队列数据错误"), - KERBEROS_STARTUP_STATE(100001,"get kerberos startup state error"), + KERBEROS_STARTUP_STATE(100001,"get kerberos startup state error", "获取kerberos启动状态错误"), ; private final int code; - private final String msg; + private final String enMsg; + private final String zhMsg; - private Status(int code, String msg) { + private Status(int code, String enMsg, String zhMsg) { this.code = code; - this.msg = msg; + this.enMsg = enMsg; + this.zhMsg = zhMsg; } public int getCode() { @@ -261,6 +265,10 @@ public enum Status { } public String getMsg() { - return this.msg; + if (Locale.SIMPLIFIED_CHINESE.getLanguage().equals(LocaleContextHolder.getLocale().getLanguage())) { + return this.zhMsg; + } else { + return this.enMsg; + } } } diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/enums/StatusTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/enums/StatusTest.java index 05d785e1a1..0c9ddff791 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/enums/StatusTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/enums/StatusTest.java @@ -16,7 +16,12 @@ */ package org.apache.dolphinscheduler.api.enums; +import org.junit.Assert; import org.junit.Test; +import org.springframework.context.i18n.LocaleContextHolder; + +import java.util.Locale; + import static org.junit.Assert.*; public class StatusTest { @@ -29,6 +34,11 @@ public class StatusTest { @Test public void testGetMsg() { - assertEquals("success", Status.SUCCESS.getMsg()); + LocaleContextHolder.setLocale(Locale.US); + Assert.assertEquals("success", Status.SUCCESS.getMsg()); + + LocaleContextHolder.setLocale(Locale.SIMPLIFIED_CHINESE); + Assert.assertEquals("成功", Status.SUCCESS.getMsg()); } + } From c9c4402cb65c89fa587a0cbbb55e2ca687cc9605 Mon Sep 17 00:00:00 2001 From: "xiaochun.liu" Date: Mon, 2 Mar 2020 10:55:40 +0800 Subject: [PATCH 048/221] modify action build --- .github/workflows/ci_backend.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/ci_backend.yml b/.github/workflows/ci_backend.yml index e527c3c4a2..afd241d94e 100644 --- a/.github/workflows/ci_backend.yml +++ b/.github/workflows/ci_backend.yml @@ -51,7 +51,7 @@ jobs: with: java-version: 1.8 - name: Compile - run: mvn -U -B -T 1C clean install -Prelease -Dmaven.compile.fork=true -Dmaven.test.skip=true + run: mvn -B clean compile package -Prelease -Dmaven.test.skip=true License-check: runs-on: ubuntu-latest steps: From 3d4acbc012d186910a3a836423a28bd72251cd59 Mon Sep 17 00:00:00 2001 From: bcai Date: Mon, 2 Mar 2020 15:02:17 +0800 Subject: [PATCH 049/221] fix github action rerun failed #2052 --- .github/workflows/ci_backend.yml | 4 ++-- .github/workflows/ci_e2e.yml | 2 +- .github/workflows/ci_frontend.yml | 4 ++-- .github/workflows/ci_ut.yml | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/.github/workflows/ci_backend.yml b/.github/workflows/ci_backend.yml index afd241d94e..1ca15c2813 100644 --- a/.github/workflows/ci_backend.yml +++ b/.github/workflows/ci_backend.yml @@ -45,7 +45,7 @@ jobs: Compile-check: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v1 + - uses: actions/checkout@v2 - name: Set up JDK 1.8 uses: actions/setup-java@v1 with: @@ -55,7 +55,7 @@ jobs: License-check: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v1 + - uses: actions/checkout@v2 - name: Set up JDK 1.8 uses: actions/setup-java@v1 with: diff --git a/.github/workflows/ci_e2e.yml b/.github/workflows/ci_e2e.yml index 13f12641fe..09f66fab11 100644 --- a/.github/workflows/ci_e2e.yml +++ b/.github/workflows/ci_e2e.yml @@ -29,7 +29,7 @@ jobs: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v1 + - uses: actions/checkout@v2 with: submodules: true - uses: actions/cache@v1 diff --git a/.github/workflows/ci_frontend.yml b/.github/workflows/ci_frontend.yml index fab75c6341..1f959b40eb 100644 --- a/.github/workflows/ci_frontend.yml +++ b/.github/workflows/ci_frontend.yml @@ -34,7 +34,7 @@ jobs: matrix: os: [ubuntu-latest, macos-latest] steps: - - uses: actions/checkout@v1 + - uses: actions/checkout@v2 - name: Set up Node.js uses: actions/setup-node@v1 with: @@ -49,7 +49,7 @@ jobs: License-check: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v1 + - uses: actions/checkout@v2 - name: Set up JDK 1.8 uses: actions/setup-java@v1 with: diff --git a/.github/workflows/ci_ut.yml b/.github/workflows/ci_ut.yml index 6f3eda0362..6451c2f8bb 100644 --- a/.github/workflows/ci_ut.yml +++ b/.github/workflows/ci_ut.yml @@ -29,7 +29,7 @@ jobs: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v1 + - uses: actions/checkout@v2 with: submodules: true - uses: actions/cache@v1 From 8b894de3d2caf644818d7c84444327a43d864d16 Mon Sep 17 00:00:00 2001 From: bcai Date: Mon, 2 Mar 2020 15:18:35 +0800 Subject: [PATCH 050/221] remove submodules for checkout v2 --- .github/workflows/ci_e2e.yml | 2 -- .github/workflows/ci_ut.yml | 2 -- 2 files changed, 4 deletions(-) diff --git a/.github/workflows/ci_e2e.yml b/.github/workflows/ci_e2e.yml index 09f66fab11..5282681fb6 100644 --- a/.github/workflows/ci_e2e.yml +++ b/.github/workflows/ci_e2e.yml @@ -30,8 +30,6 @@ jobs: steps: - uses: actions/checkout@v2 - with: - submodules: true - uses: actions/cache@v1 with: path: ~/.m2/repository diff --git a/.github/workflows/ci_ut.yml b/.github/workflows/ci_ut.yml index 6451c2f8bb..45ff3ca29d 100644 --- a/.github/workflows/ci_ut.yml +++ b/.github/workflows/ci_ut.yml @@ -30,8 +30,6 @@ jobs: steps: - uses: actions/checkout@v2 - with: - submodules: true - uses: actions/cache@v1 with: path: ~/.m2/repository From fbf05de39b5c3e2060f1a145dc7598c739916db6 Mon Sep 17 00:00:00 2001 From: Tboy Date: Tue, 3 Mar 2020 12:27:07 +0800 Subject: [PATCH 051/221] delete duplicate startLocalZkServer (#2062) --- dolphinscheduler-service/src/test/java/queue/ZKServer.java | 1 - 1 file changed, 1 deletion(-) diff --git a/dolphinscheduler-service/src/test/java/queue/ZKServer.java b/dolphinscheduler-service/src/test/java/queue/ZKServer.java index 65fb95c02b..3d503d5691 100644 --- a/dolphinscheduler-service/src/test/java/queue/ZKServer.java +++ b/dolphinscheduler-service/src/test/java/queue/ZKServer.java @@ -80,7 +80,6 @@ public class ZKServer { */ public static void startLocalZkServer(final int port) { startLocalZkServer(port, org.apache.commons.io.FileUtils.getTempDirectoryPath() + File.separator + "test-" + System.currentTimeMillis()); - startLocalZkServer(port, org.apache.commons.io.FileUtils.getTempDirectoryPath() + "test-" + System.currentTimeMillis()); } /** From 28f57f87ef265323b339ad907ce7d97d746256dd Mon Sep 17 00:00:00 2001 From: break60 <790061044@qq.com> Date: Tue, 3 Mar 2020 14:11:05 +0800 Subject: [PATCH 052/221] Change DOM label --- .../formModel/tasks/_source/nodeStatus.vue | 10 +++++----- .../dag/_source/formModel/tasks/conditions.vue | 15 +++++---------- 2 files changed, 10 insertions(+), 15 deletions(-) diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/nodeStatus.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/nodeStatus.vue index 4afb8b46c5..fa7ee89e98 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/nodeStatus.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/nodeStatus.vue @@ -27,17 +27,17 @@ - + - +
    diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/conditions.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/conditions.vue index 4ac04d91a6..e4f9b3f094 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/conditions.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/conditions.vue @@ -19,16 +19,12 @@
    {{$t('Custom Parameters')}}
    -
    +
    @@ -44,14 +40,13 @@ @click="!isDetails && _setRelation($index)"> {{el.relation === 'AND' ? $t('and') : $t('or')}} - -  - + Date: Tue, 3 Mar 2020 14:23:22 +0800 Subject: [PATCH 053/221] Change name to lowercase --- .../src/js/conf/home/pages/dag/_source/formModel/formModel.vue | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue index 2500ce5772..ec228fa7ab 100755 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue @@ -280,7 +280,7 @@ import mDependent from './tasks/dependent' import mHttp from './tasks/http' import mDatax from './tasks/datax' - import mConditions from './tasks/CONDITIONS' + import mConditions from './tasks/conditions' import mSqoop from './tasks/sqoop' import mSubProcess from './tasks/sub_process' import mSelectInput from './_source/selectInput' From 3277b90647296ab3c72d74e1b22d2ac22c394eba Mon Sep 17 00:00:00 2001 From: break60 <790061044@qq.com> Date: Tue, 3 Mar 2020 14:38:21 +0800 Subject: [PATCH 054/221] Change DOM label (#2064) * Change DOM label * Change name to lowercase --- .../pages/dag/_source/formModel/formModel.vue | 2 +- .../formModel/tasks/_source/nodeStatus.vue | 10 +++++----- .../dag/_source/formModel/tasks/conditions.vue | 15 +++++---------- 3 files changed, 11 insertions(+), 16 deletions(-) diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue index 2500ce5772..ec228fa7ab 100755 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue @@ -280,7 +280,7 @@ import mDependent from './tasks/dependent' import mHttp from './tasks/http' import mDatax from './tasks/datax' - import mConditions from './tasks/CONDITIONS' + import mConditions from './tasks/conditions' import mSqoop from './tasks/sqoop' import mSubProcess from './tasks/sub_process' import mSelectInput from './_source/selectInput' diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/nodeStatus.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/nodeStatus.vue index 4afb8b46c5..fa7ee89e98 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/nodeStatus.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/nodeStatus.vue @@ -27,17 +27,17 @@ - + - +
    diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/conditions.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/conditions.vue index 4ac04d91a6..e4f9b3f094 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/conditions.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/conditions.vue @@ -19,16 +19,12 @@
    {{$t('Custom Parameters')}}
    -
    +
    @@ -44,14 +40,13 @@ @click="!isDetails && _setRelation($index)"> {{el.relation === 'AND' ? $t('and') : $t('or')}} - -  - + Date: Wed, 4 Mar 2020 09:28:23 +0800 Subject: [PATCH 055/221] 1. add input cache for datax task (#2034) 2. add cache for codemirror component, include python, shell and sql task. --- .../dag/_source/formModel/tasks/datax.vue | 49 ++++++++++++++++++- .../dag/_source/formModel/tasks/python.vue | 34 +++++++++++-- .../dag/_source/formModel/tasks/shell.vue | 29 +++++++++-- .../pages/dag/_source/formModel/tasks/sql.vue | 44 ++++++++++++++++- 4 files changed, 145 insertions(+), 11 deletions(-) diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/datax.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/datax.vue index ce918f49cf..959610f95a 100755 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/datax.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/datax.vue @@ -232,6 +232,8 @@ * Processing code highlighting */ _handlerEditor () { + this._destroyEditor() + // editor editor = codemirror('code-sql-mirror', { mode: 'sql', @@ -249,9 +251,34 @@ // Monitor keyboard editor.on('keypress', this.keypress) + editor.on('changes', () => { + this._cacheParams() + }) + editor.setValue(this.sql) return editor + }, + _cacheParams () { + this.$emit('on-cache-params', { + dsType: this.dsType, + dataSource: this.rtDatasource, + dtType: this.dtType, + dataTarget: this.rtDatatarget, + sql: editor?editor.getValue():'', + targetTable: this.targetTable, + jobSpeedByte: this.jobSpeedByte * 1024, + jobSpeedRecord: this.jobSpeedRecord, + preStatements: this.preStatements, + postStatements: this.postStatements + }); + }, + _destroyEditor () { + if (editor) { + editor.toTextArea() // Uninstall + editor.off($('.code-sql-mirror'), 'keypress', this.keypress) + editor.off($('.code-sql-mirror'), 'changes', this.changes) + } } }, created () { @@ -286,7 +313,27 @@ editor.off($('.code-sql-mirror'), 'keypress', this.keypress) } }, - computed: {}, + watch: { + //Watch the cacheParams + cacheParams (val) { + this._cacheParams(); + } + }, + computed: { + cacheParams () { + return { + dsType: this.dsType, + dataSource: this.rtDatasource, + dtType: this.dtType, + dataTarget: this.rtDatatarget, + targetTable: this.targetTable, + jobSpeedByte: this.jobSpeedByte * 1024, + jobSpeedRecord: this.jobSpeedRecord, + preStatements: this.preStatements, + postStatements: this.postStatements + } + } + }, components: { mListBox, mDatasource, mLocalParams, mStatementList, mSelectInput } } diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/python.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/python.vue index e565b4a6bd..6f495d22a0 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/python.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/python.vue @@ -129,6 +129,8 @@ * Processing code highlighting */ _handlerEditor () { + this._destroyEditor() + // editor editor = codemirror('code-python-mirror', { mode: 'python', @@ -143,26 +145,45 @@ } } + this.changes = () => { + this._cacheParams() + } + // Monitor keyboard editor.on('keypress', this.keypress) + editor.on('changes', this.changes) + editor.setValue(this.rawScript) return editor + }, + _cacheParams () { + this.$emit('on-cache-params', { + resourceList: this.cacheResourceList, + localParams: this.localParams, + rawScript: editor ? editor.getValue() : '' + }); + }, + _destroyEditor () { + if (editor) { + editor.toTextArea() // Uninstall + editor.off($('.code-python-mirror'), 'keypress', this.keypress) + editor.off($('.code-python-mirror'), 'changes', this.changes) + } } }, watch: { //Watch the cacheParams cacheParams (val) { - this.$emit('on-cache-params', val); + this._cacheParams() } }, computed: { cacheParams () { return { resourceList: this.cacheResourceList, - localParams: this.localParams, - rawScript: editor ? editor.getValue() : '' + localParams: this.localParams } } }, @@ -193,8 +214,11 @@ }, 200) }, destroyed () { - editor.toTextArea() // Uninstall - editor.off($('.code-python-mirror'), 'keypress', this.keypress) + if (editor) { + editor.toTextArea() // Uninstall + editor.off($('.code-python-mirror'), 'keypress', this.keypress) + editor.off($('.code-python-mirror'), 'changes', this.changes) + } }, components: { mLocalParams, mListBox, mResources } } diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/shell.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/shell.vue index ad40c586b9..a4b20f3310 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/shell.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/shell.vue @@ -163,6 +163,8 @@ * Processing code highlighting */ _handlerEditor () { + this._destroyEditor() + // editor editor = codemirror('code-shell-mirror', { mode: 'shell', @@ -177,25 +179,45 @@ } } + this.changes = () => { + this._cacheParams() + } + // Monitor keyboard editor.on('keypress', this.keypress) + + editor.on('changes', this.changes) + editor.setValue(this.rawScript) return editor + }, + _cacheParams () { + this.$emit('on-cache-params', { + resourceList: this.cacheResourceList, + localParams: this.localParams, + rawScript: editor ? editor.getValue() : '' + }); + }, + _destroyEditor () { + if (editor) { + editor.toTextArea() // Uninstall + editor.off($('.code-sql-mirror'), 'keypress', this.keypress) + editor.off($('.code-sql-mirror'), 'changes', this.changes) + } } }, watch: { //Watch the cacheParams cacheParams (val) { - this.$emit('on-cache-params', val); + this._cacheParams() } }, computed: { cacheParams () { return { resourceList: this.cacheResourceList, - localParams: this.localParams, - rawScript: editor ? editor.getValue() : '' + localParams: this.localParams } } }, @@ -229,6 +251,7 @@ if (editor) { editor.toTextArea() // Uninstall editor.off($('.code-shell-mirror'), 'keypress', this.keypress) + editor.off($('.code-shell-mirror'), 'changes', this.changes) } }, components: { mLocalParams, mListBox, mResources, mScriptBox } diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/sql.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/sql.vue index eb5c40992e..843c1eaf1f 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/sql.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/sql.vue @@ -318,6 +318,8 @@ * Processing code highlighting */ _handlerEditor () { + this._destroyEditor() + // editor editor = codemirror('code-sql-mirror', { mode: 'sql', @@ -332,9 +334,15 @@ } } + this.changes = () => { + this._cacheParams() + } + // Monitor keyboard editor.on('keypress', this.keypress) + editor.on('changes', this.changes) + editor.setValue(this.sql) return editor @@ -351,6 +359,38 @@ this.receivers = res.receivers && res.receivers.split(',') || [] this.receiversCc = res.receiversCc && res.receiversCc.split(',') || [] }) + }, + _cacheParams () { + this.$emit('on-cache-params', { + type: this.type, + datasource: this.rtDatasource, + sql: editor ? editor.getValue() : '', + udfs: this.udfs, + sqlType: this.sqlType, + title: this.title, + receivers: this.receivers.join(','), + receiversCc: this.receiversCc.join(','), + showType: (() => { + + let showType = this.showType + if (showType.length === 2 && showType[0] === 'ATTACHMENT') { + return [showType[1], showType[0]].join(',') + } else { + return showType.join(',') + } + })(), + localParams: this.localParams, + connParams: this.connParams, + preStatements: this.preStatements, + postStatements: this.postStatements + }); + }, + _destroyEditor () { + if (editor) { + editor.toTextArea() // Uninstall + editor.off($('.code-sql-mirror'), 'keypress', this.keypress) + editor.off($('.code-sql-mirror'), 'changes', this.changes) + } } }, watch: { @@ -373,7 +413,7 @@ }, //Watch the cacheParams cacheParams (val) { - this.$emit('on-cache-params', val); + this._cacheParams() } }, created () { @@ -418,6 +458,7 @@ if (editor) { editor.toTextArea() // Uninstall editor.off($('.code-sql-mirror'), 'keypress', this.keypress) + editor.off($('.code-sql-mirror'), 'changes', this.changes) } }, computed: { @@ -425,7 +466,6 @@ return { type: this.type, datasource: this.rtDatasource, - sql: editor ? editor.getValue() : '', udfs: this.udfs, sqlType: this.sqlType, title: this.title, From b8014d91cf7c03ab821583f47f84a8af6606b675 Mon Sep 17 00:00:00 2001 From: qiaozhanwei Date: Thu, 5 Mar 2020 11:04:00 +0800 Subject: [PATCH 056/221] add workflow id, name for email content (#2071) * add tenantService UT * fix spell * #464 email content * modify mysql1.2.2 ddl --- .../dolphinscheduler/api/service/TenantServiceTest.java | 8 ++++++++ .../api/utils/ZookeeperMonitorUtilsTest.java | 2 +- .../java/org/apache/dolphinscheduler/dao/AlertDao.java | 6 ++++-- .../server/master/runner/MasterTaskExecThread.java | 4 +++- .../dolphinscheduler/server/utils/AlertManager.java | 4 +++- sql/upgrade/1.2.2_schema/mysql/dolphinscheduler_ddl.sql | 2 +- .../1.2.2_schema/postgresql/dolphinscheduler_ddl.sql | 2 +- 7 files changed, 21 insertions(+), 7 deletions(-) diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TenantServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TenantServiceTest.java index 31c8c0222d..d6fb6b219c 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TenantServiceTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/TenantServiceTest.java @@ -175,6 +175,14 @@ public class TenantServiceTest { logger.info(result.toString()); List tenantList = (List) result.get(Constants.DATA_LIST); Assert.assertTrue(CollectionUtils.isNotEmpty(tenantList)); + + Mockito.when( tenantMapper.queryByTenantCode("1")).thenReturn(getList()); + Map successRes = tenantService.queryTenantList("1"); + Assert.assertEquals(Status.SUCCESS,successRes.get(Constants.STATUS)); + + Mockito.when( tenantMapper.queryByTenantCode("1")).thenReturn(null); + Map tenantNotExistRes = tenantService.queryTenantList("1"); + Assert.assertEquals(Status.TENANT_NOT_EXIST,tenantNotExistRes.get(Constants.STATUS)); } @Test diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/utils/ZookeeperMonitorUtilsTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/utils/ZookeeperMonitorUtilsTest.java index b3626fa8a9..0d89d4b6e3 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/utils/ZookeeperMonitorUtilsTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/utils/ZookeeperMonitorUtilsTest.java @@ -28,7 +28,7 @@ public class ZookeeperMonitorUtilsTest { @Test - public void testGetMasterLsit(){ + public void testGetMasterList(){ ZookeeperMonitor zookeeperMonitor = new ZookeeperMonitor(); diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/AlertDao.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/AlertDao.java index 19dbf46a6b..1a8c09e611 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/AlertDao.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/AlertDao.java @@ -144,9 +144,11 @@ public class AlertDao extends AbstractBaseDao { * @param taskId taskId * @param taskName taskName */ - public void sendTaskTimeoutAlert(int alertgroupId,String receivers,String receiversCc,int taskId,String taskName){ + public void sendTaskTimeoutAlert(int alertgroupId,String receivers,String receiversCc, int processInstanceId, + String processInstanceName, int taskId,String taskName){ Alert alert = new Alert(); - String content = String.format("[{'id':'%d','name':'%s','event':'timeout','warnLevel':'middle'}]",taskId,taskName); + String content = String.format("[{'process instance id':'%d','task name':'%s','task id':'%d','task name':'%s'," + + "'event':'timeout','warnLevel':'middle'}]", processInstanceId, processInstanceName, taskId, taskName); alert.setTitle("Task Timeout Warn"); alert.setShowType(ShowType.TABLE); alert.setContent(content); diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecThread.java index 66d1a3f4c2..37262c05e2 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecThread.java @@ -128,7 +128,9 @@ public class MasterTaskExecThread extends MasterBaseTaskExecThread { // process define ProcessDefinition processDefine = processService.findProcessDefineById(processInstance.getProcessDefinitionId()); // send warn mail - alertDao.sendTaskTimeoutAlert(processInstance.getWarningGroupId(),processDefine.getReceivers(),processDefine.getReceiversCc(),taskInstance.getId(),taskInstance.getName()); + alertDao.sendTaskTimeoutAlert(processInstance.getWarningGroupId(),processDefine.getReceivers(), + processDefine.getReceiversCc(), processInstance.getId(), processInstance.getName(), + taskInstance.getId(),taskInstance.getName()); checkTimeout = false; } } diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/AlertManager.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/AlertManager.java index f3441edd17..050af1eec5 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/AlertManager.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/AlertManager.java @@ -133,6 +133,8 @@ public class AlertManager { continue; } LinkedHashMap failedTaskMap = new LinkedHashMap(); + failedTaskMap.put("process instance id", String.valueOf(processInstance.getId())); + failedTaskMap.put("process instance name", processInstance.getName()); failedTaskMap.put("task id", String.valueOf(task.getId())); failedTaskMap.put("task name", task.getName()); failedTaskMap.put("task type", task.getTaskType()); @@ -193,7 +195,7 @@ public class AlertManager { logger.info("add alert to db , alert : {}", alert.toString()); }catch (Exception e){ - logger.error("send alert failed! " + e); + logger.error("send alert failed:{} ", e.getMessage()); } } diff --git a/sql/upgrade/1.2.2_schema/mysql/dolphinscheduler_ddl.sql b/sql/upgrade/1.2.2_schema/mysql/dolphinscheduler_ddl.sql index 790a9a8ada..e5785d9997 100644 --- a/sql/upgrade/1.2.2_schema/mysql/dolphinscheduler_ddl.sql +++ b/sql/upgrade/1.2.2_schema/mysql/dolphinscheduler_ddl.sql @@ -36,7 +36,7 @@ delimiter ; CALL uc_dolphin_T_t_ds_process_definition_A_modify_by; DROP PROCEDURE uc_dolphin_T_t_ds_process_definition_A_modify_by; --- uc_dolphin_T_t_ds_process_definition_A_modify_by +-- uc_dolphin_T_t_ds_task_instance_A_executor_id drop PROCEDURE if EXISTS uc_dolphin_T_t_ds_task_instance_A_executor_id; delimiter d// CREATE PROCEDURE uc_dolphin_T_t_ds_task_instance_A_executor_id() diff --git a/sql/upgrade/1.2.2_schema/postgresql/dolphinscheduler_ddl.sql b/sql/upgrade/1.2.2_schema/postgresql/dolphinscheduler_ddl.sql index cbe7c22bbe..70471cb26d 100644 --- a/sql/upgrade/1.2.2_schema/postgresql/dolphinscheduler_ddl.sql +++ b/sql/upgrade/1.2.2_schema/postgresql/dolphinscheduler_ddl.sql @@ -32,7 +32,7 @@ delimiter ; SELECT uc_dolphin_T_t_ds_process_definition_A_modify_by(); DROP FUNCTION IF EXISTS uc_dolphin_T_t_ds_process_definition_A_modify_by(); --- uc_dolphin_T_t_ds_process_definition_A_modify_by +-- uc_dolphin_T_t_ds_task_instance_A_executor_id delimiter d// CREATE OR REPLACE FUNCTION uc_dolphin_T_t_ds_task_instance_A_executor_id() RETURNS void AS $$ BEGIN From 3e5a939913527c6aaafca5f170b2460cae2de51f Mon Sep 17 00:00:00 2001 From: qiaozhanwei Date: Thu, 5 Mar 2020 11:07:44 +0800 Subject: [PATCH 057/221] delete unused import (#2066) --- .../alert/template/impl/DefaultHTMLTemplate.java | 1 - .../apache/dolphinscheduler/alert/utils/JSONUtilsTest.java | 1 - .../apache/dolphinscheduler/api/service/MonitorService.java | 1 - .../api/controller/LoginControllerTest.java | 1 - .../api/controller/MonitorControllerTest.java | 1 - .../api/controller/ProjectControllerTest.java | 2 -- .../api/controller/ResourcesControllerTest.java | 2 -- .../api/controller/TaskRecordControllerTest.java | 5 ----- .../api/controller/WorkerGroupControllerTest.java | 1 - .../dolphinscheduler/api/utils/FourLetterWordMainTest.java | 1 - .../apache/dolphinscheduler/common/log/WorkerLogFilter.java | 3 --- .../common/log/TaskLogDiscriminatorTest.java | 2 -- .../dolphinscheduler/common/log/TaskLogFilterTest.java | 1 - .../apache/dolphinscheduler/common/utils/HttpUtilsTest.java | 1 - .../dolphinscheduler/common/utils/SchemaUtilsTest.java | 3 --- .../dolphinscheduler/common/utils/StringUtilsTest.java | 2 -- .../common/utils/TaskParametersUtilsTest.java | 3 --- .../dolphinscheduler/dao/entity/ProcessDefinition.java | 1 - .../org/apache/dolphinscheduler/dao/entity/UdfFunc.java | 1 - .../dolphinscheduler/dao/upgrade/MysqlUpgradeDao.java | 1 - .../dolphinscheduler/dao/upgrade/PostgresqlUpgradeDao.java | 1 - .../dolphinscheduler/dao/mapper/ProjectUserMapperTest.java | 2 -- .../apache/dolphinscheduler/dao/mapper/QueueMapperTest.java | 1 - .../remote/command/ExecuteTaskRequestCommand.java | 2 +- .../remote/command/ExecuteTaskResponseCommand.java | 2 +- .../org/apache/dolphinscheduler/remote/command/Ping.java | 1 - .../remote/command/log/GetLogBytesRequestCommand.java | 1 - .../remote/command/log/RollViewLogRequestCommand.java | 1 - .../remote/command/log/ViewLogRequestCommand.java | 1 - .../server/worker/task/sqoop/SqoopTaskTest.java | 6 ------ .../dolphinscheduler/service/log/LogClientService.java | 1 - 31 files changed, 2 insertions(+), 51 deletions(-) diff --git a/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/template/impl/DefaultHTMLTemplate.java b/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/template/impl/DefaultHTMLTemplate.java index 428fa4cb62..79123a1e80 100644 --- a/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/template/impl/DefaultHTMLTemplate.java +++ b/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/template/impl/DefaultHTMLTemplate.java @@ -19,7 +19,6 @@ package org.apache.dolphinscheduler.alert.template.impl; import org.apache.dolphinscheduler.alert.template.AlertTemplate; import org.apache.dolphinscheduler.alert.utils.Constants; import org.apache.dolphinscheduler.alert.utils.JSONUtils; -import org.apache.dolphinscheduler.alert.utils.MailUtils; import org.apache.dolphinscheduler.common.enums.ShowType; import org.apache.dolphinscheduler.common.utils.StringUtils; import org.slf4j.Logger; diff --git a/dolphinscheduler-alert/src/test/java/org/apache/dolphinscheduler/alert/utils/JSONUtilsTest.java b/dolphinscheduler-alert/src/test/java/org/apache/dolphinscheduler/alert/utils/JSONUtilsTest.java index cb63a22d79..277c42b5bd 100644 --- a/dolphinscheduler-alert/src/test/java/org/apache/dolphinscheduler/alert/utils/JSONUtilsTest.java +++ b/dolphinscheduler-alert/src/test/java/org/apache/dolphinscheduler/alert/utils/JSONUtilsTest.java @@ -17,7 +17,6 @@ package org.apache.dolphinscheduler.alert.utils; -import org.junit.After; import org.junit.Before; import org.junit.Test; import org.slf4j.Logger; diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/MonitorService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/MonitorService.java index 118c5ce936..3370961fd4 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/MonitorService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/MonitorService.java @@ -28,7 +28,6 @@ import org.apache.dolphinscheduler.dao.entity.ZookeeperRecord; import org.springframework.beans.factory.annotation.Autowired; import org.springframework.stereotype.Service; -import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/LoginControllerTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/LoginControllerTest.java index bddc055de3..f5a28d01ae 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/LoginControllerTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/LoginControllerTest.java @@ -28,7 +28,6 @@ import org.springframework.test.web.servlet.MvcResult; import org.springframework.util.LinkedMultiValueMap; import org.springframework.util.MultiValueMap; -import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.get; import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.post; import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.content; import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.status; diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/MonitorControllerTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/MonitorControllerTest.java index 8fc055daf1..41674d3e54 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/MonitorControllerTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/MonitorControllerTest.java @@ -19,7 +19,6 @@ package org.apache.dolphinscheduler.api.controller; import org.apache.dolphinscheduler.api.enums.Status; import org.apache.dolphinscheduler.api.utils.Result; import org.apache.dolphinscheduler.common.utils.JSONUtils; -import com.alibaba.fastjson.JSONObject; import org.junit.Assert; import org.junit.Test; import org.slf4j.Logger; diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/ProjectControllerTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/ProjectControllerTest.java index bab82df59d..42cdd1705a 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/ProjectControllerTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/ProjectControllerTest.java @@ -29,8 +29,6 @@ import org.springframework.test.web.servlet.MvcResult; import org.springframework.util.LinkedMultiValueMap; import org.springframework.util.MultiValueMap; -import javax.ws.rs.POST; - import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.get; import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.post; import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.content; diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/ResourcesControllerTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/ResourcesControllerTest.java index 46d85f4d8d..40bdd5490d 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/ResourcesControllerTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/ResourcesControllerTest.java @@ -18,13 +18,11 @@ package org.apache.dolphinscheduler.api.controller; import org.apache.dolphinscheduler.api.enums.Status; import org.apache.dolphinscheduler.api.utils.Result; -import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.enums.ResourceType; import org.apache.dolphinscheduler.common.enums.UdfType; import org.apache.dolphinscheduler.common.utils.JSONUtils; import com.alibaba.fastjson.JSONObject; import org.junit.Assert; -import org.junit.Ignore; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/TaskRecordControllerTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/TaskRecordControllerTest.java index 8bddb0f905..943e14607b 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/TaskRecordControllerTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/TaskRecordControllerTest.java @@ -18,9 +18,6 @@ package org.apache.dolphinscheduler.api.controller; import org.apache.dolphinscheduler.api.enums.Status; import org.apache.dolphinscheduler.api.utils.Result; -import org.apache.dolphinscheduler.common.enums.FailureStrategy; -import org.apache.dolphinscheduler.common.enums.Priority; -import org.apache.dolphinscheduler.common.enums.WarningType; import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.junit.Assert; import org.junit.Test; @@ -31,9 +28,7 @@ import org.springframework.test.web.servlet.MvcResult; import org.springframework.util.LinkedMultiValueMap; import org.springframework.util.MultiValueMap; -import static org.junit.Assert.*; import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.get; -import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.post; import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.content; import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.status; diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/WorkerGroupControllerTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/WorkerGroupControllerTest.java index 65ecd3f759..8517ad2d45 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/WorkerGroupControllerTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/WorkerGroupControllerTest.java @@ -29,7 +29,6 @@ import org.springframework.test.web.servlet.MvcResult; import org.springframework.util.LinkedMultiValueMap; import org.springframework.util.MultiValueMap; -import static org.junit.Assert.*; import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.get; import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.post; import static org.springframework.test.web.servlet.result.MockMvcResultMatchers.content; diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/utils/FourLetterWordMainTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/utils/FourLetterWordMainTest.java index e8adc6ca9c..69d1f21c37 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/utils/FourLetterWordMainTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/utils/FourLetterWordMainTest.java @@ -19,7 +19,6 @@ package org.apache.dolphinscheduler.api.utils; import org.junit.Assert; import org.junit.Before; -import org.junit.Ignore; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.InjectMocks; diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/log/WorkerLogFilter.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/log/WorkerLogFilter.java index abcc8bc619..6240ed9a2e 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/log/WorkerLogFilter.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/log/WorkerLogFilter.java @@ -20,9 +20,6 @@ import ch.qos.logback.classic.Level; import ch.qos.logback.classic.spi.ILoggingEvent; import ch.qos.logback.core.filter.Filter; import ch.qos.logback.core.spi.FilterReply; -import org.apache.dolphinscheduler.common.utils.LoggerUtils; - -import java.util.Arrays; /** * worker log filter diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/log/TaskLogDiscriminatorTest.java b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/log/TaskLogDiscriminatorTest.java index 8745a4f6b4..ff298000f5 100644 --- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/log/TaskLogDiscriminatorTest.java +++ b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/log/TaskLogDiscriminatorTest.java @@ -27,8 +27,6 @@ import org.slf4j.Marker; import java.util.Map; -import static org.junit.Assert.*; - public class TaskLogDiscriminatorTest { /** diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/log/TaskLogFilterTest.java b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/log/TaskLogFilterTest.java index 52767074da..5cca6403c8 100644 --- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/log/TaskLogFilterTest.java +++ b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/log/TaskLogFilterTest.java @@ -21,7 +21,6 @@ import ch.qos.logback.classic.spi.ILoggingEvent; import ch.qos.logback.classic.spi.IThrowableProxy; import ch.qos.logback.classic.spi.LoggerContextVO; import ch.qos.logback.core.spi.FilterReply; -import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.utils.LoggerUtils; import org.junit.Assert; import org.junit.Test; diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/HttpUtilsTest.java b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/HttpUtilsTest.java index 20994ac99f..41049ec335 100644 --- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/HttpUtilsTest.java +++ b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/HttpUtilsTest.java @@ -19,7 +19,6 @@ package org.apache.dolphinscheduler.common.utils; import com.alibaba.fastjson.JSON; import com.alibaba.fastjson.JSONObject; import org.junit.Assert; -import org.junit.Ignore; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/SchemaUtilsTest.java b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/SchemaUtilsTest.java index 907a09e458..7885806b96 100644 --- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/SchemaUtilsTest.java +++ b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/SchemaUtilsTest.java @@ -19,15 +19,12 @@ package org.apache.dolphinscheduler.common.utils; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; -import org.mockito.Mockito; import org.powermock.api.mockito.PowerMockito; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; import org.slf4j.LoggerFactory; import java.io.File; -import java.io.FileNotFoundException; -import java.util.ArrayList; import java.util.Arrays; import java.util.List; diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/StringUtilsTest.java b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/StringUtilsTest.java index 947e7310db..eca22def30 100644 --- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/StringUtilsTest.java +++ b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/StringUtilsTest.java @@ -19,8 +19,6 @@ package org.apache.dolphinscheduler.common.utils; import org.junit.Assert; import org.junit.Test; -import java.util.ArrayList; - public class StringUtilsTest { @Test public void testIsNotEmpty() { diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/TaskParametersUtilsTest.java b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/TaskParametersUtilsTest.java index db4a86bc26..b316b17469 100644 --- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/TaskParametersUtilsTest.java +++ b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/TaskParametersUtilsTest.java @@ -19,9 +19,6 @@ package org.apache.dolphinscheduler.common.utils; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; -import org.mockito.Mock; -import org.mockito.Mockito; -import org.powermock.api.mockito.PowerMockito; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; import org.slf4j.LoggerFactory; diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessDefinition.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessDefinition.java index 6e7ea7d64f..dbb880c025 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessDefinition.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessDefinition.java @@ -29,7 +29,6 @@ import com.baomidou.mybatisplus.core.toolkit.StringUtils; import java.util.Date; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.stream.Collectors; diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/UdfFunc.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/UdfFunc.java index 9093034ae8..3518676337 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/UdfFunc.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/UdfFunc.java @@ -16,7 +16,6 @@ */ package org.apache.dolphinscheduler.dao.entity; -import com.baomidou.mybatisplus.annotation.TableField; import org.apache.dolphinscheduler.common.enums.UdfType; import com.baomidou.mybatisplus.annotation.IdType; import com.baomidou.mybatisplus.annotation.TableId; diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/upgrade/MysqlUpgradeDao.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/upgrade/MysqlUpgradeDao.java index 5e5277b997..58bd673fc5 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/upgrade/MysqlUpgradeDao.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/upgrade/MysqlUpgradeDao.java @@ -17,7 +17,6 @@ package org.apache.dolphinscheduler.dao.upgrade; import org.apache.dolphinscheduler.common.utils.ConnectionUtils; -import org.apache.dolphinscheduler.dao.datasource.ConnectionFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/upgrade/PostgresqlUpgradeDao.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/upgrade/PostgresqlUpgradeDao.java index c0dc905848..5db273642a 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/upgrade/PostgresqlUpgradeDao.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/upgrade/PostgresqlUpgradeDao.java @@ -17,7 +17,6 @@ package org.apache.dolphinscheduler.dao.upgrade; import org.apache.dolphinscheduler.common.utils.ConnectionUtils; -import org.apache.dolphinscheduler.dao.datasource.ConnectionFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/ProjectUserMapperTest.java b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/ProjectUserMapperTest.java index 102d8cd5db..6e995ef3c7 100644 --- a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/ProjectUserMapperTest.java +++ b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/ProjectUserMapperTest.java @@ -17,8 +17,6 @@ package org.apache.dolphinscheduler.dao.mapper; -import org.apache.dolphinscheduler.common.enums.ExecutionStatus; -import org.apache.dolphinscheduler.dao.entity.ProjectUser; import org.apache.dolphinscheduler.dao.entity.ProjectUser; import org.junit.Assert; import org.junit.Test; diff --git a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/QueueMapperTest.java b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/QueueMapperTest.java index 62b034a081..30d2be03e0 100644 --- a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/QueueMapperTest.java +++ b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/QueueMapperTest.java @@ -17,7 +17,6 @@ package org.apache.dolphinscheduler.dao.mapper; -import org.apache.dolphinscheduler.dao.entity.Queue; import org.apache.dolphinscheduler.dao.entity.Queue; import com.baomidou.mybatisplus.core.metadata.IPage; import com.baomidou.mybatisplus.extension.plugins.pagination.Page; diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskRequestCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskRequestCommand.java index beec055403..93c536c347 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskRequestCommand.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskRequestCommand.java @@ -1 +1 @@ -/* * 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 shardItems; public List getShardItems() { return shardItems; } public void setShardItems(List 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 +/* * 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; /** * 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 shardItems; public List getShardItems() { return shardItems; } public void setShardItems(List 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 diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskResponseCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskResponseCommand.java index 7e35fa6e75..7543fc3d0e 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskResponseCommand.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/ExecuteTaskResponseCommand.java @@ -1 +1 @@ -/* * 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.concurrent.atomic.AtomicLong; /** * execute task response command */ public class ExecuteTaskResponseCommand implements Serializable { /** * task id */ private String taskId; /** * attempt id */ private String attemptId; /** * return result */ private Object result; /** * received time */ private long receivedTime; /** * execute count */ private int executeCount; /** * execute time */ private long executeTime; public String getAttemptId() { return attemptId; } public void setAttemptId(String attemptId) { this.attemptId = attemptId; } public String getTaskId() { return taskId; } public void setTaskId(String taskId) { this.taskId = taskId; } public Object getResult() { return result; } public void setResult(Object result) { this.result = result; } public long getReceivedTime() { return receivedTime; } public void setReceivedTime(long receivedTime) { this.receivedTime = receivedTime; } public int getExecuteCount() { return executeCount; } public void setExecuteCount(int executeCount) { this.executeCount = executeCount; } public long getExecuteTime() { return executeTime; } public void setExecuteTime(long executeTime) { this.executeTime = executeTime; } public Command convert2Command(long opaque){ Command command = new Command(); command.setType(CommandType.EXECUTE_TASK_RESPONSE); 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; /** * execute task response command */ public class ExecuteTaskResponseCommand implements Serializable { /** * task id */ private String taskId; /** * attempt id */ private String attemptId; /** * return result */ private Object result; /** * received time */ private long receivedTime; /** * execute count */ private int executeCount; /** * execute time */ private long executeTime; public String getAttemptId() { return attemptId; } public void setAttemptId(String attemptId) { this.attemptId = attemptId; } public String getTaskId() { return taskId; } public void setTaskId(String taskId) { this.taskId = taskId; } public Object getResult() { return result; } public void setResult(Object result) { this.result = result; } public long getReceivedTime() { return receivedTime; } public void setReceivedTime(long receivedTime) { this.receivedTime = receivedTime; } public int getExecuteCount() { return executeCount; } public void setExecuteCount(int executeCount) { this.executeCount = executeCount; } public long getExecuteTime() { return executeTime; } public void setExecuteTime(long executeTime) { this.executeTime = executeTime; } public Command convert2Command(long opaque){ Command command = new Command(); command.setType(CommandType.EXECUTE_TASK_RESPONSE); byte[] body = FastJsonSerializer.serialize(this); command.setBody(body); return command; } } \ No newline at end of file diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Ping.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Ping.java index c50413e98a..c5e4d075af 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Ping.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Ping.java @@ -21,7 +21,6 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import java.io.Serializable; -import java.util.concurrent.atomic.AtomicLong; /** * ping machine diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesRequestCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesRequestCommand.java index 4cc32ed42a..20cf8d9102 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesRequestCommand.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/GetLogBytesRequestCommand.java @@ -22,7 +22,6 @@ import org.apache.dolphinscheduler.remote.command.CommandType; import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer; import java.io.Serializable; -import java.util.concurrent.atomic.AtomicLong; /** * get log bytes request command diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogRequestCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogRequestCommand.java index 621d35a804..433c4a0b1f 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogRequestCommand.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/RollViewLogRequestCommand.java @@ -22,7 +22,6 @@ import org.apache.dolphinscheduler.remote.command.CommandType; import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer; import java.io.Serializable; -import java.util.concurrent.atomic.AtomicLong; /** * roll view log request command diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogRequestCommand.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogRequestCommand.java index 8835348ee3..b4773d0c0d 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogRequestCommand.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/log/ViewLogRequestCommand.java @@ -22,7 +22,6 @@ import org.apache.dolphinscheduler.remote.command.CommandType; import org.apache.dolphinscheduler.remote.utils.FastJsonSerializer; import java.io.Serializable; -import java.util.concurrent.atomic.AtomicLong; /** * view log request command diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/sqoop/SqoopTaskTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/sqoop/SqoopTaskTest.java index f8688e7c0c..511102e4b5 100644 --- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/sqoop/SqoopTaskTest.java +++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/sqoop/SqoopTaskTest.java @@ -19,12 +19,6 @@ package org.apache.dolphinscheduler.server.worker.task.sqoop; import com.alibaba.fastjson.JSONObject; import org.apache.dolphinscheduler.common.enums.DbType; import org.apache.dolphinscheduler.common.task.sqoop.SqoopParameters; -import org.apache.dolphinscheduler.common.task.sqoop.sources.SourceHdfsParameter; -import org.apache.dolphinscheduler.common.task.sqoop.sources.SourceHiveParameter; -import org.apache.dolphinscheduler.common.task.sqoop.sources.SourceMysqlParameter; -import org.apache.dolphinscheduler.common.task.sqoop.targets.TargetHdfsParameter; -import org.apache.dolphinscheduler.common.task.sqoop.targets.TargetHiveParameter; -import org.apache.dolphinscheduler.common.task.sqoop.targets.TargetMysqlParameter; import org.apache.dolphinscheduler.dao.entity.DataSource; import org.apache.dolphinscheduler.server.worker.task.TaskProps; import org.apache.dolphinscheduler.server.worker.task.sqoop.generator.SqoopJobGenerator; diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java index 5daf535625..01a49910df 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java @@ -16,7 +16,6 @@ */ package org.apache.dolphinscheduler.service.log; -import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.remote.NettyRemotingClient; import org.apache.dolphinscheduler.remote.command.Command; import org.apache.dolphinscheduler.remote.command.log.*; From d5c795bd45b490720ead48a5fe68430d4402fd5f Mon Sep 17 00:00:00 2001 From: qiaozhanwei Date: Thu, 5 Mar 2020 11:10:48 +0800 Subject: [PATCH 058/221] fix ugly logger usage (#2063) * fix ugly logger usage * remove space --- .../api/controller/DataAnalysisController.java | 4 ++-- .../dolphinscheduler/api/controller/ExecutorController.java | 2 +- .../apache/dolphinscheduler/remote/NettyRemotingServer.java | 2 +- .../dolphinscheduler/remote/handler/NettyClientHandler.java | 2 +- .../dolphinscheduler/remote/handler/NettyServerHandler.java | 2 +- .../server/master/runner/MasterExecThread.java | 2 +- .../dolphinscheduler/service/process/ProcessService.java | 6 +++--- 7 files changed, 10 insertions(+), 10 deletions(-) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/DataAnalysisController.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/DataAnalysisController.java index 1b1dc65e01..f93e7d6944 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/DataAnalysisController.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/DataAnalysisController.java @@ -103,7 +103,7 @@ public class DataAnalysisController extends BaseController{ @RequestParam(value="endDate", required=false) String endDate, @RequestParam(value="projectId", required=false, defaultValue = "0") int projectId){ try{ - logger.info("count process instance state, user:{}, start date: {}, end date:{}, project id", + logger.info("count process instance state, user:{}, start date: {}, end date:{}, project id:{}", loginUser.getUserName(), startDate, endDate, projectId); Map result = dataAnalysisService.countProcessInstanceStateByProject(loginUser, projectId, startDate, endDate); return returnDataList(result); @@ -129,7 +129,7 @@ public class DataAnalysisController extends BaseController{ public Result countDefinitionByUser(@ApiIgnore @RequestAttribute(value = Constants.SESSION_USER) User loginUser, @RequestParam(value="projectId", required=false, defaultValue = "0") int projectId){ try{ - logger.info("count process definition , user:{}, project id", + logger.info("count process definition , user:{}, project id:{}", loginUser.getUserName(), projectId); Map result = dataAnalysisService.countDefinitionByUser(loginUser, projectId); return returnDataList(result); diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ExecutorController.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ExecutorController.java index 93b095ddf8..cae4993942 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ExecutorController.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ExecutorController.java @@ -173,7 +173,7 @@ public class ExecutorController extends BaseController { @ResponseStatus(HttpStatus.OK) public Result startCheckProcessDefinition(@ApiIgnore @RequestAttribute(value = Constants.SESSION_USER) User loginUser, @RequestParam(value = "processDefinitionId") int processDefinitionId) { - logger.info("login user {}, check process definition", loginUser.getUserName(), processDefinitionId); + logger.info("login user {}, check process definition {}", loginUser.getUserName(), processDefinitionId); try { Map result = execService.startCheckByProcessDefinedId(processDefinitionId); return returnDataList(result); diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingServer.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingServer.java index 29b2317633..dbeb318f2d 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingServer.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/NettyRemotingServer.java @@ -145,7 +145,7 @@ public class NettyRemotingServer { try { future = serverBootstrap.bind(serverConfig.getListenPort()).sync(); } catch (Exception e) { - logger.error("NettyRemotingServer bind fail {}, exit", e); + logger.error("NettyRemotingServer bind fail {}, exit",e.getMessage(), e); throw new RuntimeException(String.format("NettyRemotingServer bind %s fail", serverConfig.getListenPort())); } if (future.isSuccess()) { diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java index d5d0d4df83..80e561d05c 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java @@ -107,7 +107,7 @@ public class NettyClientHandler extends ChannelInboundHandlerAdapter { */ @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { - logger.error("exceptionCaught : {}", cause); + logger.error("exceptionCaught : {}",cause.getMessage(), cause); nettyRemotingClient.closeChannel(ChannelUtils.toAddress(ctx.channel())); ctx.channel().close(); } diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyServerHandler.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyServerHandler.java index eabd6560de..22251c35c1 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyServerHandler.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyServerHandler.java @@ -140,7 +140,7 @@ public class NettyServerHandler extends ChannelInboundHandlerAdapter { */ @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { - logger.error("exceptionCaught : {}", cause); + logger.error("exceptionCaught : {}",cause.getMessage(), cause); ctx.channel().close(); } diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java index 4b22b27bec..0b81a30191 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java @@ -565,7 +565,7 @@ public class MasterExecThread implements Runnable { TaskInstance taskInstance = completeTaskList.get(nodeName); if(taskInstance == null){ - logger.error("task instance cannot find, please check it!", nodeName); + logger.error("task instance {} cannot find, please check it!", nodeName); return conditionTaskList; } diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java index b589cd4295..8bcd64f1fd 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java @@ -116,12 +116,12 @@ public class ProcessService { ProcessInstance processInstance = constructProcessInstance(command, host); //cannot construct process instance, return null; if(processInstance == null){ - logger.error("scan command, command parameter is error: %s", command.toString()); + logger.error("scan command, command parameter is error: {}", command); moveToErrorCommand(command, "process instance is null"); return null; } if(!checkThreadNum(command, validThreadNum)){ - logger.info("there is not enough thread for this command: {}",command.toString() ); + logger.info("there is not enough thread for this command: {}", command); return setWaitingThreadProcess(command, processInstance); } processInstance.setCommandType(command.getCommandType()); @@ -991,7 +991,7 @@ public class ProcessService { return insertQueueResult; }catch (Exception e){ logger.error("submit task to queue Exception: ", e); - logger.error("task queue error : %s", JSONUtils.toJson(taskInstance)); + logger.error("task queue error : {}", JSONUtils.toJson(taskInstance)); return false; } } From 18d7ff274b8757b2e37f42790473ca472cf66274 Mon Sep 17 00:00:00 2001 From: "xiaochun.liu" Date: Thu, 5 Mar 2020 11:33:24 +0800 Subject: [PATCH 059/221] action push delete --- .github/workflows/ci_e2e.yml | 2 +- .github/workflows/ci_ut.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/ci_e2e.yml b/.github/workflows/ci_e2e.yml index 5282681fb6..7d9c5e3e72 100644 --- a/.github/workflows/ci_e2e.yml +++ b/.github/workflows/ci_e2e.yml @@ -15,7 +15,7 @@ # limitations under the License. # -on: ["push", "pull_request"] +on: ["pull_request"] env: DOCKER_DIR: ./docker LOG_DIR: /tmp/dolphinscheduler diff --git a/.github/workflows/ci_ut.yml b/.github/workflows/ci_ut.yml index 45ff3ca29d..e7e47437d4 100644 --- a/.github/workflows/ci_ut.yml +++ b/.github/workflows/ci_ut.yml @@ -15,7 +15,7 @@ # limitations under the License. # -on: ["push", "pull_request"] +on: ["pull_request"] env: DOCKER_DIR: ./docker LOG_DIR: /tmp/dolphinscheduler From 6ec85540bf6d367fa11ede093325c199afc930b5 Mon Sep 17 00:00:00 2001 From: dailidong Date: Thu, 5 Mar 2020 16:36:20 +0800 Subject: [PATCH 060/221] fix typo (#2093) --- .../apache/dolphinscheduler/server/worker/task/TaskProps.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/TaskProps.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/TaskProps.java index 8e5644ed9c..edec419384 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/TaskProps.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/TaskProps.java @@ -78,7 +78,7 @@ public class TaskProps { private Map definedParams; /** - * task path + * task app id */ private String taskAppId; From 4230d5a737b7889254806969ae153817fd19d181 Mon Sep 17 00:00:00 2001 From: dailidong Date: Thu, 5 Mar 2020 17:32:40 +0800 Subject: [PATCH 061/221] add embedded database h2 and update embedded zookeeper (#2082) add embedded database h2 and update embedded zookeeper --- dolphinscheduler-dao/pom.xml | 19 +++- .../src/main/resources/application.properties | 4 + .../server/zk/StandaloneZKServerForTest.java | 100 ------------------ dolphinscheduler-service/pom.xml | 5 + .../service/zk}/ZKServer.java | 87 ++++++++------- .../src/main/resources/quartz.properties | 4 + .../service/zk/ZKServerTest.java | 41 +++++++ .../test/java/queue/BaseTaskQueueTest.java | 1 + pom.xml | 14 ++- sql/h2.mv.db | Bin 0 -> 110592 bytes 10 files changed, 121 insertions(+), 154 deletions(-) delete mode 100644 dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/zk/StandaloneZKServerForTest.java rename dolphinscheduler-service/src/{test/java/queue => main/java/org/apache/dolphinscheduler/service/zk}/ZKServer.java (67%) create mode 100644 dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/zk/ZKServerTest.java create mode 100644 sql/h2.mv.db diff --git a/dolphinscheduler-dao/pom.xml b/dolphinscheduler-dao/pom.xml index 20d19410e2..fd43458f87 100644 --- a/dolphinscheduler-dao/pom.xml +++ b/dolphinscheduler-dao/pom.xml @@ -44,6 +44,12 @@ com.baomidou mybatis-plus-boot-starter ${mybatis-plus.version} + + + org.apache.logging.log4j + log4j-to-slf4j + + org.postgresql @@ -71,6 +77,14 @@ log4j-api org.apache.logging.log4j + + org.springframework.boot + spring-boot-starter-tomcat + + + org.apache.logging.log4j + log4j-to-slf4j + @@ -78,7 +92,10 @@ mysql mysql-connector-java - + + com.h2database + h2 + com.alibaba druid diff --git a/dolphinscheduler-dao/src/main/resources/application.properties b/dolphinscheduler-dao/src/main/resources/application.properties index 34bb9f916b..06b0ee94d5 100644 --- a/dolphinscheduler-dao/src/main/resources/application.properties +++ b/dolphinscheduler-dao/src/main/resources/application.properties @@ -23,6 +23,10 @@ spring.datasource.url=jdbc:postgresql://localhost:5432/dolphinscheduler # mysql #spring.datasource.driver-class-name=com.mysql.jdbc.Driver #spring.datasource.url=jdbc:mysql://192.168.xx.xx:3306/dolphinscheduler?useUnicode=true&characterEncoding=UTF-8 +# h2 +#spring.datasource.driver-class-name=org.h2.Driver +#spring.datasource.url=jdbc:h2:file:../sql/h2;AUTO_SERVER=TRUE + spring.datasource.username=test spring.datasource.password=test diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/zk/StandaloneZKServerForTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/zk/StandaloneZKServerForTest.java deleted file mode 100644 index 679862f102..0000000000 --- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/zk/StandaloneZKServerForTest.java +++ /dev/null @@ -1,100 +0,0 @@ -/* - * 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.server.zk; - -import org.apache.dolphinscheduler.common.thread.ThreadPoolExecutors; -import org.apache.zookeeper.server.ServerConfig; -import org.apache.zookeeper.server.ZooKeeperServerMain; -import org.apache.zookeeper.server.quorum.QuorumPeerConfig; -import org.junit.Before; -import org.junit.Ignore; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.util.Properties; - - -/** - * just for test - */ -@Ignore -public class StandaloneZKServerForTest { - - private static final Logger logger = LoggerFactory.getLogger(StandaloneZKServerForTest.class); - - private static volatile ZooKeeperServerMain zkServer = null; - - - @Before - public void before() { - logger.info("standalone zookeeper server for test service start "); - - ThreadPoolExecutors.getInstance().execute(new Runnable() { - @Override - public void run() { - - //delete zk data dir ? - File zkFile = new File(System.getProperty("java.io.tmpdir"), "zookeeper"); - - startStandaloneServer("2000", zkFile.getAbsolutePath(), "2181", "10", "5"); - } - }); - - } - - - /** - * start zk server - * @param tickTime zookeeper ticktime - * @param dataDir zookeeper data dir - * @param clientPort zookeeper client port - * @param initLimit zookeeper init limit - * @param syncLimit zookeeper sync limit - */ - private void startStandaloneServer(String tickTime, String dataDir, String clientPort, String initLimit, String syncLimit) { - Properties props = new Properties(); - props.setProperty("tickTime", tickTime); - props.setProperty("dataDir", dataDir); - props.setProperty("clientPort", clientPort); - props.setProperty("initLimit", initLimit); - props.setProperty("syncLimit", syncLimit); - - QuorumPeerConfig quorumConfig = new QuorumPeerConfig(); - try { - quorumConfig.parseProperties(props); - - if(zkServer == null ){ - - synchronized (StandaloneZKServerForTest.class){ - if(zkServer == null ){ - zkServer = new ZooKeeperServerMain(); - final ServerConfig config = new ServerConfig(); - config.readFrom(quorumConfig); - zkServer.runFromConfig(config); - } - } - - } - - } catch (Exception e) { - logger.error("start standalone server fail!", e); - } - } - - -} \ No newline at end of file diff --git a/dolphinscheduler-service/pom.xml b/dolphinscheduler-service/pom.xml index 7d775d5497..03bb94dc29 100644 --- a/dolphinscheduler-service/pom.xml +++ b/dolphinscheduler-service/pom.xml @@ -48,6 +48,11 @@ + + org.apache.logging.log4j + log4j-core + + org.quartz-scheduler quartz-jobs diff --git a/dolphinscheduler-service/src/test/java/queue/ZKServer.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/ZKServer.java similarity index 67% rename from dolphinscheduler-service/src/test/java/queue/ZKServer.java rename to dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/ZKServer.java index 3d503d5691..96331405d4 100644 --- a/dolphinscheduler-service/src/test/java/queue/ZKServer.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/ZKServer.java @@ -14,11 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package queue; +package org.apache.dolphinscheduler.service.zk; -import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.framework.CuratorFrameworkFactory; -import org.apache.curator.retry.ExponentialBackoffRetry; +import org.apache.zookeeper.server.ZooKeeperServer; import org.apache.zookeeper.server.ZooKeeperServerMain; import org.apache.zookeeper.server.quorum.QuorumPeerConfig; import org.slf4j.Logger; @@ -26,27 +24,45 @@ import org.slf4j.LoggerFactory; import java.io.File; import java.io.IOException; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; /** - * just for test + * just speed experience version + * embedded zookeeper service */ public class ZKServer { - private static final Logger logger = LoggerFactory.getLogger(ZKServer.class); private static volatile PublicZooKeeperServerMain zkServer = null; public static final int DEFAULT_ZK_TEST_PORT = 2181; - public static final String DEFAULT_ZK_STR = "localhost:" + DEFAULT_ZK_TEST_PORT; - private static String dataDir = null; private static final AtomicBoolean isStarted = new AtomicBoolean(false); + public static void main(String[] args) { + if(!isStarted()){ + ZKServer.start(); + + /** + * register hooks, which are called before the process exits + */ + Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() { + @Override + public void run() { + stop(); + } + })); + }else{ + logger.info("zk server aleady started"); + } + } + + /** + * start service + */ public static void start() { try { startLocalZkServer(DEFAULT_ZK_TEST_PORT); @@ -79,7 +95,8 @@ public class ZKServer { * @param port The port to listen on */ public static void startLocalZkServer(final int port) { - startLocalZkServer(port, org.apache.commons.io.FileUtils.getTempDirectoryPath() + File.separator + "test-" + System.currentTimeMillis()); + + startLocalZkServer(port, System.getProperty("user.dir") +"/zookeeper_data", ZooKeeperServer.DEFAULT_TICK_TIME,"20"); } /** @@ -87,48 +104,28 @@ public class ZKServer { * * @param port The port to listen on * @param dataDirPath The path for the Zk data directory + * @param tickTime zk tick time + * @param maxClientCnxns zk max client connections */ - private static synchronized void startLocalZkServer(final int port, final String dataDirPath) { + private static synchronized void startLocalZkServer(final int port, final String dataDirPath,final int tickTime,String maxClientCnxns) { if (zkServer != null) { throw new RuntimeException("Zookeeper server is already started!"); } - try { - zkServer = new PublicZooKeeperServerMain(); - logger.info("Zookeeper data path : {} ", dataDirPath); - dataDir = dataDirPath; - final String[] args = new String[]{Integer.toString(port), dataDirPath}; - Thread init = new Thread(new Runnable() { - @Override - public void run() { - try { - zkServer.initializeAndRun(args); - } catch (QuorumPeerConfig.ConfigException e) { - logger.warn("Caught exception while starting ZK", e); - } catch (IOException e) { - logger.warn("Caught exception while starting ZK", e); - } - } - }, "init-zk-thread"); - init.start(); - } catch (Exception e) { - logger.warn("Caught exception while starting ZK", e); - throw new RuntimeException(e); - } - - CuratorFramework zkClient = CuratorFrameworkFactory.builder() - .connectString(DEFAULT_ZK_STR) - .retryPolicy(new ExponentialBackoffRetry(10,100)) - .sessionTimeoutMs(1000 * 30) - .connectionTimeoutMs(1000 * 30) - .build(); + zkServer = new PublicZooKeeperServerMain(); + logger.info("Zookeeper data path : {} ", dataDirPath); + dataDir = dataDirPath; + final String[] args = new String[]{Integer.toString(port), dataDirPath, Integer.toString(tickTime), maxClientCnxns}; try { - zkClient.blockUntilConnected(10, TimeUnit.SECONDS); - zkClient.close(); - } catch (InterruptedException ignore) { + logger.info("Zookeeper server started "); + isStarted.compareAndSet(false, true); + + zkServer.initializeAndRun(args); + } catch (QuorumPeerConfig.ConfigException e) { + logger.warn("Caught exception while starting ZK", e); + } catch (IOException e) { + logger.warn("Caught exception while starting ZK", e); } - isStarted.compareAndSet(false, true); - logger.info("zk server started"); } /** diff --git a/dolphinscheduler-service/src/main/resources/quartz.properties b/dolphinscheduler-service/src/main/resources/quartz.properties index 9c8930b647..b01be394c6 100644 --- a/dolphinscheduler-service/src/main/resources/quartz.properties +++ b/dolphinscheduler-service/src/main/resources/quartz.properties @@ -26,6 +26,10 @@ org.quartz.dataSource.myDs.URL = jdbc:postgresql://localhost:5432/dolphinschedul # mysql #org.quartz.dataSource.myDs.driver = com.mysql.jdbc.Driver #org.quartz.dataSource.myDs.URL = jdbc:mysql://localhost:3306/dolphinscheduler?characterEncoding=utf8 +#h2 +#org.quartz.dataSource.myDs.driver=org.h2.Driver +#org.quartz.dataSource.myDs.URL=jdbc:h2:file:/Users/stone/work/myworkspace/incubator-dolphinscheduler/h2;AUTO_SERVER=TRUE + org.quartz.dataSource.myDs.user = test org.quartz.dataSource.myDs.password = test diff --git a/dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/zk/ZKServerTest.java b/dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/zk/ZKServerTest.java new file mode 100644 index 0000000000..48cde32287 --- /dev/null +++ b/dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/zk/ZKServerTest.java @@ -0,0 +1,41 @@ +/* + * 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.service.zk; + +import org.junit.Ignore; +import org.junit.Test; + +import static org.junit.Assert.*; + +@Ignore +public class ZKServerTest { + + @Test + public void start() { + //ZKServer is a process, can't unit test + } + + @Test + public void isStarted() { + + } + + @Test + public void stop() { + ZKServer.stop(); + } +} \ No newline at end of file diff --git a/dolphinscheduler-service/src/test/java/queue/BaseTaskQueueTest.java b/dolphinscheduler-service/src/test/java/queue/BaseTaskQueueTest.java index a0cc457e22..97ab9969a3 100644 --- a/dolphinscheduler-service/src/test/java/queue/BaseTaskQueueTest.java +++ b/dolphinscheduler-service/src/test/java/queue/BaseTaskQueueTest.java @@ -18,6 +18,7 @@ package queue; import org.apache.dolphinscheduler.service.queue.ITaskQueue; import org.apache.dolphinscheduler.service.queue.TaskQueueFactory; +import org.apache.dolphinscheduler.service.zk.ZKServer; import org.junit.*; /** diff --git a/pom.xml b/pom.xml index 8a1e9b98f2..57063a86c2 100644 --- a/pom.xml +++ b/pom.xml @@ -72,7 +72,7 @@ 5.0.5 1.2.61 1.1.14 - 1.3.163 + 1.4.200 1.6 1.1.1 4.4.1 @@ -200,13 +200,6 @@ ${spring.version} test - - com.h2database - h2 - ${h2.version} - test - - org.apache.dolphinscheduler dolphinscheduler-server @@ -349,6 +342,11 @@ ${mysql.connector.version} test + + com.h2database + h2 + ${h2.version} + org.slf4j slf4j-api diff --git a/sql/h2.mv.db b/sql/h2.mv.db new file mode 100644 index 0000000000000000000000000000000000000000..42ea540bd37b54496241f1669d3455f3b9806cf4 GIT binary patch literal 110592 zcmeIbdz2hkeJ9>snrr-$(ZmKDzvyXO9?2e!x*q)+4E8*Fq?YETo*7xPWZFI5)zaY6 zj64rpLMX^@gTcl;Y-|+A7#r9uXE~gcO*Y|`B{^9qyGb}Wu+V7&#_TVTBf)>~k`1=d?&y#>}=V7&#_TVTBf)?45g)&kBX zza^Fd7<~&io$5bQJ35dr;BjmqSI_h<+Q(M0MBr%cki7zz!Sd45>VQ=*Vx0i~nVHO7 z-|GAkdmvXZjYdB2T?=6K9i3ksURqmR9mu54^*_3-;E_w~iSv?)u0O&1o^K|)UOtuR zUHd3|ZrQV!J-0shb@sgRiVAa)$NH@z|5~lhE!h2f;Sv_Mw%Ax2TRPNlBobYBy!jo8%M%x}2JOch2*Z6fS$;ap zT+i@__%isH$sfFYJJ>GS1KSEzI9zkAv;?92SAlk15Xdx)&4mIzS=NU`sd@3IKzVZ8TLl=zVL8Yb$nR zaCNV{M8xo7#PDF`1(!0hBxOLa2~bMXymV!}WKpR4Fj}YwVl*d+av&Q(qv@p);CFER zCd&PRrK9%Z%yMmUrB+|X!k3k6*slU1STw^H%g8F5Jd~qJ&jp7{bOncsw8h|1p4O0y zbOOsYjL5WBa40XAE&~dxktjDG94gWlfkZ77q$$GJ+Q6xy@{aPvOlo2h|DPQj>${Nl zQ|Z#gNOh(>UP_B_@$>ND7(xas)rmnGHoSANGB`X_s-*S~R`!=Cc5E>+nXMx3czJrX zTq#w{6Vo%5bZUBhaBK`EiO^k>L)DSNnL!k=f2K5e;nwc1-c--Vdxk5e!I@HOs!|?D zmZ{yP{i*cS?9fU)l8=*Cp=+$Dl@lNXDa0#J4%)5^p^DW@Xpc*r?W4W z76C2aJ1Ud2Q)v{wCAE3uOUc18TA7)_p|MhE)~Ugfk<{=ct<@lHIhQ>5M@pTi3LR3D zDo>1*_IWZ^>)qY8HI>@Cg+S2yDk0EHcX#yxfuE0qz~Rc|gyaFl@->e?MsI1~RHZaM zU7nl>(s6I(n?Gm+R;a zvtcx?%-p%Ppt+68m+Z@J0M!1U^BPsm)?*F;eNI|ej?GuJ0&Tvc*L2EyOs8{9KWzT; zrJvhCsQrIyA~Ez%_9P3Z+4F)6zQvv!f5e}c&hzKx-~aX%^xt;!7NV0fsgdJ}#AW~H zyoWYik$FxX$+{!qpS-r?JG~uOsCS*81+qKsFmui)fz_8 zwzEYuZ{>>lMsI58;PlQ^ZzfYKwMVF(@P z4S@ekBnJFl!|63YLUY<8pmPf{LtMtiOb}nlCQAEes?5j@u$3`{+#3;Co~)E-0Cse- zC77(DGnIDM^b7HwGE>0~%B({uzl$E^RAv;R|R$3Xzk zL4boGm8Ld`<3bPyi5MxlLDcQQZ42sR>)HgH;JvYJ0u%1K)_DY~aaBK(&oYp#mezs% zb}WRhZkPz4k1`Qj%7{0x0KE~0f}|sau1JFcO@Ze7HJA+Fm)3jK+``ZpEFw|fS=h`9 z#jmDjXY~4}oVqUU$RRrv=#cd_M8`>B--VC-Mj;$kxMuT*OEC3brv0R35dZ|vj94!8LlroS> zrql7*3yLDxD{~VBL`pp>2Li7EJD-ok&Z$9o^e{KfCtBeVnK}!`y1CF<*mlBy;nqar zlC|@{3r8hlWuhxk&|H^jGqM&SJ!2m=?puk5-Bnt8r1XlqB|257u3w3!o7&r_5 z!+luUA2MTB%(l2KWQAV!F&SO5FgP(>>PX>;M)53~aCD9svhZnludB!(1Kd>ILjDMy z_=Vaz$^=zy{bM2?e8a(xUlAVe$Ym<`YKY&V*eW+ry{rWYE0i=3sL)V9DU0$GdotQR5|OrP* zO%Crq3yn*YhiWy;GaE2tFF7qdixd=gMwD*g)&JN#Zyu&+=QYYmI^4Mwt zOOvpWrSpnFH+M*R(2qjp0@$cR zM`uv!ju0$coHsC9JfQULDE&l?CP(RFX>Ug)sLO$aNyvbNC*pvRwZN-FMk^Wo76~YbZ{1{*QgG~^7AddtJTIzwN|g&D=XF2rMv9K>U^WRV&A=HFV^ke7B>c6 z633h#N@nif3tlMQQbAX5Zpm3&-6GvsSffmLS8$_@{)KKIp9+e}5gaF`Sg#e8C$kHR z;95lphG(?xSu!DhWji(WtWQJ(S~!iyrFa;CDR6&?DbSiMo<)G-Jw$F{MMaPPvvm0X zHeluDHcJONNwCgmVuEDBUS5qx$4q&KuD)x-{pp>Pw^jLqpAZybh29KXo;L%S3KtD_ z@6dHWY@JX@G9HDF6K3Iy_D1T>^onf8CLxXfY?+Wn!5ZuPN@XnYL$;o@&ac&B49YzL z=N`2;1GT$iqqZ0bTFz>T*0%$#$Ak9_d5{qcr6{g5kQOOM-vNyNaV&jC3FmrHc#zs? zGYZs0%S&rVqtVK_&Kj zXzhbZch>I$J>`iqUFIC6QDl}=*o8Q*D36)J>D_R;OtEE*QR;3K3%+O3jnd70Scl2i zV>;k4##_p(vO5TtP|wlrc-fno=! zFqV=y3`sl@p|K_IV@V;3>`Vv;sSJ)l27eSsXVbL`)049ma4VM1Zq!z5D@$w3bvs%T zRc%d&m%h!v_*4LbCC35k+*Rbhq8D1k-3cweCq23sAJ^&JcwZfYc5^>gR1HZkarkDivr5334B*e61_BtIOvA z5X8|sQa`LKb-BAfwo;FQaG`{~s@*e+mTi9yb+afC2IutP4gi*x#>8wV#4Ns5SO}6b zA3m9&B@%B=7|j1xCcDUPGDgP91eqjLWDlv3X);4*$zF0B*+=%1+sOfPklaDuNvh;d z>VGRkMDL3ld8sAFl2TCKnUEH!jP8Pr{xpu-AIF?4o$kdvYN--W^B<|LtlGa2z;H?C6yl}8Hfp%+Mkn2fMmD%R z2~qh0I)_b4?<`?lG(?XIXzQ)m-4QPR>p_{`^zlS?0AuaYrKXF8F8}&Y>KKXiRk6=FERgC!*0$m&eJo0qH(4UGQtL#)E&U~GI|{i9;{VL*n&7s zhv#fNR7fn;X&3MqWhy9Rk^oG#HcNHaG~NF1(WQ!U>iK;K7Qzl84YYm8x*l!#nps`13?q}rVDy#GWtL~Le;YjX)cDF&@QM615<)uq!sSV4-xG}cy5 z&koU@F4X;{;=^t17Guk2v31a+?8He77{67A$1&dU^kuFYQB}NDYJb$y5>R+&>G2T_ zaSl2pW{XsyP4~v`J_ZAwNB&y6oyRmfkB8%Q9ug|?R`pXdI_o-+0$s-pDl1Px2+*=`gXr9XHxhgv)0G!Cw<=}fQ?s9?3@`14wlG3mJtd>xN1 zJ-M*K7iWkZZtR25oQ!b2N)E;vWzjM_6BVR@sO(40c84HVa3>P3qDqpHbR>B@Ao^q+ zh~k`+*_f_eEFD;DGhkx5i3KoTnKZ?U|7_ZF0W9Q#+M}Cz8C!UU4 zwijy5%=WnTaguNw_l^W68)5o}2d+0dYS27Ys$ee7J3w{EFy|g3?*v5dh^=V?5fa|a z5;RqS<^xgPb!Zl`bWz7RVlcWBF#1$SFlrr-@UwdS=DE5q$-$BI;KZ}XU^rm)h#ZkZ zC!%AfJEB?&bEh>$5;pf5E54ZQjf_h-{dNCja+$8btdTiVCkzkau-=3 zN5~>sB1g&HWSOjxRkB9zA@`DF)c;!pzT)p-7!92IanRAq0Ew_UAnb`)2y4kB7Q`rF zo#`s1K=tbY*%vzn8S@mJfuaJy*r@94w3HYIL;~#>s-hZz(1F-0s_5Uj#yluq2zi=U z!p8=D9*YYfH+?MVP=Y#x4pIO*O;kI~9~D>~a4dZ26qo`QhU~BCK4xNfmgKBbF0Tyg2e26Y|Q#{h=tmY z^&L}p?*f9p&=G=WM@G*GJZp{S+2RI8zjl<|T|i8HcYDddHB(h`o<@~uSpZrh-R+Ko za1+M^ihm;}b#7z$8h(H}h=;N{0;qhhGpNvI+iZEPSl!pr=!@x?TFh0IZYfa+k-AeE ztb5F~b3a;N1WMi+yDKaO`UPA@``c4uWeHdb+YagjnlFoZi0RmMfQkSyM}e3Z;;aXE z^j%y-PxeERy9Nfw%7gI2wlVcb%Q)^|^<0HG1%e${A(Qj%pcc7CXu5nm)Va84BETzl z?s0lIa5@nir~b{L0%Kxm;d*_Sfy*cdmdp28I7G3YzBz{bVVI(D1t|PhoTXmUeeY}* zZyB8^+gDcVhwaANg1r(wO()G4clm&$p?uy3k0UqbPK)^>1$CCU9s@<5V2m?RG+ z$-_zVNRm97B=1X-_b16?N%Dat`CyVfo+M8s$%m5U$t3x3l6)jdKAI#SOOmIOP0#Zy!_!C zgg8XuMe%*b>;AE3`>tv{pPbsShpnk%H1G~tf-PhSAQe%7GWUW_pj*av)K&!284WE9 zsL&t~vPUgq0(`t2y_`=_65O=}kQ~VoFq?1E3em$1xhW~VF0VKaRY;g~V` zhP&38KTpe4N^RF+OhqtbaB8ZGo40m*s1Dho)k=ly`{zV}cjA1X5H8YHBcb$q>ZC{t z+|1{w2W(j);bk||n(v0z#3c$bZy6iP1odnR@$LZe1a#*EQEognT`Uz!3Fu6PA_Y{| zd!VxZHjaJE*Z9TMRrX?SvCXu(STi_0IRa5{qku$QRj-98tB&jyJ~Q9KEwM8oAq9Z+UI6Kl zI0grG+{A+sbDl*SyHP{+(b2b;>T_Lt}a`a8bw#fSxocoiU}4LML1mprBfY;N6#D9LwNT5qV4XDCMsp#1(MVE$j?cy}l?W5PV%Y`x075behVVhY#!VCy7S z>`CNA#dd)m0IkwEHomsaXKE*Iv~y$Wdf*Z$p~Zhu+E(c0Hj7%APJV!5RWA4Oz#?QK zWW)>xE91BX>w^@mQA(KsBqmtT0IdwJ4^g;&ERF>$cUX8VSY_m0lD#W>DE)kR@jwaV0<@NOdu zix|@D1n8rXp2r)LA?y9L_2UoUi=%KZdea%HD|io!&_mUhU84E`fNv8 z$8JR`WuKr~%gN2QRLka!B$;M=nr17{K5Cn-v^@FRnMiE@2F>R)0k1`5eu2 zPa6QZdD3YaWl2Q{=g^Pj%uG5 zXm;%swPV$VH`b?UK5uW+Sd61l?erqerJXcK~t|XREUvhreLh@%DF6uf?ezxIO1l&CTgRTt;I$4IPvFDQ_yqkx! zua`Gm+O>zgyx~^yby?Rq{X*I+4|mZ_zavv!#wDaGtJO5)LFo7RZtf8Y*$FH=&= zi#Uzsp4visK*uj@i=zv*Lx?ajSs5Q3LnK;(ve*?qwLHJHJimGz;p3%|^6a<>r`1_n zTV*8J$Syr_B8}OHxdl{BwBEc^Oy(pq(l%e7Ve&~aArgK0|o>hYtr znWlG^#wbzTB-%lbJS?M?x9KO>xews{3UmKu$o`>)r8y}GTuhnt;q`Ge>Ofh4ck$nn=KdydHTa&IQ9O1@@Oy8n$LdjDy+@i5G&#NNRUWO zJp1s~T0%Dx-C9ekDYiCIT>B6ZUXgc6aBnzrn|fyA8ZC$bB`ZCELSJ46BaZAl{ZHlv zpw6_iId+NZjzssxJqJc>Fga7JOV~L?57A29TU%b7Up$oBbmxH+z0`#19q1ir$3JeH zX2(DF_VpfKT3N-628K=(``f<(0pA`mOFu_r>X0RNcmA9bY}F{`}v4oKwYKx-Y&ioT~k9TX9#* ztI$ZF{8K>=d+EOT`fzf7fvo$7pFhVNiM@1Ry!kaj)o=anS2$JdrTgL=Vo~*K1L_Y? zf0a|kUb-*tibvL~Dt+Jk&@B&r>&sUsE|M=GtM`}RTA=3CKM1ne3&`5>njq_okN-)E zwHAAUYOS7r#!4piME7|;a}D0f$h}U>jy7dWBT{X`FZ%iWZ}}fT{dRZ4c|rEiJn%0? zmi?tYJE_Ml)zj0n>F2M!^2!^>pNG}C3Es<16Yv#ye0K0;sg8lw+VUYAf8gYzX8a<2 zii>MU=BU$)+1bb7NVD>NX*kL9xsxY1X$S7ul-R^H^3Q+khu0=jLrJQp*?YOxI5NKo z!Ztnd{|wL$vZqI<9Zd2|_bu;FZmY}WqlRg?0}PhaiAi- zlp&EGN}>t6Q)zF#3`4gtbSSWT0k7Zq$rrC>!1#;jzjSRxy%6;W|Mbe0PQ6ZRzvOy> zd2hb&qc_vB26%}-X+B~|Tz}t}-i}{2hif|CWlcq#E%*KXI~Y9Qz~MR4(?g?w@7v!; zW5^ej@WxZu-2${|94v{K=W0eXJ6GIhnuY8(tC`JiD>RDDZ8a-znN~BGohukUzZycx>O>nkPocC2D5EndF%k9bjJVB?M3SC1Us?pGz-?PGR*yEtQKyUzc^RS#W{y`+smVt~<}?#t6m zNBByA0<#RP_`i{&_xHaoWbjM>2`^__ys>e|_F0VfR<^IKE-l*IsRwQQk>m8Sar zx6d!u*XC-gOUv6@YO}C(Xoc0m=*KIn@%2w@z>cWKb>3>ExdJ%;9aatS>ZMm8K9p(y z_kE|XVlNLrjh8d4hOyl|`)VK)QH?MBm4_SG2UX)qjdBCj|Nc*f!Z>{j3WNUg;*)qe z(`s1fq8gT{#y|hi;Rb}F|Dit{7eZcUtmc15n5aMUzrV{d&0Y}u&WFA*$zosF!#j(O zseG+rSbAp8Xj=75-m>z=TA^s>Yeu1F7;}YYYU0?jW4GRYcRz0W89jEa=c0dp^p>yx z^@qV+$jX2I(f6)RNbP|9FMH6QV)<)5Wxe0%&#>S2@1bJLe_$O_;*kfxl{k;j{oRoG zCY$@~-FX>%Zn=(sZ@qhpeQ*7{zhuuF@A*1^{`Fs8amD7VQdeAc-jUkTe#2r9bB@rj z=0@7P_RnK3?O4B6Q=9bx3fj^4;QR9y}{;2augFkHkXtIaF%QJX+ z1~1PrKb2%O4^W!><}iV9*Fdvx!CoAg>sy$=#~!a8!(VtCnPS~$5scBA#YAeX^|1yLa8Jzc57N5O5a4$JFtYAnHhKkR%&&6 zK;X(X>{o&CY+rrZ##E$v0H0ZhDP$?b3p(1TDIFzR~^x#mQ761*TN~E=dLq*z< z8i};|$h3vvP@dMz1c&l+&5#<2v}R=5$Qqd;ToC0(*2oO0kw_cDBOYo+*4JvOZz8c_ z>4Wcra^zZwYc_*xHiK(6gKIW}Yc_*xHiK(6gKIW}Yc_*xHiK(6qsjCs*O?~QZzk7o zCf9GK2otKEmum7-Oc_Dj-55 z%gu!}I!}wvPgPu^X+&s9eMM*_vkNT@!&oHuJEs$yxwx5JdQ2`oCYK(QOOMH=hl_^A zrN`pZV{z%Rxb#>e3YQ*>OOM5+$Kujsap|$R%v!uuiP|vi$od@&8dZwjhHMpf{H8`Yf zk+~jSw%*e6O6Gctu<>9~F+IBMmNABn8sln@{`ObJqD#28UV*BYCZtXc@qkQVZGL&3X0?nM) zh}6|DC81$>V;WiItKj`Ib>+)azLXNoyb`A7Qe@q)G;h5Tt&H-e6lW=9TBZ_HA;nUv zU@3%I3Sm}(t2EhyKJhb?4b8GNYcAQ;vbv_@f2O8qH9fECMNOa6^oFL}n%>lmjArP` zqGlMHVQPk@8ClK9X+~Z%3Yvl6Yl(!G`JJyMQVE4_g%Me$_F1L&S*7+_rS@5+_Svkr zd6nuGlmZG$vVziR1;HaBvrLh|aYH*0=N(FgS#$kfeh2uq(r&QVy2;pIdV5j)Xd$}Os!h=kK0`aAUWreWtu&h7<|5Ac-0*@j{fJb~O zROS^b^9q%D0Z*Q;pu{ZXyfsjqU`3?{MWqHsh0LOWl+)d4x}oWorsp)hkSu6=P1C_9 zLo>}JIA&>PRx@*&nb*vMW)?LQoV1hRP)4(K%`!C0)be@A^7QW~p|LrRWah>js2i$Z zuZicJc-F}n~zaUGu<_g}gx&dhwVY*$bqhD}h)^iz8 zx`v^A)e0`1mR+~px?9a6G_lMjJEs@CUyY2DPPc7T-;LSGW*l1JpscxHdNX4*>WImm zmzJQKeK}OiFiy4j5>(4CPPOh&@L`_kGOjKCcCbyj#?6%TE;ueo5aVma9IfMMCWW;WAcTXoLap9>vFICJ3Y zbJBzBIK=ZzezOg`E*#?7LLpbrngQNrz2WgLNU%-a_8Fqm*uxCpF87) zav_W0&Y8>6GuYBph-9WI0^5s*B#Lzq0lPsKO&kHX27JF{|5r1Af&!3ln|Jzs4 zf7{7hh)&A5RP#92Z2p_`?#B(AeD_(NoKG$w7vdtD9XmS#F;!FfYnV6gy z#{Xt3^orQ1h~iXFdURT7vk2e7RFA>o|As3f0dd2qEv%tRGAlnK4Wtxs;z+p;EnRyk3jNY_d9W7T% zRen7HKZRFF;8dlwHzLq;;YDdjsUi}|7Gc|w(*ZTqb1hm&rr`&Ew+ecpB?aG2<)zs# zj3r&-g`vmq-`_&ECsw~)+u^pnsFQkZ)o%=g3=X8%#$ zJ{bE1NVX9}C!|?^I}*JG68%z?M2i&D=~&|IKiFH0hngUW1@vqR#Q|x8dN-M-Bk2}f`-UfHS=Um=O;|I2aEdqx5qHsu z)31HKF}tio*1d!ya(&Tg2XZ4YcQQ8S{MVcXw{fS?y9vvL3vM4mnSCzRdA4e;9{8S?YJ!gr#A+rDP*=% zxNQU6-V+~g{%dv_-h3_EDcbtmp-s&{9@hMq@Sa1k(c5~r18pz1xr4@$#meN=>j`k^ z9C7^>XvVd)t$lY^H{^)kiD<-)h)yUnM<`IIXt!r}enmO%uVa%FxY10|sf-ES5F}7S zMoPm@l8M>zQl&h+MK}9WmaLK!0Vx9`P#eU_LKq}s0O1DFV!5rtt_au!8GCPR88acS ztwsUWxGIi16|=5x$l2%HX?3NHSfWPhjW85spL-^H9a-bR+FygLeP3GdQ6|IC7_8UV z(a3_)>l<=*UD}aC9tu>*zJ}-+DVMd=$Z7$xU_vQRBePkQyC)-H|{>rYmfamdjMgIHw z*zh8-@;>IPb)w9_63mZu76oM+OUnIM`UMZWgpC`C0(RBkRRF<%jWf)mcD;jvv*176 zhn4*yBjzG*eeoM~mGTa_N;*;lDmwzkLqV3SbDah2Mq?%tjl%AU^jVr>$imqZ<#vHg zyElPTw~)UHI`Ip&bCh{JF_8i&;=wl@?D!So;f`FUa<7K?9g3}T12wK6c0S2-&_spq z8VKD}Q5x5mSuA;r*1B%_k)p=*?X?iKAIH(R3^TL1G^fg0f5sl1Gr}~xIca!jX?VAD zA$c&gWgHj`!9E#=fwuw!M`B|jaIn?V9zGDL)uc#B0STw0kl@cU78Ih#S3W8t@xX>t zI8tll2jk#jY;t(_S!i6EJXC*Lo*6C1FC6u?u^O9SFko9PDM7D~OD-9dL)U>rk4JHc zXA+AqAR=O5D}C_IT#5T3djsVP|I1*AA^ZleWiZ zH2a;R*&_!SYsgYBw?;&yag zo*3bvN?SaLji{3URzSwe&?@!xNQsP+ z9b_jpI(5|O#g1!4&FXX7xSm=?fZhPHEA0?~rl)Aa6}QtfVkFUudf(vM?;!CFkI+sg zivB>`c0$>#SlY+1Zg+SuT`zK$<)d+e`(X=29gWc*yQ+bTzD*;9ca&L_+g9zPErNT> z7{$8%We!EP)w7i4XH-S0c7hc@Re5=oFRj8*l}F1zlNOf6yVWgVnEjPD)0Qolp2oXU zx5n-RtFA8HWiLiwpe^pQW3wwM?+wSoke->`UBbj;n>*eyN9is+VN=~RgX2>{`Edk` zB`T3Il4v{7M9oJ4!{(+Kg1MhO%~50K=B?TV_ftNf&tC9 z0V^-JSvtr`qF36+1PPAuUyVk`OnHW`zH7t%>7A3eRr!J+nB0L}pM)t+*HFx0j3SW5 zF2RgW#-q@I*era}-n@HpiNJ+)1!{V164L0;mI)d2<#`iL%#MvY{-NPgnO?xCq{XT~ z{%ds@gK|%%=N`2;1GT$^I`;q~1hs*8(zDiqzaNwQRC~Q0Xg!XFtnky#&_$D;tUSnw zh0>01qi2`VzXKTk<5>EPVwCGa;X$J76ckP=SH{;UP!G||@S@Smxz63B60$}%_`+@; zHxjhU`qp;&oqL4b0))sraN8qf)IUey@?5mE#L2C|$*D-K=CUUiT4>Q80%PbjqyS(C zftMHCtOsSa5LaTy(`q;s_(*NB5segi$#x|F#R2c|mw4`|_TkjkC^E|_tj!)*ln1*D zcLb-zPhgs2{}lA80(CcwQ7rhL#rjPwt{9vt?Z9Bt8_`vHw+&Y4QS$+rwZ`E%Z;>yz zQ0Z`t(Hocd5ABP}~W3lx$R+uYxfC^(NiNlb@6A>C);y#uXqR7sKaFEL22xRa_ zadftO<9IwVY~Xyjm8G@ix*aWvsu zsh_u*xMlKCs#KsMB*=Xu0blD%?HZ{+<7geJAJ&z+916!)>M;;5l(1K|dq&Z+?a!fZ z76p!Df$ZQ80G5`<#B3+TEWTA(2$C`%KAE85`jdpg{BLEli|i(2WSmTpNis$DkP4Y5 zGh~+RCAX1%WIwr`93ThD9ps&)O75iow=zWZzNnFxT4F3I1?8OyX@SbRwp0hnrS zmg=r)y8PdxOBLhP^ZO1ggdM~(Wom_}&W+KrL3m0+>V~=KK{^mplZXM5`lI6{QRs1g zJ#*Iym7U31pRBtU4ID^=SR?K?WqL6}_m9}(_f+GPBjr(S#`Gu;)1wl4`QU#7Lc1^a z;NSAEJz!GX%2d~NGYQdsG^%3=NE}OkDJh+G4@iOjVG0#~m#A>7xK9qFyH(6$su6;^ z&;Yu(r17Ja5X^Q^m4XuYKxcg80#RhhN^hkd%QeLrRR*Ima58%SdP88W|tlhnc?pK_t5x;x+F- zkqr@>nc>=8f=`M;YY)+??|XIWG!IsgVla)hmD96Bbf*iIa}^(MW49PvK8vk`9%Uy^ zV!-&VIy{c?hNmxc&4{YvrBeH&mX?6RJ4=s`Xoz#rA+a%E1=@6P?CxVQ(0SyqrQ3N- zqw{z;PUj(^5^vKmC8M*h11X@$XHZ$WvOTQpkd$CF-lpS_F;QLF0`6dus#~`QnMJ|3 zhxA#lz;#rln~e2Qd(lVzIF8i>QfUttyK8!vU2iiHr;cFt3>HTQ3>=j~p)`irn(~0i z?M9(d`hz!fsI?A4Mj9gi(Nxv;?(XNVkb?1RvpjBve5 z4#pZ~(K0&|6{LWu>_^RZhagsPClaorN|KRu7QsBgc zz_PtiV`jF;t&eLMm!pT#5}0g+=^Gw+$naZq9_CJ!Dws?24p7}O%(;iiI{}e9Vr!Z} zgoHP<1Wgs7`9M^69hyZfUDPp-7>w=&j6T&7jAU=@cF=q|Mhi2QnS2LF(t{Jv9)sb4 z)gy953Z00K8S{ow=9Wmp=3ZmP=cC=|yy>RD?w?Gq(e;-#GDqs9L2S|_hsa?vPu@lD zA`9dQStLv3D7l*~lNGW`*2q2NUUH23e`~;3{2dIVfm1&YI$9YZ5jF>eJrN6GEqTO( z7$vMTU4;~=ejOnDVy7Tuo`N$_Q~($oRh^xd62pKa#g%6zqQ^3NI{S_TZ;hOP{|FI*m zVs>G2wgQv+O#0Pjn@^UnRL>5uE)gw?*j@7Ri|thpgLiVLjfeN1agg|xO&~-oJ-2#X zB}mN9$HuHLhghiXSl=;q_bwpl3mqY7c4YL7z_Zq9o-J-r^lL}S-37$Ncej`PTQgN9 z=V?@lmIa_C(%tST2sd#&p!hdpLI|Vh%RoGo%@IK5bDcp2L-#6M04i4Zbu{{7IwtO_ zO1G3Kgh<_~4Awm(`Ex&7Uj$0t8M`Yi1^NYCM*G`SVr2O+ zQs5kyQ$fk0V{Me2Oc0Adw9L-6>_ z@)k_VIIp(J@YNghd(N0a1zN%H6;=oKBM8NRsE04pGuM!ljNl& z`E-(eCP}?W$AOnWe1i~&D7+}XuXx=*)@6Hk#8xS7vU57@Fq!pm-^HQx=bIUT!a8_ERrYzp!20PzHL=L1o0JT+Y` z6-o){Oobu^RMvZ-vi>%XeaqMQ#ne^yVr{X_w7FO_I6OImL4O+sB;u-iEks#$WUuha z(FmCc87X;gVCW(m5dJVn?2wvQ|gN@q;zqtFRg1t_SNA$a#?7{@X=l?2e^ zZ=)VDgA^#oiiG)nN#Ok55jbbjW1*ZD*BQ!@0w}*f37G$vINlu!&6qHcH(ReVFGTw> zftVVcp2pTmtk{!)j*9I9J3ywaiYB&OV&N0)uyTim$AVQxZi*8~ z++P;$0+Km8AXH9JW$pejZTI-5X#s~Hw66S-CNC&XN4_^HKWmpi|2eWp)HZ zScPfJJw@{@w;@sH>G25$SKCC6PtzRR@d7!r1x`_Lh(iVL_PFBu@ zS7d!ZNwaFFE)?ftXm(Wlyg;*Sr>GsPF1)cmMe})ko5o@sjcTVCX)f)gIU2uInqQ*X zw9_um_&lT=qZxggX4I}z^o|5W+nS=_Gc>n$tW&!D@a%Bmb9VmIT^sm?&)5FmDR#m0 zwO{=s{`~Vl;Lrayl-!Wiup;^kN%E5<61`>que*|1K7GmgT?@&dZMdlGIQiLzTM=;c z{CB!GBKZ54n+_lB z(h}rJjhxm*me(m+Zsu7clN$~{NRpyDKf0fj|IxkSWOS4PaI6Itg-VxCzcxa!Suj^gXx3m zJNgc$o3;4`yP^DgGTrB9u`+|{zVtw|wyP^N)TVOrstPOZD2q6iZ60N2?~+%TkKj zZZ+TtSyc%)D9uS#AF3`z#ml>)Z!$eucUJGGzbI!qS`+ z1TLmb`tW+f51;_q43@+biYW11J@UqXa6}al<$~%J({vh$Z`dnLm~j>hA_2kyqSm6j zPIxsJib_Lmd9)V)g=K!!(MBim?Nc-93@uL;_Owo=N{S&05;Dh$bES20ZDFBL6=oFB zPARm=r&swLxZ@m;4mKODF!8D9QE}JbF26_6!UpV#te)4E9 z(wfiy+pDlPb3?3z=OaNPG4bquS8EB~NOWs0sixT4#47i72ZUGTT@pM7a+`Xle2o@F zfRdFSK%p-$gAqseo&KqL0jM*rY>r)Gx+BqjanFI#8cfdA>JrXlr-x{z?yW5^&MzKH zZMyToiC${L^bYinv*RDPO~dH2mwWqq4==5(;%B{R^y|4of4)D5qx*a9d+bH}GoSQH zHJ0vO#IG{H1q3hf(D|jcmEMy$<{;6NNL=*Y58d+0-#q^3{VlUknUl&f*ez&c-tL0T_q)+~-AcwtlUwnNyIln;G{lm|ns6J$?|tZ&hradYs}mQ=7m(Ha%Wo}E zbLt-iS?mR5?RZU)^~J~kB*j{by+E~APd{fR6MCZiyq>uR?_}g&r)5W*vZWEJHsKfj z{QbB5kDq?KJK?+_`)3~be~T>pOM7-wk6Wszr)Sg8UwP$~H;z9Kt8){)mzyTwEAaU2 z;K@=Q1FN;=LpJ`v$wkfhMfemK*N)6lrx&xckHL{<<@?fblI3$JPj1o<+_5RKiD~4Y z|JDz$O{9jBR86z@a;ei4Lidf-O`w1e#F(P;;h{L=kvZ{xjRs%IaibX#T{?Ot`u z%;-iopY7>Q(Oe$;%nMu!{^gr^Nlz)g$R-Vr$y5)z1{(G6{sM(3cy0GQh1&KcQm;;J zujWhs#Y9q>TQ9@VEestBtX{zDH-7TPYZ);9;`uLK z8&NMr{lP!Ia-~zR)7me&USQsv@B7HjbgThh;!m277!udt_rzy(ZBcY@1rr~3rcw7sq1e6S~L!pMCR0mx3XlUXE#p#rDG8Jq~QPT z-~W(H0eeC6^r@XYM0^ww6l2G!H||O%F!t)&c)gvg8O`imahqutvfHd?HoL9RC^om% zth{Af&0Kb_VDyaZM!r9T|8!cX-+k_>Yfw(YI`x^|l+JjWutw0nbN>xvU2J5P=)Rmc zLz#JfrKBFO{cJ-oZH@o+mjE<<`SG{K%h&!9FNzFoym9;Lk)zxFs${!;%&u=2XY6d( z`G2_Tq3f}iv=K-QFxu07d79}6U+GU^mVp)jH&XQe{ssUcT^a{j>GVTAq@6=W7<>9CCa%R;qwwq^P4P+v!@rA$gaO3)*YCNe? zZh-pV|EW+Ir%yp)&|hAB5-(?34eMM~!xGi_=N~%UfKc>5^rz!O$jgk?{O<@8^+*2q zcR8ln3u52-(C$i&hYn9F%_3MR8SlHTPV`*&ZP(Sno z`Nre_n0QO#!usL0#k=}-Q@7pMrt@lK+*h5IqO&6P41b6(gMXR)!Ku=-{42*FdHyKy zN0C2j{4vKLb^d7Zhs_^N_Aq#P1~1RxDXXU5ZUgXNu2YPC>mQO37WDoKmWc%Zo>fou@^|tg1E9G$J&lz9KY|*%&GKa>fWM_q%%%o4L4|TzX6{ zJtmhPlS_}urH6}##ihsM(qnPyvAFbDA_|uti%XBirN`pZV{z%RxXfC-REw8t@lq{b zs>Ms?d1ZO2Szct1Z3;aWxmr%=zdz?}#4 z#CoQsWHq>@Xf-&bY>~MhUAErR@=E4|sEVR+nE!V6`;V7=Cuq7*~7rx4$YT z59C$xH>e2YQ1HV@N$IbnlG4v*b%^ShDiJF1=dX_mf9wx_`>Tno5=@0C7OS4&HmV~w z%BL>uS;;je(9C&FUtRrD5*mg#rjb>?3f?bMSH3LeODVz3D`9FbMb`aFqud+O$|zq- zah5`+WhyZhQY@tkmO_}N5M~v)N|P<<>(I<(L$fT+noBmdtgh+!pQ-6tP0wq3QPbx% zy`ky0rZ+VsqZxX#s2PT4n3`c}MpiR&nvvIxf@a|NS|Xukes3j_N+@(IjL0gr&nmUg zDz(olwa+TG&t|>Nt5mn36i`r-6_iFR2p$QUWr_suCfb2G?@%htn(OylbG&>GcL895 z{fx;mDnE1TXHogo4fQjtehN%UsZs(BUH!Dw&%E+!sx+D^jiwq|B2*L$F3)L3T{9Y* zVQWTHGc%g0C+9S?rkQokY-naPnM+#9Y*J5}nw8b8oMz=UtDsp$&8j7B&8lk_V1^ff zETd%&EenvrQIi@@E}v5Kzu1-Ss^SuEGtmJ zzm%Yyz@rEf;1OR6m3f8Ayh3GOz>}vdC@~8;Zw(aJT~Vn)QK>;uA+snT<#aciZfLrt z={Zd=Bnz5e({!-O&`dK4j#-+S)y$k`<~6gRnMKV6C+#FSl+i3*vkc8LwR|43JpJW^ z(AXSDGIQe%)MM7K*Ti#9JnQ1wIFE&!{L_?AOMDl_QzSP#9Yij7VNDW0^0loxxZ}`mEt+<38)A*Vk$EH5cqvEC*X2FwAVN zX_yVVQ|H|M;~stLvNlEedQCV`J>GHZ)b)FFso%N9N1D%hsfP>LaeSXUl9ik;#JtKQ zmp1qu${!>WdwzLyC||k!f5wSBGm1QUsh{cN-t=3W!d@VjS}v@ISI#cp4pE|KT5`%Z#$ zPfq103yDLwhOv!)Vq~x)&fgo}i94@{aUv)^3u-@>Vr`*kL5aBI<>^t}6VI-9s^G8? z?ES-Xjf@4`;3OV~n%{tbJEdGPNI-&;0sfS

    9+dVANTZwqCRi-f@HUQ_ zNG$Ceo}8V)wdCShD<3f;4!|VNq79D54(pV4FX4zBUo?uj8-cl#u`%aA<~+EKJB8j& zfZpe#py%geECl+GM)&aCgZ~hDh(sVgLcI?N{Cpe)(&PIj51ivcJ+uVC;+Kj)MsI1~ z6izsqW`}gfl#lXwsd_t3OT_67>3}*FGFvI!wgGPMi4QmbH9HJ%z838iZT;=grsf|H zYyL}kJPbZtu+iIkwgYW1wmE~=k;Tg7)awawxTBQbvN%=6wKT0*%j$;gPo0RkZ!%s) zbV894p+KFY-JaR`73H|Uj!jPBL~}u>GA3|CkU$9;DGfVGCT7P=mGbZw-Rw(Qyh@J! d@3Ft#wPUc#9S1<(Y#KSEIj7jaXZvyC{|~3$rtbg% literal 0 HcmV?d00001 From 7ed28af4fe3a205e83f38834c3db0b749a214120 Mon Sep 17 00:00:00 2001 From: Tboy Date: Thu, 5 Mar 2020 19:00:06 +0800 Subject: [PATCH 062/221] Upgrade curator version (#2083) * upgrade curator version issue: https://github.com/apache/incubator-dolphinscheduler/issues/2020 curator issue: https://github.com/apache/curator/pull/297 * add DefaultEnsembleProvider override * add some unit test * add License --- dolphinscheduler-api/pom.xml | 6 ++ dolphinscheduler-server/pom.xml | 6 ++ .../service/zk/DefaultEnsembleProvider.java | 10 +++ .../zk/DefaultEnsembleProviderTest.java | 65 +++++++++++++++++++ pom.xml | 9 ++- 5 files changed, 95 insertions(+), 1 deletion(-) create mode 100644 dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/zk/DefaultEnsembleProviderTest.java diff --git a/dolphinscheduler-api/pom.xml b/dolphinscheduler-api/pom.xml index 41971734ec..b7c3f3da69 100644 --- a/dolphinscheduler-api/pom.xml +++ b/dolphinscheduler-api/pom.xml @@ -140,6 +140,12 @@ org.apache.curator curator-recipes + + + org.apache.zookeeper + zookeeper + + diff --git a/dolphinscheduler-server/pom.xml b/dolphinscheduler-server/pom.xml index 080b87ebaa..86490197b6 100644 --- a/dolphinscheduler-server/pom.xml +++ b/dolphinscheduler-server/pom.xml @@ -86,6 +86,12 @@ org.apache.curator curator-recipes + + + org.apache.zookeeper + zookeeper + + org.apache.zookeeper diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/DefaultEnsembleProvider.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/DefaultEnsembleProvider.java index 9eedf7a4ca..dbe8bd6395 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/DefaultEnsembleProvider.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/DefaultEnsembleProvider.java @@ -45,4 +45,14 @@ public class DefaultEnsembleProvider implements EnsembleProvider { public void close() throws IOException { //NOP } + + @Override + public void setConnectionString(String connectionString) { + //NOP + } + + @Override + public boolean updateServerListEnabled() { + return false; + } } diff --git a/dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/zk/DefaultEnsembleProviderTest.java b/dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/zk/DefaultEnsembleProviderTest.java new file mode 100644 index 0000000000..cdec9d0547 --- /dev/null +++ b/dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/zk/DefaultEnsembleProviderTest.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.service.zk; + +import org.apache.curator.ensemble.EnsembleProvider; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.IOException; + +import static org.junit.Assert.*; + +public class DefaultEnsembleProviderTest { + private static final String DEFAULT_SERVER_LIST = "localhost:2181"; + + @Test + public void startAndClose() { + EnsembleProvider ensembleProvider = new DefaultEnsembleProvider(DEFAULT_SERVER_LIST); + try { + ensembleProvider.start(); + } catch (Exception e) { + Assert.fail("EnsembleProvider start error: " + e.getMessage()); + } + try { + ensembleProvider.close(); + } catch (IOException e) { + Assert.fail("EnsembleProvider close error: " + e.getMessage()); + } + } + + @Test + public void getConnectionString() { + EnsembleProvider ensembleProvider = new DefaultEnsembleProvider(DEFAULT_SERVER_LIST); + Assert.assertEquals(DEFAULT_SERVER_LIST, ensembleProvider.getConnectionString()); + } + + @Test + public void setConnectionString() { + EnsembleProvider ensembleProvider = new DefaultEnsembleProvider(DEFAULT_SERVER_LIST); + ensembleProvider.setConnectionString("otherHost:2181"); + Assert.assertEquals(DEFAULT_SERVER_LIST, ensembleProvider.getConnectionString()); + } + + @Test + public void updateServerListEnabled() { + EnsembleProvider ensembleProvider = new DefaultEnsembleProvider(DEFAULT_SERVER_LIST); + Assert.assertFalse(ensembleProvider.updateServerListEnabled()); + } +} \ No newline at end of file diff --git a/pom.xml b/pom.xml index 57063a86c2..dee1dce8b2 100644 --- a/pom.xml +++ b/pom.xml @@ -59,7 +59,7 @@ UTF-8 UTF-8 - 2.12.0 + 4.3.0 5.1.5.RELEASE 2.1.3.RELEASE 1.8 @@ -244,6 +244,12 @@ org.apache.curator curator-recipes ${curator.version} + + + org.apache.zookeeper + zookeeper + + org.apache.zookeeper @@ -728,6 +734,7 @@ **/server/worker/task/datax/DataxTaskTest.java **/server/worker/task/sqoop/SqoopTaskTest.java **/server/utils/DataxUtilsTest.java + **/service/zk/DefaultEnsembleProviderTest.java From 1f92b4c4db3c2b96d767476bf77002d894de49c2 Mon Sep 17 00:00:00 2001 From: dailidong Date: Thu, 5 Mar 2020 23:21:10 +0800 Subject: [PATCH 063/221] Optimize ParameterUtils.curingGlobalParams() execution efficiency (#2090) * Optimize ParameterUtils.curingGlobalParams() execution efficiency * Remove excess null check --- .../common/utils/ParameterUtils.java | 36 +++++++++---------- 1 file changed, 16 insertions(+), 20 deletions(-) diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/ParameterUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/ParameterUtils.java index 7f2888384e..9492b49cb1 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/ParameterUtils.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/ParameterUtils.java @@ -119,10 +119,15 @@ public class ParameterUtils { */ public static String curingGlobalParams(Map globalParamMap, List globalParamList, CommandType commandType, Date scheduleTime){ - Map globalMap = new HashMap<>(); - if(globalParamMap!= null){ - globalMap.putAll(globalParamMap); - } + + if (globalParamList == null || globalParamList.isEmpty()) { + return null; + } + + Map globalMap = new HashMap<>(); + if (globalParamMap!= null){ + globalMap.putAll(globalParamMap); + } Map allParamMap = new HashMap<>(); //If it is a complement, a complement time needs to be passed in, according to the task type Map timeParams = BusinessTimeUtils @@ -132,9 +137,7 @@ public class ParameterUtils { allParamMap.putAll(timeParams); } - if (globalMap != null) { - allParamMap.putAll(globalMap); - } + allParamMap.putAll(globalMap); Set> entries = allParamMap.entrySet(); @@ -146,22 +149,15 @@ public class ParameterUtils { resolveMap.put(entry.getKey(),str); } } + globalMap.putAll(resolveMap); - if (globalMap != null){ - globalMap.putAll(resolveMap); - } - - if (globalParamList != null && globalParamList.size() > 0){ - - for (Property property : globalParamList){ - String val = globalMap.get(property.getProp()); - if (val != null){ - property.setValue(val); - } + for (Property property : globalParamList){ + String val = globalMap.get(property.getProp()); + if (val != null){ + property.setValue(val); } - return JSONObject.toJSONString(globalParamList); } - return null; + return JSONObject.toJSONString(globalParamList); } From 6070738acbb89e0a7ed49b487fc1225888e29019 Mon Sep 17 00:00:00 2001 From: dailidong Date: Thu, 5 Mar 2020 23:34:37 +0800 Subject: [PATCH 064/221] Support worker server to run bat script (#2023) * Support worker server to run bat script 1. Reimplement ProcessImpl.java, ProcessEnvironment.java and ProcessBuilder.java for Windows 2. Modify shell task code for windows 3. Add ASF License * Add Unit Test --- .../dolphinscheduler/common/Constants.java | 2 +- .../utils/process/ProcessBuilderForWin32.java | 1065 +++++++++++++++++ .../process/ProcessEnvironmentForWin32.java | 286 +++++ .../utils/process/ProcessImplForWin32.java | 752 ++++++++++++ .../common/ConstantsTest.java | 40 + .../common/utils/OSUtilsTest.java | 24 +- .../process/ProcessBuilderForWin32Test.java | 210 ++++ .../ProcessEnvironmentForWin32Test.java | 124 ++ .../process/ProcessImplForWin32Test.java | 70 ++ .../worker/task/AbstractCommandExecutor.java | 92 +- .../worker/task/ShellCommandExecutor.java | 39 +- .../server/worker/task/datax/DataxTask.java | 11 +- .../server/worker/task/shell/ShellTask.java | 9 +- .../server/utils/ProcessUtilsTest.java | 17 + .../worker/task/shell/ShellTaskTest.java | 198 +++ pom.xml | 4 + 16 files changed, 2887 insertions(+), 56 deletions(-) create mode 100644 dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/process/ProcessBuilderForWin32.java create mode 100644 dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/process/ProcessEnvironmentForWin32.java create mode 100644 dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/process/ProcessImplForWin32.java create mode 100644 dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/ConstantsTest.java create mode 100644 dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/process/ProcessBuilderForWin32Test.java create mode 100644 dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/process/ProcessEnvironmentForWin32Test.java create mode 100644 dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/process/ProcessImplForWin32Test.java create mode 100644 dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTaskTest.java diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java index 73125f4926..6af0e6445f 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java @@ -746,7 +746,7 @@ public final class Constants { * application regex */ public static final String APPLICATION_REGEX = "application_\\d+_\\d+"; - public static final String PID = "pid"; + public static final String PID = OSUtils.isWindows() ? "handle" : "pid"; /** * month_begin */ diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/process/ProcessBuilderForWin32.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/process/ProcessBuilderForWin32.java new file mode 100644 index 0000000000..4fb5f94616 --- /dev/null +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/process/ProcessBuilderForWin32.java @@ -0,0 +1,1065 @@ +/* + * 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.common.utils.process; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +/** + * This class is used to create operating system processes. + * + *

    Each {@code ProcessBuilderForWindows} instance manages a collection + * of process attributes. The {@link #start()} method creates a new + * {@link Process} instance with those attributes. The {@link + * #start()} method can be invoked repeatedly from the same instance + * to create new subprocesses with identical or related attributes. + * + *

    Each process builder manages these process attributes: + * + *

      + * + *
    • a command, a list of strings which signifies the + * external program file to be invoked and its arguments, if any. + * Which string lists represent a valid operating system command is + * system-dependent. For example, it is common for each conceptual + * argument to be an element in this list, but there are operating + * systems where programs are expected to tokenize command line + * strings themselves - on such a system a Java implementation might + * require commands to contain exactly two elements. + * + *
    • an environment, which is a system-dependent mapping from + * variables to values. The initial value is a copy of + * the environment of the current process (see {@link System#getenv()}). + * + *
    • a working directory. The default value is the current + * working directory of the current process, usually the directory + * named by the system property {@code user.dir}. + * + *
    • a source of standard input. + * By default, the subprocess reads input from a pipe. Java code + * can access this pipe via the output stream returned by + * {@link Process#getOutputStream()}. However, standard input may + * be redirected to another source using + * {@link #redirectInput(ProcessBuilderForWin32.Redirect) redirectInput}. + * In this case, {@link Process#getOutputStream()} will return a + * null output stream, for which: + * + *
        + *
      • the {@link OutputStream#write(int) write} methods always + * throw {@code IOException} + *
      • the {@link OutputStream#close() close} method does nothing + *
      + * + *
    • a destination for standard output + * and standard error. By default, the subprocess writes standard + * output and standard error to pipes. Java code can access these pipes + * via the input streams returned by {@link Process#getInputStream()} and + * {@link Process#getErrorStream()}. However, standard output and + * standard error may be redirected to other destinations using + * {@link #redirectOutput(ProcessBuilderForWin32.Redirect) redirectOutput} and + * {@link #redirectError(ProcessBuilderForWin32.Redirect) redirectError}. + * In this case, {@link Process#getInputStream()} and/or + * {@link Process#getErrorStream()} will return a null input + * stream, for which: + * + *
        + *
      • the {@link InputStream#read() read} methods always return + * {@code -1} + *
      • the {@link InputStream#available() available} method always returns + * {@code 0} + *
      • the {@link InputStream#close() close} method does nothing + *
      + * + *
    • a redirectErrorStream property. Initially, this property + * is {@code false}, meaning that the standard output and error + * output of a subprocess are sent to two separate streams, which can + * be accessed using the {@link Process#getInputStream()} and {@link + * Process#getErrorStream()} methods. + * + *

      If the value is set to {@code true}, then: + * + *

        + *
      • standard error is merged with the standard output and always sent + * to the same destination (this makes it easier to correlate error + * messages with the corresponding output) + *
      • the common destination of standard error and standard output can be + * redirected using + * {@link #redirectOutput(ProcessBuilderForWin32.Redirect) redirectOutput} + *
      • any redirection set by the + * {@link #redirectError(ProcessBuilderForWin32.Redirect) redirectError} + * method is ignored when creating a subprocess + *
      • the stream returned from {@link Process#getErrorStream()} will + * always be a null input stream + *
      + * + *
    + * + *

    Modifying a process builder's attributes will affect processes + * subsequently started by that object's {@link #start()} method, but + * will never affect previously started processes or the Java process + * itself. + * + *

    Most error checking is performed by the {@link #start()} method. + * It is possible to modify the state of an object so that {@link + * #start()} will fail. For example, setting the command attribute to + * an empty list will not throw an exception unless {@link #start()} + * is invoked. + * + *

    Note that this class is not synchronized. + * If multiple threads access a {@code ProcessBuilderForWindows} instance + * concurrently, and at least one of the threads modifies one of the + * attributes structurally, it must be synchronized externally. + * + *

    Starting a new process which uses the default working directory + * and environment is easy: + * + *

     {@code
    + * Process p = new ProcessBuilderForWindows("myCommand", "myArg").start();
    + * }
    + * + *

    Here is an example that starts a process with a modified working + * directory and environment, and redirects standard output and error + * to be appended to a log file: + * + *

     {@code
    + * ProcessBuilderForWindows pb =
    + *   new ProcessBuilderForWindows("myCommand", "myArg1", "myArg2");
    + * Map env = pb.environment();
    + * env.put("VAR1", "myValue");
    + * env.remove("OTHERVAR");
    + * env.put("VAR2", env.get("VAR1") + "suffix");
    + * pb.directory(new File("myDir"));
    + * File log = new File("log");
    + * pb.redirectErrorStream(true);
    + * pb.redirectOutput(Redirect.appendTo(log));
    + * Process p = pb.start();
    + * assert pb.redirectInput() == Redirect.PIPE;
    + * assert pb.redirectOutput().file() == log;
    + * assert p.getInputStream().read() == -1;
    + * }
    + * + *

    To start a process with an explicit set of environment + * variables, first call {@link Map#clear() Map.clear()} + * before adding environment variables. + * + * @author Martin Buchholz + * @since 1.5 + */ + +public class ProcessBuilderForWin32 { + + private String username; + private String password; + private List command; + private File directory; + private Map environment; + private boolean redirectErrorStream; + private ProcessBuilderForWin32.Redirect[] redirects; + + /** + * Constructs a process builder with the specified operating + * system program and arguments. This constructor does not + * make a copy of the {@code command} list. Subsequent + * updates to the list will be reflected in the state of the + * process builder. It is not checked whether + * {@code command} corresponds to a valid operating system + * command. + * + * @param command the list containing the program and its arguments + * @throws NullPointerException if the argument is null + */ + public ProcessBuilderForWin32(List command) { + if (command == null) + throw new NullPointerException(); + this.command = command; + } + + /** + * Constructs a process builder with the specified operating + * system program and arguments. This is a convenience + * constructor that sets the process builder's command to a string + * list containing the same strings as the {@code command} + * array, in the same order. It is not checked whether + * {@code command} corresponds to a valid operating system + * command. + * + * @param command a string array containing the program and its arguments + */ + public ProcessBuilderForWin32(String... command) { + this.command = new ArrayList<>(command.length); + for (String arg : command) + this.command.add(arg); + } + + /** + * set username and password for process + * + * @param username username + * @param password password + * @return this process builder + */ + public ProcessBuilderForWin32 user(String username, String password) { + this.username = username; + this.password = password; + return this; + } + + /** + * Sets this process builder's operating system program and + * arguments. This method does not make a copy of the + * {@code command} list. Subsequent updates to the list will + * be reflected in the state of the process builder. It is not + * checked whether {@code command} corresponds to a valid + * operating system command. + * + * @param command the list containing the program and its arguments + * @return this process builder + * + * @throws NullPointerException if the argument is null + */ + public ProcessBuilderForWin32 command(List command) { + if (command == null) + throw new NullPointerException(); + this.command = command; + return this; + } + + /** + * Sets this process builder's operating system program and + * arguments. This is a convenience method that sets the command + * to a string list containing the same strings as the + * {@code command} array, in the same order. It is not + * checked whether {@code command} corresponds to a valid + * operating system command. + * + * @param command a string array containing the program and its arguments + * @return this process builder + */ + public ProcessBuilderForWin32 command(String... command) { + this.command = new ArrayList<>(command.length); + for (String arg : command) + this.command.add(arg); + return this; + } + + /** + * Returns this process builder's operating system program and + * arguments. The returned list is not a copy. Subsequent + * updates to the list will be reflected in the state of this + * process builder. + * + * @return this process builder's program and its arguments + */ + public List command() { + return command; + } + + /** + * Returns a string map view of this process builder's environment. + * + * Whenever a process builder is created, the environment is + * initialized to a copy of the current process environment (see + * {@link System#getenv()}). Subprocesses subsequently started by + * this object's {@link #start()} method will use this map as + * their environment. + * + *

    The returned object may be modified using ordinary {@link + * Map Map} operations. These modifications will be + * visible to subprocesses started via the {@link #start()} + * method. Two {@code ProcessBuilderForWindows} instances always + * contain independent process environments, so changes to the + * returned map will never be reflected in any other + * {@code ProcessBuilderForWindows} instance or the values returned by + * {@link System#getenv System.getenv}. + * + *

    If the system does not support environment variables, an + * empty map is returned. + * + *

    The returned map does not permit null keys or values. + * Attempting to insert or query the presence of a null key or + * value will throw a {@link NullPointerException}. + * Attempting to query the presence of a key or value which is not + * of type {@link String} will throw a {@link ClassCastException}. + * + *

    The behavior of the returned map is system-dependent. A + * system may not allow modifications to environment variables or + * may forbid certain variable names or values. For this reason, + * attempts to modify the map may fail with + * {@link UnsupportedOperationException} or + * {@link IllegalArgumentException} + * if the modification is not permitted by the operating system. + * + *

    Since the external format of environment variable names and + * values is system-dependent, there may not be a one-to-one + * mapping between them and Java's Unicode strings. Nevertheless, + * the map is implemented in such a way that environment variables + * which are not modified by Java code will have an unmodified + * native representation in the subprocess. + * + *

    The returned map and its collection views may not obey the + * general contract of the {@link Object#equals} and + * {@link Object#hashCode} methods. + * + *

    The returned map is typically case-sensitive on all platforms. + * + *

    If a security manager exists, its + * {@link SecurityManager#checkPermission checkPermission} method + * is called with a + * {@link RuntimePermission}{@code ("getenv.*")} permission. + * This may result in a {@link SecurityException} being thrown. + * + *

    When passing information to a Java subprocess, + * system properties + * are generally preferred over environment variables. + * + * @return this process builder's environment + * + * @throws SecurityException + * if a security manager exists and its + * {@link SecurityManager#checkPermission checkPermission} + * method doesn't allow access to the process environment + * + * @see Runtime#exec(String[],String[], File) + * @see System#getenv() + */ + public Map environment() { + SecurityManager security = System.getSecurityManager(); + if (security != null) + security.checkPermission(new RuntimePermission("getenv.*")); + + if (environment == null) + environment = ProcessEnvironmentForWin32.environment(); + + assert environment != null; + + return environment; + } + + // Only for use by Runtime.exec(...envp...) + ProcessBuilderForWin32 environment(String[] envp) { + assert environment == null; + if (envp != null) { + environment = ProcessEnvironmentForWin32.emptyEnvironment(envp.length); + assert environment != null; + + for (String envstring : envp) { + // Before 1.5, we blindly passed invalid envstrings + // to the child process. + // We would like to throw an exception, but do not, + // for compatibility with old broken code. + + // Silently discard any trailing junk. + if (envstring.indexOf((int) '\u0000') != -1) + envstring = envstring.replaceFirst("\u0000.*", ""); + + int eqlsign = + envstring.indexOf('=', ProcessEnvironmentForWin32.MIN_NAME_LENGTH); + // Silently ignore envstrings lacking the required `='. + if (eqlsign != -1) + environment.put(envstring.substring(0,eqlsign), + envstring.substring(eqlsign+1)); + } + } + return this; + } + + /** + * Returns this process builder's working directory. + * + * Subprocesses subsequently started by this object's {@link + * #start()} method will use this as their working directory. + * The returned value may be {@code null} -- this means to use + * the working directory of the current Java process, usually the + * directory named by the system property {@code user.dir}, + * as the working directory of the child process. + * + * @return this process builder's working directory + */ + public File directory() { + return directory; + } + + /** + * Sets this process builder's working directory. + * + * Subprocesses subsequently started by this object's {@link + * #start()} method will use this as their working directory. + * The argument may be {@code null} -- this means to use the + * working directory of the current Java process, usually the + * directory named by the system property {@code user.dir}, + * as the working directory of the child process. + * + * @param directory the new working directory + * @return this process builder + */ + public ProcessBuilderForWin32 directory(File directory) { + this.directory = directory; + return this; + } + + // ---------------- I/O Redirection ---------------- + + /** + * Implements a null input stream. + */ + static class NullInputStream extends InputStream { + static final ProcessBuilderForWin32.NullInputStream INSTANCE = new ProcessBuilderForWin32.NullInputStream(); + private NullInputStream() {} + public int read() { return -1; } + public int available() { return 0; } + } + + /** + * Implements a null output stream. + */ + static class NullOutputStream extends OutputStream { + static final ProcessBuilderForWin32.NullOutputStream INSTANCE = new ProcessBuilderForWin32.NullOutputStream(); + private NullOutputStream() {} + public void write(int b) throws IOException { + throw new IOException("Stream closed"); + } + } + + /** + * Represents a source of subprocess input or a destination of + * subprocess output. + * + * Each {@code Redirect} instance is one of the following: + * + *

      + *
    • the special value {@link #PIPE Redirect.PIPE} + *
    • the special value {@link #INHERIT Redirect.INHERIT} + *
    • a redirection to read from a file, created by an invocation of + * {@link ProcessBuilderForWin32.Redirect#from Redirect.from(File)} + *
    • a redirection to write to a file, created by an invocation of + * {@link ProcessBuilderForWin32.Redirect#to Redirect.to(File)} + *
    • a redirection to append to a file, created by an invocation of + * {@link ProcessBuilderForWin32.Redirect#appendTo Redirect.appendTo(File)} + *
    + * + *

    Each of the above categories has an associated unique + * {@link ProcessBuilderForWin32.Redirect.Type Type}. + * + * @since 1.7 + */ + public static abstract class Redirect { + /** + * The type of a {@link ProcessBuilderForWin32.Redirect}. + */ + public enum Type { + /** + * The type of {@link ProcessBuilderForWin32.Redirect#PIPE Redirect.PIPE}. + */ + PIPE, + + /** + * The type of {@link ProcessBuilderForWin32.Redirect#INHERIT Redirect.INHERIT}. + */ + INHERIT, + + /** + * The type of redirects returned from + * {@link ProcessBuilderForWin32.Redirect#from Redirect.from(File)}. + */ + READ, + + /** + * The type of redirects returned from + * {@link ProcessBuilderForWin32.Redirect#to Redirect.to(File)}. + */ + WRITE, + + /** + * The type of redirects returned from + * {@link ProcessBuilderForWin32.Redirect#appendTo Redirect.appendTo(File)}. + */ + APPEND + }; + + /** + * Returns the type of this {@code Redirect}. + * @return the type of this {@code Redirect} + */ + public abstract ProcessBuilderForWin32.Redirect.Type type(); + + /** + * Indicates that subprocess I/O will be connected to the + * current Java process over a pipe. + * + * This is the default handling of subprocess standard I/O. + * + *

    It will always be true that + *

     {@code
    +         * Redirect.PIPE.file() == null &&
    +         * Redirect.PIPE.type() == Redirect.Type.PIPE
    +         * }
    + */ + public static final ProcessBuilderForWin32.Redirect PIPE = new ProcessBuilderForWin32.Redirect() { + public Type type() { return Type.PIPE; } + public String toString() { return type().toString(); }}; + + /** + * Indicates that subprocess I/O source or destination will be the + * same as those of the current process. This is the normal + * behavior of most operating system command interpreters (shells). + * + *

    It will always be true that + *

     {@code
    +         * Redirect.INHERIT.file() == null &&
    +         * Redirect.INHERIT.type() == Redirect.Type.INHERIT
    +         * }
    + */ + public static final ProcessBuilderForWin32.Redirect INHERIT = new ProcessBuilderForWin32.Redirect() { + public Type type() { return Type.INHERIT; } + public String toString() { return type().toString(); }}; + + /** + * Returns the {@link File} source or destination associated + * with this redirect, or {@code null} if there is no such file. + * + * @return the file associated with this redirect, + * or {@code null} if there is no such file + */ + public File file() { return null; } + + /** + * When redirected to a destination file, indicates if the output + * is to be written to the end of the file. + */ + boolean append() { + throw new UnsupportedOperationException(); + } + + /** + * Returns a redirect to read from the specified file. + * + *

    It will always be true that + *

     {@code
    +         * Redirect.from(file).file() == file &&
    +         * Redirect.from(file).type() == Redirect.Type.READ
    +         * }
    + * + * @param file The {@code File} for the {@code Redirect}. + * @throws NullPointerException if the specified file is null + * @return a redirect to read from the specified file + */ + public static ProcessBuilderForWin32.Redirect from(final File file) { + if (file == null) + throw new NullPointerException(); + return new ProcessBuilderForWin32.Redirect() { + public Type type() { return Type.READ; } + public File file() { return file; } + public String toString() { + return "redirect to read from file \"" + file + "\""; + } + }; + } + + /** + * Returns a redirect to write to the specified file. + * If the specified file exists when the subprocess is started, + * its previous contents will be discarded. + * + *

    It will always be true that + *

     {@code
    +         * Redirect.to(file).file() == file &&
    +         * Redirect.to(file).type() == Redirect.Type.WRITE
    +         * }
    + * + * @param file The {@code File} for the {@code Redirect}. + * @throws NullPointerException if the specified file is null + * @return a redirect to write to the specified file + */ + public static ProcessBuilderForWin32.Redirect to(final File file) { + if (file == null) + throw new NullPointerException(); + return new ProcessBuilderForWin32.Redirect() { + public Type type() { return Type.WRITE; } + public File file() { return file; } + public String toString() { + return "redirect to write to file \"" + file + "\""; + } + boolean append() { return false; } + }; + } + + /** + * Returns a redirect to append to the specified file. + * Each write operation first advances the position to the + * end of the file and then writes the requested data. + * Whether the advancement of the position and the writing + * of the data are done in a single atomic operation is + * system-dependent and therefore unspecified. + * + *

    It will always be true that + *

     {@code
    +         * Redirect.appendTo(file).file() == file &&
    +         * Redirect.appendTo(file).type() == Redirect.Type.APPEND
    +         * }
    + * + * @param file The {@code File} for the {@code Redirect}. + * @throws NullPointerException if the specified file is null + * @return a redirect to append to the specified file + */ + public static ProcessBuilderForWin32.Redirect appendTo(final File file) { + if (file == null) + throw new NullPointerException(); + return new ProcessBuilderForWin32.Redirect() { + public Type type() { return Type.APPEND; } + public File file() { return file; } + public String toString() { + return "redirect to append to file \"" + file + "\""; + } + boolean append() { return true; } + }; + } + + /** + * Compares the specified object with this {@code Redirect} for + * equality. Returns {@code true} if and only if the two + * objects are identical or both objects are {@code Redirect} + * instances of the same type associated with non-null equal + * {@code File} instances. + */ + public boolean equals(Object obj) { + if (obj == this) + return true; + if (! (obj instanceof ProcessBuilderForWin32.Redirect)) + return false; + ProcessBuilderForWin32.Redirect r = (ProcessBuilderForWin32.Redirect) obj; + if (r.type() != this.type()) + return false; + assert this.file() != null; + return this.file().equals(r.file()); + } + + /** + * Returns a hash code value for this {@code Redirect}. + * @return a hash code value for this {@code Redirect} + */ + public int hashCode() { + File file = file(); + if (file == null) + return super.hashCode(); + else + return file.hashCode(); + } + + /** + * No public constructors. Clients must use predefined + * static {@code Redirect} instances or factory methods. + */ + private Redirect() {} + } + + private ProcessBuilderForWin32.Redirect[] redirects() { + if (redirects == null) + redirects = new ProcessBuilderForWin32.Redirect[] { + ProcessBuilderForWin32.Redirect.PIPE, ProcessBuilderForWin32.Redirect.PIPE, ProcessBuilderForWin32.Redirect.PIPE + }; + return redirects; + } + + /** + * Sets this process builder's standard input source. + * + * Subprocesses subsequently started by this object's {@link #start()} + * method obtain their standard input from this source. + * + *

    If the source is {@link ProcessBuilderForWin32.Redirect#PIPE Redirect.PIPE} + * (the initial value), then the standard input of a + * subprocess can be written to using the output stream + * returned by {@link Process#getOutputStream()}. + * If the source is set to any other value, then + * {@link Process#getOutputStream()} will return a + * null output stream. + * + * @param source the new standard input source + * @return this process builder + * @throws IllegalArgumentException + * if the redirect does not correspond to a valid source + * of data, that is, has type + * {@link ProcessBuilderForWin32.Redirect.Type#WRITE WRITE} or + * {@link ProcessBuilderForWin32.Redirect.Type#APPEND APPEND} + * @since 1.7 + */ + public ProcessBuilderForWin32 redirectInput(ProcessBuilderForWin32.Redirect source) { + if (source.type() == ProcessBuilderForWin32.Redirect.Type.WRITE || + source.type() == ProcessBuilderForWin32.Redirect.Type.APPEND) + throw new IllegalArgumentException( + "Redirect invalid for reading: " + source); + redirects()[0] = source; + return this; + } + + /** + * Sets this process builder's standard output destination. + * + * Subprocesses subsequently started by this object's {@link #start()} + * method send their standard output to this destination. + * + *

    If the destination is {@link ProcessBuilderForWin32.Redirect#PIPE Redirect.PIPE} + * (the initial value), then the standard output of a subprocess + * can be read using the input stream returned by {@link + * Process#getInputStream()}. + * If the destination is set to any other value, then + * {@link Process#getInputStream()} will return a + * null input stream. + * + * @param destination the new standard output destination + * @return this process builder + * @throws IllegalArgumentException + * if the redirect does not correspond to a valid + * destination of data, that is, has type + * {@link ProcessBuilderForWin32.Redirect.Type#READ READ} + * @since 1.7 + */ + public ProcessBuilderForWin32 redirectOutput(ProcessBuilderForWin32.Redirect destination) { + if (destination.type() == ProcessBuilderForWin32.Redirect.Type.READ) + throw new IllegalArgumentException( + "Redirect invalid for writing: " + destination); + redirects()[1] = destination; + return this; + } + + /** + * Sets this process builder's standard error destination. + * + * Subprocesses subsequently started by this object's {@link #start()} + * method send their standard error to this destination. + * + *

    If the destination is {@link ProcessBuilderForWin32.Redirect#PIPE Redirect.PIPE} + * (the initial value), then the error output of a subprocess + * can be read using the input stream returned by {@link + * Process#getErrorStream()}. + * If the destination is set to any other value, then + * {@link Process#getErrorStream()} will return a + * null input stream. + * + *

    If the {@link #redirectErrorStream redirectErrorStream} + * attribute has been set {@code true}, then the redirection set + * by this method has no effect. + * + * @param destination the new standard error destination + * @return this process builder + * @throws IllegalArgumentException + * if the redirect does not correspond to a valid + * destination of data, that is, has type + * {@link ProcessBuilderForWin32.Redirect.Type#READ READ} + * @since 1.7 + */ + public ProcessBuilderForWin32 redirectError(ProcessBuilderForWin32.Redirect destination) { + if (destination.type() == ProcessBuilderForWin32.Redirect.Type.READ) + throw new IllegalArgumentException( + "Redirect invalid for writing: " + destination); + redirects()[2] = destination; + return this; + } + + /** + * Sets this process builder's standard input source to a file. + * + *

    This is a convenience method. An invocation of the form + * {@code redirectInput(file)} + * behaves in exactly the same way as the invocation + * {@link #redirectInput(ProcessBuilderForWin32.Redirect) redirectInput} + * {@code (Redirect.from(file))}. + * + * @param file the new standard input source + * @return this process builder + * @since 1.7 + */ + public ProcessBuilderForWin32 redirectInput(File file) { + return redirectInput(ProcessBuilderForWin32.Redirect.from(file)); + } + + /** + * Sets this process builder's standard output destination to a file. + * + *

    This is a convenience method. An invocation of the form + * {@code redirectOutput(file)} + * behaves in exactly the same way as the invocation + * {@link #redirectOutput(ProcessBuilderForWin32.Redirect) redirectOutput} + * {@code (Redirect.to(file))}. + * + * @param file the new standard output destination + * @return this process builder + * @since 1.7 + */ + public ProcessBuilderForWin32 redirectOutput(File file) { + return redirectOutput(ProcessBuilderForWin32.Redirect.to(file)); + } + + /** + * Sets this process builder's standard error destination to a file. + * + *

    This is a convenience method. An invocation of the form + * {@code redirectError(file)} + * behaves in exactly the same way as the invocation + * {@link #redirectError(ProcessBuilderForWin32.Redirect) redirectError} + * {@code (Redirect.to(file))}. + * + * @param file the new standard error destination + * @return this process builder + * @since 1.7 + */ + public ProcessBuilderForWin32 redirectError(File file) { + return redirectError(ProcessBuilderForWin32.Redirect.to(file)); + } + + /** + * Returns this process builder's standard input source. + * + * Subprocesses subsequently started by this object's {@link #start()} + * method obtain their standard input from this source. + * The initial value is {@link ProcessBuilderForWin32.Redirect#PIPE Redirect.PIPE}. + * + * @return this process builder's standard input source + * @since 1.7 + */ + public ProcessBuilderForWin32.Redirect redirectInput() { + return (redirects == null) ? ProcessBuilderForWin32.Redirect.PIPE : redirects[0]; + } + + /** + * Returns this process builder's standard output destination. + * + * Subprocesses subsequently started by this object's {@link #start()} + * method redirect their standard output to this destination. + * The initial value is {@link ProcessBuilderForWin32.Redirect#PIPE Redirect.PIPE}. + * + * @return this process builder's standard output destination + * @since 1.7 + */ + public ProcessBuilderForWin32.Redirect redirectOutput() { + return (redirects == null) ? ProcessBuilderForWin32.Redirect.PIPE : redirects[1]; + } + + /** + * Returns this process builder's standard error destination. + * + * Subprocesses subsequently started by this object's {@link #start()} + * method redirect their standard error to this destination. + * The initial value is {@link ProcessBuilderForWin32.Redirect#PIPE Redirect.PIPE}. + * + * @return this process builder's standard error destination + * @since 1.7 + */ + public ProcessBuilderForWin32.Redirect redirectError() { + return (redirects == null) ? ProcessBuilderForWin32.Redirect.PIPE : redirects[2]; + } + + /** + * Sets the source and destination for subprocess standard I/O + * to be the same as those of the current Java process. + * + *

    This is a convenience method. An invocation of the form + *

     {@code
    +     * pb.inheritIO()
    +     * }
    + * behaves in exactly the same way as the invocation + *
     {@code
    +     * pb.redirectInput(Redirect.INHERIT)
    +     *   .redirectOutput(Redirect.INHERIT)
    +     *   .redirectError(Redirect.INHERIT)
    +     * }
    + * + * This gives behavior equivalent to most operating system + * command interpreters, or the standard C library function + * {@code system()}. + * + * @return this process builder + * @since 1.7 + */ + public ProcessBuilderForWin32 inheritIO() { + Arrays.fill(redirects(), ProcessBuilderForWin32.Redirect.INHERIT); + return this; + } + + /** + * Tells whether this process builder merges standard error and + * standard output. + * + *

    If this property is {@code true}, then any error output + * generated by subprocesses subsequently started by this object's + * {@link #start()} method will be merged with the standard + * output, so that both can be read using the + * {@link Process#getInputStream()} method. This makes it easier + * to correlate error messages with the corresponding output. + * The initial value is {@code false}. + * + * @return this process builder's {@code redirectErrorStream} property + */ + public boolean redirectErrorStream() { + return redirectErrorStream; + } + + /** + * Sets this process builder's {@code redirectErrorStream} property. + * + *

    If this property is {@code true}, then any error output + * generated by subprocesses subsequently started by this object's + * {@link #start()} method will be merged with the standard + * output, so that both can be read using the + * {@link Process#getInputStream()} method. This makes it easier + * to correlate error messages with the corresponding output. + * The initial value is {@code false}. + * + * @param redirectErrorStream the new property value + * @return this process builder + */ + public ProcessBuilderForWin32 redirectErrorStream(boolean redirectErrorStream) { + this.redirectErrorStream = redirectErrorStream; + return this; + } + + /** + * Starts a new process using the attributes of this process builder. + * + *

    The new process will + * invoke the command and arguments given by {@link #command()}, + * in a working directory as given by {@link #directory()}, + * with a process environment as given by {@link #environment()}. + * + *

    This method checks that the command is a valid operating + * system command. Which commands are valid is system-dependent, + * but at the very least the command must be a non-empty list of + * non-null strings. + * + *

    A minimal set of system dependent environment variables may + * be required to start a process on some operating systems. + * As a result, the subprocess may inherit additional environment variable + * settings beyond those in the process builder's {@link #environment()}. + * + *

    If there is a security manager, its + * {@link SecurityManager#checkExec checkExec} + * method is called with the first component of this object's + * {@code command} array as its argument. This may result in + * a {@link SecurityException} being thrown. + * + *

    Starting an operating system process is highly system-dependent. + * Among the many things that can go wrong are: + *

      + *
    • The operating system program file was not found. + *
    • Access to the program file was denied. + *
    • The working directory does not exist. + *
    + * + *

    In such cases an exception will be thrown. The exact nature + * of the exception is system-dependent, but it will always be a + * subclass of {@link IOException}. + * + *

    Subsequent modifications to this process builder will not + * affect the returned {@link Process}. + * + * @return a new {@link Process} object for managing the subprocess + * + * @throws NullPointerException + * if an element of the command list is null + * + * @throws IndexOutOfBoundsException + * if the command is an empty list (has size {@code 0}) + * + * @throws SecurityException + * if a security manager exists and + *

      + * + *
    • its + * {@link SecurityManager#checkExec checkExec} + * method doesn't allow creation of the subprocess, or + * + *
    • the standard input to the subprocess was + * {@linkplain #redirectInput redirected from a file} + * and the security manager's + * {@link SecurityManager#checkRead checkRead} method + * denies read access to the file, or + * + *
    • the standard output or standard error of the + * subprocess was + * {@linkplain #redirectOutput redirected to a file} + * and the security manager's + * {@link SecurityManager#checkWrite checkWrite} method + * denies write access to the file + * + *
    + * + * @throws IOException if an I/O error occurs + * + * @see Runtime#exec(String[], String[], File) + */ + public Process start() throws IOException { + // Must convert to array first -- a malicious user-supplied + // list might try to circumvent the security check. + String[] cmdarray = command.toArray(new String[command.size()]); + cmdarray = cmdarray.clone(); + + for (String arg : cmdarray) + if (arg == null) + throw new NullPointerException(); + // Throws IndexOutOfBoundsException if command is empty + String prog = cmdarray[0]; + + SecurityManager security = System.getSecurityManager(); + if (security != null) + security.checkExec(prog); + + String dir = directory == null ? null : directory.toString(); + + for (int i = 1; i < cmdarray.length; i++) { + if (cmdarray[i].indexOf('\u0000') >= 0) { + throw new IOException("invalid null character in command"); + } + } + + try { + return ProcessImplForWin32.start( + username, + password, + cmdarray, + environment, + dir, + redirects, + redirectErrorStream); + } catch (IOException | IllegalArgumentException e) { + String exceptionInfo = ": " + e.getMessage(); + Throwable cause = e; + if ((e instanceof IOException) && security != null) { + // Can not disclose the fail reason for read-protected files. + try { + security.checkRead(prog); + } catch (SecurityException se) { + exceptionInfo = ""; + cause = se; + } + } + // It's much easier for us to create a high-quality error + // message than the low-level C code which found the problem. + throw new IOException( + "Cannot run program \"" + prog + "\"" + + (dir == null ? "" : " (in directory \"" + dir + "\")") + + exceptionInfo, + cause); + } + } + +} diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/process/ProcessEnvironmentForWin32.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/process/ProcessEnvironmentForWin32.java new file mode 100644 index 0000000000..3dbe7cb50f --- /dev/null +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/process/ProcessEnvironmentForWin32.java @@ -0,0 +1,286 @@ +/* + * 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.common.utils.process; + +import com.sun.jna.platform.win32.Kernel32Util; + +import java.util.*; + +final class ProcessEnvironmentForWin32 extends HashMap { + + private static final long serialVersionUID = -8017839552603542824L; + + private static String validateName(String name) { + // An initial `=' indicates a magic Windows variable name -- OK + if (name.indexOf('=', 1) != -1 || + name.indexOf('\u0000') != -1) + throw new IllegalArgumentException + ("Invalid environment variable name: \"" + name + "\""); + return name; + } + + private static String validateValue(String value) { + if (value.indexOf('\u0000') != -1) + throw new IllegalArgumentException + ("Invalid environment variable value: \"" + value + "\""); + return value; + } + + private static String nonNullString(Object o) { + if (o == null) + throw new NullPointerException(); + return (String) o; + } + + public String put(String key, String value) { + return super.put(validateName(key), validateValue(value)); + } + + public String get(Object key) { + return super.get(nonNullString(key)); + } + + public boolean containsKey(Object key) { + return super.containsKey(nonNullString(key)); + } + + public boolean containsValue(Object value) { + return super.containsValue(nonNullString(value)); + } + + public String remove(Object key) { + return super.remove(nonNullString(key)); + } + + private static class CheckedEntry implements Entry { + private final Entry e; + public CheckedEntry(Entry e) {this.e = e;} + public String getKey() { return e.getKey();} + public String getValue() { return e.getValue();} + public String setValue(String value) { + return e.setValue(validateValue(value)); + } + public String toString() { return getKey() + "=" + getValue();} + public boolean equals(Object o) {return e.equals(o);} + public int hashCode() {return e.hashCode();} + } + + private static class CheckedEntrySet extends AbstractSet> { + private final Set> s; + public CheckedEntrySet(Set> s) {this.s = s;} + public int size() {return s.size();} + public boolean isEmpty() {return s.isEmpty();} + public void clear() { s.clear();} + public Iterator> iterator() { + return new Iterator>() { + Iterator> i = s.iterator(); + public boolean hasNext() { return i.hasNext();} + public Entry next() { + return new CheckedEntry(i.next()); + } + public void remove() { i.remove();} + }; + } + private static Entry checkedEntry(Object o) { + @SuppressWarnings("unchecked") + Entry e = (Entry) o; + nonNullString(e.getKey()); + nonNullString(e.getValue()); + return e; + } + public boolean contains(Object o) {return s.contains(checkedEntry(o));} + public boolean remove(Object o) {return s.remove(checkedEntry(o));} + } + + private static class CheckedValues extends AbstractCollection { + private final Collection c; + public CheckedValues(Collection c) {this.c = c;} + public int size() {return c.size();} + public boolean isEmpty() {return c.isEmpty();} + public void clear() { c.clear();} + public Iterator iterator() {return c.iterator();} + public boolean contains(Object o) {return c.contains(nonNullString(o));} + public boolean remove(Object o) {return c.remove(nonNullString(o));} + } + + private static class CheckedKeySet extends AbstractSet { + private final Set s; + public CheckedKeySet(Set s) {this.s = s;} + public int size() {return s.size();} + public boolean isEmpty() {return s.isEmpty();} + public void clear() { s.clear();} + public Iterator iterator() {return s.iterator();} + public boolean contains(Object o) {return s.contains(nonNullString(o));} + public boolean remove(Object o) {return s.remove(nonNullString(o));} + } + + public Set keySet() { + return new CheckedKeySet(super.keySet()); + } + + public Collection values() { + return new CheckedValues(super.values()); + } + + public Set> entrySet() { + return new CheckedEntrySet(super.entrySet()); + } + + private static final class NameComparator implements Comparator { + public int compare(String s1, String s2) { + // We can't use String.compareToIgnoreCase since it + // canonicalizes to lower case, while Windows + // canonicalizes to upper case! For example, "_" should + // sort *after* "Z", not before. + int n1 = s1.length(); + int n2 = s2.length(); + int min = Math.min(n1, n2); + for (int i = 0; i < min; i++) { + char c1 = s1.charAt(i); + char c2 = s2.charAt(i); + if (c1 != c2) { + c1 = Character.toUpperCase(c1); + c2 = Character.toUpperCase(c2); + if (c1 != c2) + // No overflow because of numeric promotion + return c1 - c2; + } + } + return n1 - n2; + } + } + + private static final class EntryComparator implements Comparator> { + public int compare(Entry e1, + Entry e2) { + return nameComparator.compare(e1.getKey(), e2.getKey()); + } + } + + // Allow `=' as first char in name, e.g. =C:=C:\DIR + static final int MIN_NAME_LENGTH = 1; + + private static final NameComparator nameComparator; + private static final EntryComparator entryComparator; + private static final ProcessEnvironmentForWin32 theEnvironment; + private static final Map theUnmodifiableEnvironment; + private static final Map theCaseInsensitiveEnvironment; + + static { + nameComparator = new NameComparator(); + entryComparator = new EntryComparator(); + theEnvironment = new ProcessEnvironmentForWin32(); + theUnmodifiableEnvironment = Collections.unmodifiableMap(theEnvironment); + + theEnvironment.putAll(environmentBlock()); + + theCaseInsensitiveEnvironment = new TreeMap<>(nameComparator); + theCaseInsensitiveEnvironment.putAll(theEnvironment); + } + + private ProcessEnvironmentForWin32() { + super(); + } + + private ProcessEnvironmentForWin32(int capacity) { + super(capacity); + } + + // Only for use by System.getenv(String) + static String getenv(String name) { + // The original implementation used a native call to _wgetenv, + // but it turns out that _wgetenv is only consistent with + // GetEnvironmentStringsW (for non-ASCII) if `wmain' is used + // instead of `main', even in a process created using + // CREATE_UNICODE_ENVIRONMENT. Instead we perform the + // case-insensitive comparison ourselves. At least this + // guarantees that System.getenv().get(String) will be + // consistent with System.getenv(String). + return theCaseInsensitiveEnvironment.get(name); + } + + // Only for use by System.getenv() + static Map getenv() { + return theUnmodifiableEnvironment; + } + + // Only for use by ProcessBuilder.environment() + @SuppressWarnings("unchecked") + static Map environment() { + return (Map) theEnvironment.clone(); + } + + // Only for use by ProcessBuilder.environment(String[] envp) + static Map emptyEnvironment(int capacity) { + return new ProcessEnvironmentForWin32(capacity); + } + + private static Map environmentBlock() { + return Kernel32Util.getEnvironmentVariables(); + } + + // Only for use by ProcessImpl.start() + String toEnvironmentBlock() { + // Sort Unicode-case-insensitively by name + List> list = new ArrayList<>(entrySet()); + Collections.sort(list, entryComparator); + + StringBuilder sb = new StringBuilder(size()*30); + int cmp = -1; + + // Some versions of MSVCRT.DLL require SystemRoot to be set. + // So, we make sure that it is always set, even if not provided + // by the caller. + final String SYSTEMROOT = "SystemRoot"; + + for (Entry e : list) { + String key = e.getKey(); + String value = e.getValue(); + if (cmp < 0 && (cmp = nameComparator.compare(key, SYSTEMROOT)) > 0) { + // Not set, so add it here + addToEnvIfSet(sb, SYSTEMROOT); + } + addToEnv(sb, key, value); + } + if (cmp < 0) { + // Got to end of list and still not found + addToEnvIfSet(sb, SYSTEMROOT); + } + if (sb.length() == 0) { + // Environment was empty and SystemRoot not set in parent + sb.append('\u0000'); + } + // Block is double NUL terminated + sb.append('\u0000'); + return sb.toString(); + } + + // add the environment variable to the child, if it exists in parent + private static void addToEnvIfSet(StringBuilder sb, String name) { + String s = getenv(name); + if (s != null) + addToEnv(sb, name, s); + } + + private static void addToEnv(StringBuilder sb, String name, String val) { + sb.append(name).append('=').append(val).append('\u0000'); + } + + static String toEnvironmentBlock(Map map) { + return map == null ? null : ((ProcessEnvironmentForWin32)map).toEnvironmentBlock(); + } +} diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/process/ProcessImplForWin32.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/process/ProcessImplForWin32.java new file mode 100644 index 0000000000..4583be8aff --- /dev/null +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/process/ProcessImplForWin32.java @@ -0,0 +1,752 @@ +/* + * 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.common.utils.process; + +import com.sun.jna.Pointer; +import com.sun.jna.platform.win32.*; +import com.sun.jna.ptr.IntByReference; +import sun.security.action.GetPropertyAction; + +import java.io.*; +import java.security.AccessController; +import java.security.PrivilegedAction; +import java.util.ArrayList; +import java.util.Locale; +import java.util.concurrent.TimeUnit; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import static com.sun.jna.platform.win32.WinBase.STILL_ACTIVE; + +public class ProcessImplForWin32 extends Process { + private static final sun.misc.JavaIOFileDescriptorAccess fdAccess + = sun.misc.SharedSecrets.getJavaIOFileDescriptorAccess(); + + private static final int PIPE_SIZE = 4096 + 24; + + private static final int HANDLE_STORAGE_SIZE = 6; + + private static final int OFFSET_READ = 0; + + private static final int OFFSET_WRITE = 1; + + private static final WinNT.HANDLE JAVA_INVALID_HANDLE_VALUE = new WinNT.HANDLE(Pointer.createConstant(-1)); + + /** + * Open a file for writing. If {@code append} is {@code true} then the file + * is opened for atomic append directly and a FileOutputStream constructed + * with the resulting handle. This is because a FileOutputStream created + * to append to a file does not open the file in a manner that guarantees + * that writes by the child process will be atomic. + */ + private static FileOutputStream newFileOutputStream(File f, boolean append) + throws IOException + { + if (append) { + String path = f.getPath(); + SecurityManager sm = System.getSecurityManager(); + if (sm != null) + sm.checkWrite(path); + long handle = openForAtomicAppend(path); + final FileDescriptor fd = new FileDescriptor(); + fdAccess.setHandle(fd, handle); + return AccessController.doPrivileged( + new PrivilegedAction() { + public FileOutputStream run() { + return new FileOutputStream(fd); + } + } + ); + } else { + return new FileOutputStream(f); + } + } + + // System-dependent portion of ProcessBuilderForWindows.start() + static Process start(String username, + String password, + String cmdarray[], + java.util.Map environment, + String dir, + ProcessBuilderForWin32.Redirect[] redirects, + boolean redirectErrorStream) + throws IOException + { + String envblock = ProcessEnvironmentForWin32.toEnvironmentBlock(environment); + + FileInputStream f0 = null; + FileOutputStream f1 = null; + FileOutputStream f2 = null; + + try { + long[] stdHandles; + if (redirects == null) { + stdHandles = new long[] { -1L, -1L, -1L }; + } else { + stdHandles = new long[3]; + + if (redirects[0] == ProcessBuilderForWin32.Redirect.PIPE) + stdHandles[0] = -1L; + else if (redirects[0] == ProcessBuilderForWin32.Redirect.INHERIT) + stdHandles[0] = fdAccess.getHandle(FileDescriptor.in); + else { + f0 = new FileInputStream(redirects[0].file()); + stdHandles[0] = fdAccess.getHandle(f0.getFD()); + } + + if (redirects[1] == ProcessBuilderForWin32.Redirect.PIPE) + stdHandles[1] = -1L; + else if (redirects[1] == ProcessBuilderForWin32.Redirect.INHERIT) + stdHandles[1] = fdAccess.getHandle(FileDescriptor.out); + else { + f1 = newFileOutputStream(redirects[1].file(), + redirects[1].append()); + stdHandles[1] = fdAccess.getHandle(f1.getFD()); + } + + if (redirects[2] == ProcessBuilderForWin32.Redirect.PIPE) + stdHandles[2] = -1L; + else if (redirects[2] == ProcessBuilderForWin32.Redirect.INHERIT) + stdHandles[2] = fdAccess.getHandle(FileDescriptor.err); + else { + f2 = newFileOutputStream(redirects[2].file(), + redirects[2].append()); + stdHandles[2] = fdAccess.getHandle(f2.getFD()); + } + } + + return new ProcessImplForWin32(username, password, cmdarray, envblock, dir, stdHandles, redirectErrorStream); + } finally { + // In theory, close() can throw IOException + // (although it is rather unlikely to happen here) + try { if (f0 != null) f0.close(); } + finally { + try { if (f1 != null) f1.close(); } + finally { if (f2 != null) f2.close(); } + } + } + + } + + private static class LazyPattern { + // Escape-support version: + // "(\")((?:\\\\\\1|.)+?)\\1|([^\\s\"]+)"; + private static final Pattern PATTERN = + Pattern.compile("[^\\s\"]+|\"[^\"]*\""); + }; + + /* Parses the command string parameter into the executable name and + * program arguments. + * + * The command string is broken into tokens. The token separator is a space + * or quota character. The space inside quotation is not a token separator. + * There are no escape sequences. + */ + private static String[] getTokensFromCommand(String command) { + ArrayList matchList = new ArrayList<>(8); + Matcher regexMatcher = ProcessImplForWin32.LazyPattern.PATTERN.matcher(command); + while (regexMatcher.find()) + matchList.add(regexMatcher.group()); + return matchList.toArray(new String[matchList.size()]); + } + + private static final int VERIFICATION_CMD_BAT = 0; + private static final int VERIFICATION_WIN32 = 1; + private static final int VERIFICATION_WIN32_SAFE = 2; // inside quotes not allowed + private static final int VERIFICATION_LEGACY = 3; + // See Command shell overview for documentation of special characters. + // https://docs.microsoft.com/en-us/previous-versions/windows/it-pro/windows-xp/bb490954(v=technet.10) + private static final char ESCAPE_VERIFICATION[][] = { + // We guarantee the only command file execution for implicit [cmd.exe] run. + // http://technet.microsoft.com/en-us/library/bb490954.aspx + {' ', '\t', '<', '>', '&', '|', '^'}, + {' ', '\t', '<', '>'}, + {' ', '\t', '<', '>'}, + {' ', '\t'} + }; + + private static String createCommandLine(int verificationType, + final String executablePath, + final String cmd[]) + { + StringBuilder cmdbuf = new StringBuilder(80); + + cmdbuf.append(executablePath); + + for (int i = 1; i < cmd.length; ++i) { + cmdbuf.append(' '); + String s = cmd[i]; + if (needsEscaping(verificationType, s)) { + cmdbuf.append('"'); + + if (verificationType == VERIFICATION_WIN32_SAFE) { + // Insert the argument, adding '\' to quote any interior quotes + int length = s.length(); + for (int j = 0; j < length; j++) { + char c = s.charAt(j); + if (c == DOUBLEQUOTE) { + int count = countLeadingBackslash(verificationType, s, j); + while (count-- > 0) { + cmdbuf.append(BACKSLASH); // double the number of backslashes + } + cmdbuf.append(BACKSLASH); // backslash to quote the quote + } + cmdbuf.append(c); + } + } else { + cmdbuf.append(s); + } + // The code protects the [java.exe] and console command line + // parser, that interprets the [\"] combination as an escape + // sequence for the ["] char. + // http://msdn.microsoft.com/en-us/library/17w5ykft.aspx + // + // If the argument is an FS path, doubling of the tail [\] + // char is not a problem for non-console applications. + // + // The [\"] sequence is not an escape sequence for the [cmd.exe] + // command line parser. The case of the [""] tail escape + // sequence could not be realized due to the argument validation + // procedure. + int count = countLeadingBackslash(verificationType, s, s.length()); + while (count-- > 0) { + cmdbuf.append(BACKSLASH); // double the number of backslashes + } + cmdbuf.append('"'); + } else { + cmdbuf.append(s); + } + } + return cmdbuf.toString(); + } + + /** + * Return the argument without quotes (1st and last) if present, else the arg. + * @param str a string + * @return the string without 1st and last quotes + */ + private static String unQuote(String str) { + int len = str.length(); + return (len >= 2 && str.charAt(0) == DOUBLEQUOTE && str.charAt(len - 1) == DOUBLEQUOTE) + ? str.substring(1, len - 1) + : str; + } + + private static boolean needsEscaping(int verificationType, String arg) { + // Switch off MS heuristic for internal ["]. + // Please, use the explicit [cmd.exe] call + // if you need the internal ["]. + // Example: "cmd.exe", "/C", "Extended_MS_Syntax" + + // For [.exe] or [.com] file the unpaired/internal ["] + // in the argument is not a problem. + String unquotedArg = unQuote(arg); + boolean argIsQuoted = !arg.equals(unquotedArg); + boolean embeddedQuote = unquotedArg.indexOf(DOUBLEQUOTE) >= 0; + + switch (verificationType) { + case VERIFICATION_CMD_BAT: + if (embeddedQuote) { + throw new IllegalArgumentException("Argument has embedded quote, " + + "use the explicit CMD.EXE call."); + } + break; // break determine whether to quote + case VERIFICATION_WIN32_SAFE: + if (argIsQuoted && embeddedQuote) { + throw new IllegalArgumentException("Malformed argument has embedded quote: " + + unquotedArg); + } + break; + default: + break; + } + + if (!argIsQuoted) { + char testEscape[] = ESCAPE_VERIFICATION[verificationType]; + for (int i = 0; i < testEscape.length; ++i) { + if (arg.indexOf(testEscape[i]) >= 0) { + return true; + } + } + } + return false; + } + + private static String getExecutablePath(String path) + throws IOException + { + String name = unQuote(path); + if (name.indexOf(DOUBLEQUOTE) >= 0) { + throw new IllegalArgumentException("Executable name has embedded quote, " + + "split the arguments: " + name); + } + // Win32 CreateProcess requires path to be normalized + File fileToRun = new File(name); + + // From the [CreateProcess] function documentation: + // + // "If the file name does not contain an extension, .exe is appended. + // Therefore, if the file name extension is .com, this parameter + // must include the .com extension. If the file name ends in + // a period (.) with no extension, or if the file name contains a path, + // .exe is not appended." + // + // "If the file name !does not contain a directory path!, + // the system searches for the executable file in the following + // sequence:..." + // + // In practice ANY non-existent path is extended by [.exe] extension + // in the [CreateProcess] function with the only exception: + // the path ends by (.) + + return fileToRun.getPath(); + } + + /** + * An executable is any program that is an EXE or does not have an extension + * and the Windows createProcess will be looking for .exe. + * The comparison is case insensitive based on the name. + * @param executablePath the executable file + * @return true if the path ends in .exe or does not have an extension. + */ + private boolean isExe(String executablePath) { + File file = new File(executablePath); + String upName = file.getName().toUpperCase(Locale.ROOT); + return (upName.endsWith(".EXE") || upName.indexOf('.') < 0); + } + + // Old version that can be bypassed + private boolean isShellFile(String executablePath) { + String upPath = executablePath.toUpperCase(); + return (upPath.endsWith(".CMD") || upPath.endsWith(".BAT")); + } + + private String quoteString(String arg) { + StringBuilder argbuf = new StringBuilder(arg.length() + 2); + return argbuf.append('"').append(arg).append('"').toString(); + } + + // Count backslashes before start index of string. + // .bat files don't include backslashes as part of the quote + private static int countLeadingBackslash(int verificationType, + CharSequence input, int start) { + if (verificationType == VERIFICATION_CMD_BAT) + return 0; + int j; + for (j = start - 1; j >= 0 && input.charAt(j) == BACKSLASH; j--) { + // just scanning backwards + } + return (start - 1) - j; // number of BACKSLASHES + } + + private static final char DOUBLEQUOTE = '\"'; + private static final char BACKSLASH = '\\'; + + private WinNT.HANDLE handle; + private OutputStream stdin_stream; + private InputStream stdout_stream; + private InputStream stderr_stream; + + private ProcessImplForWin32( + String username, + String password, + String cmd[], + final String envblock, + final String path, + final long[] stdHandles, + final boolean redirectErrorStream) + throws IOException + { + String cmdstr; + final SecurityManager security = System.getSecurityManager(); + GetPropertyAction action = new GetPropertyAction("jdk.lang.Process.allowAmbiguousCommands", + (security == null) ? "true" : "false"); + final boolean allowAmbiguousCommands = !"false".equalsIgnoreCase(action.run()); + if (allowAmbiguousCommands && security == null) { + // Legacy mode. + + // Normalize path if possible. + String executablePath = new File(cmd[0]).getPath(); + + // No worry about internal, unpaired ["], and redirection/piping. + if (needsEscaping(VERIFICATION_LEGACY, executablePath) ) + executablePath = quoteString(executablePath); + + cmdstr = createCommandLine( + //legacy mode doesn't worry about extended verification + VERIFICATION_LEGACY, + executablePath, + cmd); + } else { + String executablePath; + try { + executablePath = getExecutablePath(cmd[0]); + } catch (IllegalArgumentException e) { + // Workaround for the calls like + // Runtime.getRuntime().exec("\"C:\\Program Files\\foo\" bar") + + // No chance to avoid CMD/BAT injection, except to do the work + // right from the beginning. Otherwise we have too many corner + // cases from + // Runtime.getRuntime().exec(String[] cmd [, ...]) + // calls with internal ["] and escape sequences. + + // Restore original command line. + StringBuilder join = new StringBuilder(); + // terminal space in command line is ok + for (String s : cmd) + join.append(s).append(' '); + + // Parse the command line again. + cmd = getTokensFromCommand(join.toString()); + executablePath = getExecutablePath(cmd[0]); + + // Check new executable name once more + if (security != null) + security.checkExec(executablePath); + } + + // Quotation protects from interpretation of the [path] argument as + // start of longer path with spaces. Quotation has no influence to + // [.exe] extension heuristic. + boolean isShell = allowAmbiguousCommands ? isShellFile(executablePath) + : !isExe(executablePath); + cmdstr = createCommandLine( + // We need the extended verification procedures + isShell ? VERIFICATION_CMD_BAT + : (allowAmbiguousCommands ? VERIFICATION_WIN32 : VERIFICATION_WIN32_SAFE), + quoteString(executablePath), + cmd); + } + + handle = create(username, password, cmdstr, envblock, path, stdHandles, redirectErrorStream); + + AccessController.doPrivileged( + new PrivilegedAction() { + public Void run() { + if (stdHandles[0] == -1L) + stdin_stream = ProcessBuilderForWin32.NullOutputStream.INSTANCE; + else { + FileDescriptor stdin_fd = new FileDescriptor(); + fdAccess.setHandle(stdin_fd, stdHandles[0]); + stdin_stream = new BufferedOutputStream( + new FileOutputStream(stdin_fd)); + } + + if (stdHandles[1] == -1L) + stdout_stream = ProcessBuilderForWin32.NullInputStream.INSTANCE; + else { + FileDescriptor stdout_fd = new FileDescriptor(); + fdAccess.setHandle(stdout_fd, stdHandles[1]); + stdout_stream = new BufferedInputStream( + new FileInputStream(stdout_fd)); + } + + if (stdHandles[2] == -1L) + stderr_stream = ProcessBuilderForWin32.NullInputStream.INSTANCE; + else { + FileDescriptor stderr_fd = new FileDescriptor(); + fdAccess.setHandle(stderr_fd, stdHandles[2]); + stderr_stream = new FileInputStream(stderr_fd); + } + + return null; }}); + } + + public OutputStream getOutputStream() { + return stdin_stream; + } + + public InputStream getInputStream() { + return stdout_stream; + } + + public InputStream getErrorStream() { + return stderr_stream; + } + + protected void finalize() { + closeHandle(handle); + } + + public int exitValue() { + int exitCode = getExitCodeProcess(handle); + if (exitCode == STILL_ACTIVE) + throw new IllegalThreadStateException("process has not exited"); + return exitCode; + } + + public int waitFor() throws InterruptedException { + waitForInterruptibly(handle); + if (Thread.interrupted()) + throw new InterruptedException(); + return exitValue(); + } + + @Override + public boolean waitFor(long timeout, TimeUnit unit) + throws InterruptedException + { + if (getExitCodeProcess(handle) != STILL_ACTIVE) return true; + if (timeout <= 0) return false; + + long remainingNanos = unit.toNanos(timeout); + long deadline = System.nanoTime() + remainingNanos ; + + do { + // Round up to next millisecond + long msTimeout = TimeUnit.NANOSECONDS.toMillis(remainingNanos + 999_999L); + waitForTimeoutInterruptibly(handle, msTimeout); + if (Thread.interrupted()) + throw new InterruptedException(); + if (getExitCodeProcess(handle) != STILL_ACTIVE) { + return true; + } + remainingNanos = deadline - System.nanoTime(); + } while (remainingNanos > 0); + + return (getExitCodeProcess(handle) != STILL_ACTIVE); + } + + public void destroy() { terminateProcess(handle); } + + public Process destroyForcibly() { + destroy(); + return this; + } + + public boolean isAlive() { + return isProcessAlive(handle); + } + + private static boolean initHolder(WinNT.HANDLEByReference pjhandles, + WinNT.HANDLEByReference[] pipe, + int offset, + WinNT.HANDLEByReference phStd) { + if (!pjhandles.getValue().equals(JAVA_INVALID_HANDLE_VALUE)) { + phStd.setValue(pjhandles.getValue()); + pjhandles.setValue(JAVA_INVALID_HANDLE_VALUE); + } else { + if (!Kernel32.INSTANCE.CreatePipe(pipe[0], pipe[1], null, PIPE_SIZE)) { + throw new Win32Exception(Kernel32.INSTANCE.GetLastError()); + } else { + WinNT.HANDLE thisProcessEnd = offset == OFFSET_READ ? pipe[1].getValue() : pipe[0].getValue(); + phStd.setValue(pipe[offset].getValue()); + pjhandles.setValue(thisProcessEnd); + } + } + Kernel32.INSTANCE.SetHandleInformation(phStd.getValue(), Kernel32.HANDLE_FLAG_INHERIT, Kernel32.HANDLE_FLAG_INHERIT); + return true; + } + + private static void releaseHolder(boolean complete, WinNT.HANDLEByReference[] pipe, int offset) { + closeHandle(pipe[offset].getValue()); + if (complete) { + closeHandle(pipe[offset == OFFSET_READ ? OFFSET_WRITE : OFFSET_READ].getValue()); + } + } + + private static void prepareIOEHandleState(WinNT.HANDLE[] stdIOE, Boolean[] inherit) { + for(int i = 0; i < HANDLE_STORAGE_SIZE; ++i) { + WinNT.HANDLE hstd = stdIOE[i]; + if (!Kernel32.INVALID_HANDLE_VALUE.equals(hstd)) { + inherit[i] = Boolean.TRUE; + Kernel32.INSTANCE.SetHandleInformation(hstd, Kernel32.HANDLE_FLAG_INHERIT, 0); + } + } + } + + private static void restoreIOEHandleState(WinNT.HANDLE[] stdIOE, Boolean[] inherit) { + for (int i = HANDLE_STORAGE_SIZE - 1; i >= 0; --i) { + if (!Kernel32.INVALID_HANDLE_VALUE.equals(stdIOE[i])) { + Kernel32.INSTANCE.SetHandleInformation(stdIOE[i], Kernel32.HANDLE_FLAG_INHERIT, inherit[i] ? Kernel32.HANDLE_FLAG_INHERIT : 0); + } + } + } + + private static WinNT.HANDLE processCreate(String username, + String password, + String cmd, + final String envblock, + final String path, + final WinNT.HANDLEByReference[] stdHandles, + final boolean redirectErrorStream) { + WinNT.HANDLE ret = new WinNT.HANDLE(Pointer.createConstant(0)); + + WinNT.HANDLE[] stdIOE = new WinNT.HANDLE[] { + Kernel32.INVALID_HANDLE_VALUE, Kernel32.INVALID_HANDLE_VALUE, Kernel32.INVALID_HANDLE_VALUE, + stdHandles[0].getValue(), stdHandles[1].getValue(), stdHandles[2].getValue() + }; + stdIOE[0] = Kernel32.INSTANCE.GetStdHandle(Kernel32.STD_INPUT_HANDLE); + stdIOE[1] = Kernel32.INSTANCE.GetStdHandle(Kernel32.STD_OUTPUT_HANDLE); + stdIOE[2] = Kernel32.INSTANCE.GetStdHandle(Kernel32.STD_ERROR_HANDLE); + + Boolean[] inherit = new Boolean[] { + Boolean.FALSE, Boolean.FALSE, Boolean.FALSE, + Boolean.FALSE, Boolean.FALSE, Boolean.FALSE + }; + + prepareIOEHandleState(stdIOE, inherit); + + // input + WinNT.HANDLEByReference hStdInput = new WinNT.HANDLEByReference(); + WinNT.HANDLEByReference[] pipeIn = new WinNT.HANDLEByReference[] { + new WinNT.HANDLEByReference(Kernel32.INVALID_HANDLE_VALUE), new WinNT.HANDLEByReference(Kernel32.INVALID_HANDLE_VALUE) }; + + // output + WinNT.HANDLEByReference hStdOutput = new WinNT.HANDLEByReference(); + WinNT.HANDLEByReference[] pipeOut = new WinNT.HANDLEByReference[] { + new WinNT.HANDLEByReference(Kernel32.INVALID_HANDLE_VALUE), new WinNT.HANDLEByReference(Kernel32.INVALID_HANDLE_VALUE) }; + + // error + WinNT.HANDLEByReference hStdError = new WinNT.HANDLEByReference(); + WinNT.HANDLEByReference[] pipeError = new WinNT.HANDLEByReference[] { + new WinNT.HANDLEByReference(Kernel32.INVALID_HANDLE_VALUE), new WinNT.HANDLEByReference(Kernel32.INVALID_HANDLE_VALUE) }; + + boolean success; + if (initHolder(stdHandles[0], pipeIn, OFFSET_READ, hStdInput)) { + if (initHolder(stdHandles[1], pipeOut, OFFSET_WRITE, hStdOutput)) { + WinBase.STARTUPINFO si = new WinBase.STARTUPINFO(); + si.hStdInput = hStdInput.getValue(); + si.hStdOutput = hStdOutput.getValue(); + + if (redirectErrorStream) { + si.hStdError = si.hStdOutput; + stdHandles[2].setValue(JAVA_INVALID_HANDLE_VALUE); + success = true; + } else { + success = initHolder(stdHandles[2], pipeError, OFFSET_WRITE, hStdError); + si.hStdError = hStdError.getValue(); + } + + if (success) { + WTypes.LPSTR lpEnvironment = envblock == null ? new WTypes.LPSTR() : new WTypes.LPSTR(envblock); + Kernel32.PROCESS_INFORMATION pi = new WinBase.PROCESS_INFORMATION(); + si.dwFlags = Kernel32.STARTF_USESTDHANDLES; + if (!Advapi32.INSTANCE.CreateProcessWithLogonW( + username + , null + , password + , Advapi32.LOGON_WITH_PROFILE + , null + , cmd + , Kernel32.CREATE_NO_WINDOW + , lpEnvironment.getPointer() + , path + , si + , pi)) { + throw new Win32Exception(Kernel32.INSTANCE.GetLastError()); + } else { + closeHandle(pi.hThread); + ret = pi.hProcess; + } + } + releaseHolder(ret.getPointer().equals(Pointer.createConstant(0)), pipeError, OFFSET_WRITE); + releaseHolder(ret.getPointer().equals(Pointer.createConstant(0)), pipeOut, OFFSET_WRITE); + } + releaseHolder(ret.getPointer().equals(Pointer.createConstant(0)), pipeIn, OFFSET_READ); + } + restoreIOEHandleState(stdIOE, inherit); + return ret; + } + + private static synchronized WinNT.HANDLE create(String username, + String password, + String cmd, + final String envblock, + final String path, + final long[] stdHandles, + final boolean redirectErrorStream) { + WinNT.HANDLE ret = new WinNT.HANDLE(Pointer.createConstant(0)); + WinNT.HANDLEByReference[] handles = new WinNT.HANDLEByReference[stdHandles.length]; + for (int i = 0; i < stdHandles.length; i++) { + handles[i] = new WinNT.HANDLEByReference(new WinNT.HANDLE(Pointer.createConstant(stdHandles[i]))); + } + + if (cmd != null) { + if (username != null && password != null) { + ret = processCreate(username, password, cmd, envblock, path, handles, redirectErrorStream); + } + } + + for (int i = 0; i < stdHandles.length; i++) { + stdHandles[i] = handles[i].getPointer().getLong(0); + } + + return ret; + } + + private static int getExitCodeProcess(WinNT.HANDLE handle) { + IntByReference exitStatus = new IntByReference(); + if (!Kernel32.INSTANCE.GetExitCodeProcess(handle, exitStatus)) { + throw new Win32Exception(Kernel32.INSTANCE.GetLastError()); + } + return exitStatus.getValue(); + } + + private static void terminateProcess(WinNT.HANDLE handle) { + Kernel32.INSTANCE.TerminateProcess(handle, 1); + } + + private static boolean isProcessAlive(WinNT.HANDLE handle) { + IntByReference exitStatus = new IntByReference(); + Kernel32.INSTANCE.GetExitCodeProcess(handle, exitStatus); + return exitStatus.getValue() == STILL_ACTIVE; + } + + private static void closeHandle(WinNT.HANDLE handle) { + Kernel32Util.closeHandle(handle); + } + + /** + * Opens a file for atomic append. The file is created if it doesn't + * already exist. + * + * @param path the file to open or create + * @return the native HANDLE + */ + private static long openForAtomicAppend(String path) throws IOException { + int access = Kernel32.GENERIC_READ | Kernel32.GENERIC_WRITE; + int sharing = Kernel32.FILE_SHARE_READ | Kernel32.FILE_SHARE_WRITE; + int disposition = Kernel32.OPEN_ALWAYS; + int flagsAndAttributes = Kernel32.FILE_ATTRIBUTE_NORMAL; + if (path == null || path.isEmpty()) { + return -1; + } else { + WinNT.HANDLE handle = Kernel32.INSTANCE.CreateFile(path, access, sharing, null, disposition, flagsAndAttributes, null); + if (handle == Kernel32.INVALID_HANDLE_VALUE) { + throw new Win32Exception(Kernel32.INSTANCE.GetLastError()); + } + return handle.getPointer().getLong(0); + } + } + + private static void waitForInterruptibly(WinNT.HANDLE handle) { + int result = Kernel32.INSTANCE.WaitForMultipleObjects(1, new WinNT.HANDLE[]{handle}, false, Kernel32.INFINITE); + if (result == Kernel32.WAIT_FAILED) { + throw new Win32Exception(Kernel32.INSTANCE.GetLastError()); + } + } + + private static void waitForTimeoutInterruptibly(WinNT.HANDLE handle, long timeout) { + int result = Kernel32.INSTANCE.WaitForMultipleObjects(1, new WinNT.HANDLE[]{handle}, false, (int) timeout); + if (result == Kernel32.WAIT_FAILED) { + throw new Win32Exception(Kernel32.INSTANCE.GetLastError()); + } + } + +} diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/ConstantsTest.java b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/ConstantsTest.java new file mode 100644 index 0000000000..3280a9629f --- /dev/null +++ b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/ConstantsTest.java @@ -0,0 +1,40 @@ +/* + * 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.common; + +import org.apache.dolphinscheduler.common.utils.OSUtils; +import org.junit.Assert; +import org.junit.Test; + +/** + * Constants Test + */ +public class ConstantsTest { + + /** + * Test PID via env + */ + @Test + public void testPID() { + if (OSUtils.isWindows()) { + Assert.assertEquals(Constants.PID, "handle"); + } else { + Assert.assertEquals(Constants.PID, "pid"); + } + } + +} diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/OSUtilsTest.java b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/OSUtilsTest.java index 7106804aaf..b955787c69 100644 --- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/OSUtilsTest.java +++ b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/OSUtilsTest.java @@ -39,16 +39,20 @@ public class OSUtilsTest { @Test public void testOSMetric(){ - double availablePhysicalMemorySize = OSUtils.availablePhysicalMemorySize(); - Assert.assertTrue(availablePhysicalMemorySize > 0.0f); - double totalMemorySize = OSUtils.totalMemorySize(); - Assert.assertTrue(totalMemorySize > 0.0f); - double loadAverage = OSUtils.loadAverage(); - logger.info("loadAverage {}", loadAverage); - double memoryUsage = OSUtils.memoryUsage(); - Assert.assertTrue(memoryUsage > 0.0f); - double cpuUsage = OSUtils.cpuUsage(); - Assert.assertTrue(cpuUsage > 0.0f); + if (!OSUtils.isWindows()) { + double availablePhysicalMemorySize = OSUtils.availablePhysicalMemorySize(); + Assert.assertTrue(availablePhysicalMemorySize > 0.0f); + double totalMemorySize = OSUtils.totalMemorySize(); + Assert.assertTrue(totalMemorySize > 0.0f); + double loadAverage = OSUtils.loadAverage(); + logger.info("loadAverage {}", loadAverage); + double memoryUsage = OSUtils.memoryUsage(); + Assert.assertTrue(memoryUsage > 0.0f); + double cpuUsage = OSUtils.cpuUsage(); + Assert.assertTrue(cpuUsage > 0.0f); + } else { + // TODO window ut + } } @Test diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/process/ProcessBuilderForWin32Test.java b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/process/ProcessBuilderForWin32Test.java new file mode 100644 index 0000000000..ce04346743 --- /dev/null +++ b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/process/ProcessBuilderForWin32Test.java @@ -0,0 +1,210 @@ +/* + * 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.common.utils.process; + +import org.apache.dolphinscheduler.common.utils.OSUtils; +import org.apache.dolphinscheduler.common.utils.StringUtils; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.*; +import java.nio.charset.Charset; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +@RunWith(PowerMockRunner.class) +@PrepareForTest(OSUtils.class) +public class ProcessBuilderForWin32Test { + + private static final Logger logger = LoggerFactory.getLogger(ProcessBuilderForWin32Test.class); + + @Before + public void before() { + PowerMockito.mockStatic(OSUtils.class); + PowerMockito.when(OSUtils.isWindows()).thenReturn(true); + } + + @Test + public void testCreateProcessBuilderForWin32() { + try { + ProcessBuilderForWin32 builder = new ProcessBuilderForWin32(); + Assert.assertNotNull(builder); + + builder = new ProcessBuilderForWin32("net"); + Assert.assertNotNull(builder); + + builder = new ProcessBuilderForWin32(Collections.singletonList("net")); + Assert.assertNotNull(builder); + + builder = new ProcessBuilderForWin32((List) null); + Assert.assertNotNull(builder); + } catch (Error | Exception e) { + logger.error(e.getMessage()); + } + } + + @Test + public void testBuildUser() { + try { + ProcessBuilderForWin32 builder = new ProcessBuilderForWin32(); + builder.user("test", StringUtils.EMPTY); + Assert.assertNotNull(builder); + } catch (Error | Exception e) { + logger.error(e.getMessage()); + } + } + + @Test + public void testBuildCommand() { + try { + ProcessBuilderForWin32 builder = new ProcessBuilderForWin32(); + builder.command(Collections.singletonList("net")); + Assert.assertNotEquals(0, builder.command().size()); + + builder = new ProcessBuilderForWin32(); + builder.command("net"); + Assert.assertNotEquals(0, builder.command().size()); + + builder = new ProcessBuilderForWin32(); + builder.command((List) null); + Assert.assertNotEquals(0, builder.command().size()); + } catch (Error | Exception e) { + logger.error(e.getMessage()); + } + } + + @Test + public void testEnvironment() { + try { + ProcessBuilderForWin32 builder = new ProcessBuilderForWin32(); + Assert.assertNotNull(builder.environment()); + } catch (Error | Exception e) { + logger.error(e.getMessage()); + } + + try { + ProcessBuilderForWin32 builder = new ProcessBuilderForWin32(); + builder.environment(new String[]{ "a=123" }); + Assert.assertNotEquals(0, builder.environment().size()); + } catch (Error | Exception e) { + logger.error(e.getMessage()); + } + } + + @Test + public void testDirectory() { + try { + ProcessBuilderForWin32 builder = new ProcessBuilderForWin32(); + builder.directory(new File("/tmp")); + Assert.assertNotNull(builder.directory()); + } catch (Error | Exception e) { + logger.error(e.getMessage()); + } + } + + @Test + public void testStream() { + try { + InputStream in = ProcessBuilderForWin32.NullInputStream.INSTANCE; + Assert.assertNotNull(in); + Assert.assertEquals(-1, in.read()); + Assert.assertEquals(0, in.available()); + + OutputStream out = ProcessBuilderForWin32.NullOutputStream.INSTANCE; + Assert.assertNotNull(out); + out.write(new byte[] {1}); + } catch (Exception e) { + logger.error(e.getMessage()); + } + } + + @Test + public void testRedirect() { + try { + ProcessBuilderForWin32 builder = new ProcessBuilderForWin32(); + + builder.redirectInput(new File("/tmp")); + Assert.assertNotNull(builder.redirectInput()); + Assert.assertNotNull(builder.redirectInput().file()); + + builder.redirectOutput(new File("/tmp")); + Assert.assertNotNull(builder.redirectOutput()); + Assert.assertNotNull(builder.redirectOutput().file()); + + builder.redirectError(new File("/tmp")); + Assert.assertNotNull(builder.redirectError()); + Assert.assertNotNull(builder.redirectError().file()); + + builder.redirectInput(builder.redirectOutput()); + builder.redirectOutput(builder.redirectInput()); + builder.redirectError(builder.redirectInput()); + + Assert.assertNotNull(ProcessBuilderForWin32.Redirect.PIPE.type()); + Assert.assertNotNull(ProcessBuilderForWin32.Redirect.PIPE.toString()); + Assert.assertNotNull(ProcessBuilderForWin32.Redirect.INHERIT.type()); + Assert.assertNotNull(ProcessBuilderForWin32.Redirect.INHERIT.toString()); + } catch (Error | Exception e) { + logger.error(e.getMessage()); + } + } + + @Test + public void testRedirectErrorStream() { + try { + ProcessBuilderForWin32 builder = new ProcessBuilderForWin32(); + builder.redirectErrorStream(true); + Assert.assertTrue(builder.redirectErrorStream()); + } catch (Error | Exception e) { + logger.error(e.getMessage()); + } + } + + @Test + public void runCmdViaUser() { + try { + ProcessBuilderForWin32 builder = new ProcessBuilderForWin32(); + builder.user("test123", StringUtils.EMPTY); + + List commands = new ArrayList<>(); + commands.add("cmd.exe"); + commands.add("/c"); + commands.add("net user"); + builder.command(commands); + + Process process = builder.start(); + BufferedReader inReader = new BufferedReader(new InputStreamReader(process.getInputStream(), Charset.forName("GBK"))); + String line; + StringBuilder sb = new StringBuilder(); + while ((line = inReader.readLine()) != null) { + sb.append(line); + } + logger.info("net user: {}", sb.toString()); + Assert.assertNotEquals(StringUtils.EMPTY, sb.toString()); + } catch (Error | Exception e) { + logger.error(e.getMessage()); + } + } + +} diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/process/ProcessEnvironmentForWin32Test.java b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/process/ProcessEnvironmentForWin32Test.java new file mode 100644 index 0000000000..00c54c0164 --- /dev/null +++ b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/process/ProcessEnvironmentForWin32Test.java @@ -0,0 +1,124 @@ +/* + * 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.common.utils.process; + +import org.apache.dolphinscheduler.common.utils.OSUtils; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.Map; +import java.util.Set; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({OSUtils.class, ProcessEnvironmentForWin32.class}) +public class ProcessEnvironmentForWin32Test { + + private static final Logger logger = LoggerFactory.getLogger(ProcessBuilderForWin32Test.class); + + @Before + public void before() { + try { + PowerMockito.mockStatic(OSUtils.class); + PowerMockito.when(OSUtils.isWindows()).thenReturn(true); + } catch (Error | Exception e) { + logger.error(e.getMessage()); + } + } + + @Test + public void testPutAndGet() { + try { + ProcessEnvironmentForWin32 processEnvironmentForWin32 = (ProcessEnvironmentForWin32) ProcessEnvironmentForWin32.emptyEnvironment(0); + processEnvironmentForWin32.put("a", "123"); + Assert.assertEquals("123", processEnvironmentForWin32.get("a")); + Assert.assertTrue(processEnvironmentForWin32.containsKey("a")); + Assert.assertTrue(processEnvironmentForWin32.containsValue("123")); + Assert.assertEquals("123", processEnvironmentForWin32.remove("a")); + } catch (Error | Exception e) { + logger.error(e.getMessage()); + } + + try { + ProcessEnvironmentForWin32 processEnvironmentForWin32 = (ProcessEnvironmentForWin32) ProcessEnvironmentForWin32.emptyEnvironment(0); + processEnvironmentForWin32.put("b=", "123"); + } catch (Error | Exception e) { + logger.error(e.getMessage()); + } + + try { + ProcessEnvironmentForWin32 processEnvironmentForWin32 = (ProcessEnvironmentForWin32) ProcessEnvironmentForWin32.emptyEnvironment(0); + processEnvironmentForWin32.put("b", "\u0000"); + } catch (Error | Exception e) { + logger.error(e.getMessage()); + } + + try { + ProcessEnvironmentForWin32 processEnvironmentForWin32 = (ProcessEnvironmentForWin32) ProcessEnvironmentForWin32.emptyEnvironment(0); + processEnvironmentForWin32.get(null); + } catch (Error | Exception e) { + logger.error(e.getMessage()); + } + } + + @Test + public void testEntrySet() { + try { + ProcessEnvironmentForWin32 processEnvironmentForWin32 = (ProcessEnvironmentForWin32) ProcessEnvironmentForWin32.emptyEnvironment(0); + processEnvironmentForWin32.clear(); + processEnvironmentForWin32.put("a", "123"); + Assert.assertEquals(0, processEnvironmentForWin32.entrySet().size()); + Assert.assertTrue(processEnvironmentForWin32.entrySet().isEmpty()); + for (Map.Entry entry : processEnvironmentForWin32.entrySet()) { + Assert.assertNotNull(entry); + Assert.assertNotNull(entry.getKey()); + Assert.assertNotNull(entry.getValue()); + Assert.assertNotNull(entry.setValue("123")); + } + + processEnvironmentForWin32.clear(); + Set keys = processEnvironmentForWin32.keySet(); + Assert.assertEquals(0, keys.size()); + Assert.assertTrue(keys.isEmpty()); + + processEnvironmentForWin32.clear(); + Collection values = processEnvironmentForWin32.values(); + Assert.assertEquals(0, keys.size()); + Assert.assertTrue(keys.isEmpty()); + } catch (Error | Exception e) { + logger.error(e.getMessage()); + } + } + + @Test + public void testToEnvironmentBlock() { + try { + ProcessEnvironmentForWin32 processEnvironmentForWin32 = (ProcessEnvironmentForWin32) ProcessEnvironmentForWin32.emptyEnvironment(0); + Assert.assertNotNull(processEnvironmentForWin32.toEnvironmentBlock()); + } catch (Error | Exception e) { + logger.error(e.getMessage()); + } + } + +} diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/process/ProcessImplForWin32Test.java b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/process/ProcessImplForWin32Test.java new file mode 100644 index 0000000000..3f8bcbfb66 --- /dev/null +++ b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/process/ProcessImplForWin32Test.java @@ -0,0 +1,70 @@ +/* + * 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.common.utils.process; + +import org.apache.dolphinscheduler.common.utils.OSUtils; +import org.apache.dolphinscheduler.common.utils.StringUtils; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import sun.security.action.GetPropertyAction; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({OSUtils.class, GetPropertyAction.class}) +public class ProcessImplForWin32Test { + + private static final Logger logger = LoggerFactory.getLogger(ProcessBuilderForWin32Test.class); + + @Before + public void before() { + PowerMockito.mockStatic(OSUtils.class); + PowerMockito.mockStatic(GetPropertyAction.class); + PowerMockito.when(OSUtils.isWindows()).thenReturn(true); + } + + @Test + public void testStart() { + try { + Process process = ProcessImplForWin32.start( + "test123", StringUtils.EMPTY, new String[]{"net"}, + null, null, null, false); + Assert.assertNotNull(process); + } catch (Error | Exception e) { + logger.error(e.getMessage()); + } + + try { + Process process = ProcessImplForWin32.start( + "test123", StringUtils.EMPTY, new String[]{"net"}, + null, null, new ProcessBuilderForWin32.Redirect[]{ + ProcessBuilderForWin32.Redirect.PIPE, + ProcessBuilderForWin32.Redirect.PIPE, + ProcessBuilderForWin32.Redirect.PIPE + }, false); + Assert.assertNotNull(process); + } catch (Error | Exception e) { + logger.error(e.getMessage()); + } + } + +} diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractCommandExecutor.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractCommandExecutor.java index bac498c150..8e0ccee16c 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractCommandExecutor.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/AbstractCommandExecutor.java @@ -16,21 +16,29 @@ */ package org.apache.dolphinscheduler.server.worker.task; +import com.sun.jna.platform.win32.Kernel32; +import com.sun.jna.platform.win32.WinNT; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.thread.ThreadUtils; import org.apache.dolphinscheduler.common.utils.HadoopUtils; +import org.apache.dolphinscheduler.common.utils.LoggerUtils; +import org.apache.dolphinscheduler.common.utils.OSUtils; import org.apache.dolphinscheduler.common.utils.StringUtils; +import org.apache.dolphinscheduler.common.utils.process.ProcessBuilderForWin32; import org.apache.dolphinscheduler.dao.entity.TaskInstance; -import org.apache.dolphinscheduler.common.utils.LoggerUtils; import org.apache.dolphinscheduler.server.utils.ProcessUtils; import org.apache.dolphinscheduler.service.process.ProcessService; import org.slf4j.Logger; import java.io.*; import java.lang.reflect.Field; +import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; -import java.util.*; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Date; +import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import java.util.function.Consumer; @@ -194,26 +202,49 @@ public abstract class AbstractCommandExecutor { * @throws IOException IO Exception */ private void buildProcess(String commandFile) throws IOException { + // command list + List command = new ArrayList<>(); + //init process builder - ProcessBuilder processBuilder = new ProcessBuilder(); - // setting up a working directory - processBuilder.directory(new File(taskDir)); - // merge error information to standard output stream - processBuilder.redirectErrorStream(true); - // setting up user to run commands - List command = new LinkedList<>(); - command.add("sudo"); - command.add("-u"); - command.add(tenantCode); - command.add(commandInterpreter()); - command.addAll(commandOptions()); - command.add(commandFile); - processBuilder.command(command); - - process = processBuilder.start(); + if (OSUtils.isWindows()) { + ProcessBuilderForWin32 processBuilder = new ProcessBuilderForWin32(); + // setting up a working directory + processBuilder.directory(new File(taskDir)); + processBuilder.user(tenantCode, StringUtils.EMPTY); + // merge error information to standard output stream + processBuilder.redirectErrorStream(true); + + // setting up user to run commands + command.add(commandInterpreter()); + command.add("/c"); + command.addAll(commandOptions()); + command.add(commandFile); + + // setting commands + processBuilder.command(command); + process = processBuilder.start(); + } else { + ProcessBuilder processBuilder = new ProcessBuilder(); + // setting up a working directory + processBuilder.directory(new File(taskDir)); + // merge error information to standard output stream + processBuilder.redirectErrorStream(true); + + // setting up user to run commands + command.add("sudo"); + command.add("-u"); + command.add(tenantCode); + command.add(commandInterpreter()); + command.addAll(commandOptions()); + command.add(commandFile); + + // setting commands + processBuilder.command(command); + process = processBuilder.start(); + } // print command - printCommand(processBuilder); + printCommand(command); } /** @@ -320,13 +351,13 @@ public abstract class AbstractCommandExecutor { /** * print command - * @param processBuilder process builder + * @param command command */ - private void printCommand(ProcessBuilder processBuilder) { + private void printCommand(List command) { String cmdStr; try { - cmdStr = ProcessUtils.buildCommandStr(processBuilder.command()); + cmdStr = ProcessUtils.buildCommandStr(command); logger.info("task run command:\n{}", cmdStr); } catch (IOException e) { logger.error(e.getMessage(), e); @@ -358,7 +389,11 @@ public abstract class AbstractCommandExecutor { BufferedReader inReader = null; try { - inReader = new BufferedReader(new InputStreamReader(process.getInputStream())); + if (OSUtils.isWindows()) { + inReader = new BufferedReader(new InputStreamReader(process.getInputStream(), Charset.forName("GBK"))); + } else { + inReader = new BufferedReader(new InputStreamReader(process.getInputStream())); + } String line; long lastFlushTime = System.currentTimeMillis(); @@ -406,7 +441,7 @@ public abstract class AbstractCommandExecutor { } Thread.sleep(Constants.SLEEP_TIME_MILLIS); } - } + } } catch (Exception e) { logger.error("yarn applications: {} status failed ", appIds,e); result = false; @@ -510,12 +545,15 @@ public abstract class AbstractCommandExecutor { */ private int getProcessId(Process process) { int processId = 0; - try { Field f = process.getClass().getDeclaredField(Constants.PID); f.setAccessible(true); - - processId = f.getInt(process); + if (OSUtils.isWindows()) { + WinNT.HANDLE handle = (WinNT.HANDLE) f.get(process); + processId = Kernel32.INSTANCE.GetProcessId(handle); + } else { + processId = f.getInt(process); + } } catch (Throwable e) { logger.error(e.getMessage(), e); } diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/ShellCommandExecutor.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/ShellCommandExecutor.java index db46d0d856..5d14e6b2a7 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/ShellCommandExecutor.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/ShellCommandExecutor.java @@ -17,11 +17,12 @@ package org.apache.dolphinscheduler.server.worker.task; import org.apache.commons.io.FileUtils; +import org.apache.dolphinscheduler.common.utils.OSUtils; import org.slf4j.Logger; import java.io.File; import java.io.IOException; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Paths; import java.util.Date; @@ -34,10 +35,15 @@ import java.util.function.Consumer; public class ShellCommandExecutor extends AbstractCommandExecutor { /** - * sh + * For Unix-like, using sh */ public static final String SH = "sh"; + /** + * For Windows, using cmd.exe + */ + public static final String CMD = "cmd.exe"; + /** * constructor * @param logHandler log handler @@ -66,7 +72,7 @@ public class ShellCommandExecutor extends AbstractCommandExecutor { @Override protected String buildCommandFilePath() { // command file - return String.format("%s/%s.command", taskDir, taskAppId); + return String.format("%s/%s.%s", taskDir, taskAppId, OSUtils.isWindows() ? "bat" : "command"); } /** @@ -75,7 +81,7 @@ public class ShellCommandExecutor extends AbstractCommandExecutor { */ @Override protected String commandInterpreter() { - return SH; + return OSUtils.isWindows() ? CMD : SH; } /** @@ -103,21 +109,26 @@ public class ShellCommandExecutor extends AbstractCommandExecutor { logger.info("create command file:{}", commandFile); StringBuilder sb = new StringBuilder(); - sb.append("#!/bin/sh\n"); - sb.append("BASEDIR=$(cd `dirname $0`; pwd)\n"); - sb.append("cd $BASEDIR\n"); - - if (envFile != null) { - sb.append("source " + envFile + "\n"); + if (OSUtils.isWindows()) { + sb.append("@echo off\n"); + sb.append("cd /d %~dp0\n"); + if (envFile != null) { + sb.append("call ").append(envFile).append("\n"); + } + } else { + sb.append("#!/bin/sh\n"); + sb.append("BASEDIR=$(cd `dirname $0`; pwd)\n"); + sb.append("cd $BASEDIR\n"); + if (envFile != null) { + sb.append("source ").append(envFile).append("\n"); + } } - sb.append("\n\n"); sb.append(execCommand); - logger.info("command : {}",sb.toString()); + logger.info("command : {}", sb.toString()); // write data to file - FileUtils.writeStringToFile(new File(commandFile), sb.toString(), - Charset.forName("UTF-8")); + FileUtils.writeStringToFile(new File(commandFile), sb.toString(), StandardCharsets.UTF_8); } } diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java index ef941cd062..7537ca2edc 100755 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java @@ -44,6 +44,7 @@ import org.apache.dolphinscheduler.common.task.AbstractParameters; import org.apache.dolphinscheduler.common.task.datax.DataxParameters; import org.apache.dolphinscheduler.common.utils.CollectionUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils; +import org.apache.dolphinscheduler.common.utils.OSUtils; import org.apache.dolphinscheduler.common.utils.ParameterUtils; import org.apache.dolphinscheduler.dao.datasource.BaseDataSource; import org.apache.dolphinscheduler.dao.datasource.DataSourceFactory; @@ -338,7 +339,7 @@ public class DataxTask extends AbstractTask { private String buildShellCommandFile(String jobConfigFilePath) throws Exception { // generate scripts - String fileName = String.format("%s/%s_node.sh", taskDir, taskProps.getTaskAppId()); + String fileName = String.format("%s/%s_node.%s", taskDir, taskProps.getTaskAppId(), OSUtils.isWindows() ? "bat" : "sh"); Path path = new File(fileName).toPath(); if (Files.exists(path)) { @@ -370,7 +371,13 @@ public class DataxTask extends AbstractTask { // create shell command file Set perms = PosixFilePermissions.fromString(Constants.RWXR_XR_X); FileAttribute> attr = PosixFilePermissions.asFileAttribute(perms); - Files.createFile(path, attr); + + if (OSUtils.isWindows()) { + Files.createFile(path); + } else { + Files.createFile(path, attr); + } + Files.write(path, dataxCommand.getBytes(), StandardOpenOption.APPEND); return fileName; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTask.java index 5704c8052e..90661a690a 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTask.java @@ -22,6 +22,7 @@ import org.apache.dolphinscheduler.common.process.Property; import org.apache.dolphinscheduler.common.task.AbstractParameters; import org.apache.dolphinscheduler.common.task.shell.ShellParameters; import org.apache.dolphinscheduler.common.utils.JSONUtils; +import org.apache.dolphinscheduler.common.utils.OSUtils; import org.apache.dolphinscheduler.common.utils.ParameterUtils; import org.apache.dolphinscheduler.server.utils.ParamUtils; import org.apache.dolphinscheduler.server.worker.task.AbstractTask; @@ -123,7 +124,7 @@ public class ShellTask extends AbstractTask { */ private String buildCommand() throws Exception { // generate scripts - String fileName = String.format("%s/%s_node.sh", taskDir, taskProps.getTaskAppId()); + String fileName = String.format("%s/%s_node.%s", taskDir, taskProps.getTaskAppId(), OSUtils.isWindows() ? "bat" : "sh"); Path path = new File(fileName).toPath(); if (Files.exists(path)) { @@ -154,7 +155,11 @@ public class ShellTask extends AbstractTask { Set perms = PosixFilePermissions.fromString(Constants.RWXR_XR_X); FileAttribute> attr = PosixFilePermissions.asFileAttribute(perms); - Files.createFile(path, attr); + if (OSUtils.isWindows()) { + Files.createFile(path); + } else { + Files.createFile(path, attr); + } Files.write(path, shellParameters.getRawScript().getBytes(), StandardOpenOption.APPEND); diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/utils/ProcessUtilsTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/utils/ProcessUtilsTest.java index 77fc398702..1e0adaad9b 100644 --- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/utils/ProcessUtilsTest.java +++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/utils/ProcessUtilsTest.java @@ -21,7 +21,12 @@ import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + public class ProcessUtilsTest { + private static final Logger logger = LoggerFactory.getLogger(ProcessUtilsTest.class); @Test @@ -30,4 +35,16 @@ public class ProcessUtilsTest { Assert.assertNotEquals("The child process of process 1 should not be empty", pidList, ""); logger.info("Sub process list : {}", pidList); } + + @Test + public void testBuildCommandStr() { + List commands = new ArrayList<>(); + commands.add("sudo"); + try { + Assert.assertEquals(ProcessUtils.buildCommandStr(commands), "sudo"); + } catch (IOException e) { + Assert.fail(e.getMessage()); + } + } + } diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTaskTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTaskTest.java new file mode 100644 index 0000000000..5536665e26 --- /dev/null +++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTaskTest.java @@ -0,0 +1,198 @@ +/* + * 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.server.worker.task.shell; + +import org.apache.dolphinscheduler.common.enums.CommandType; +import org.apache.dolphinscheduler.common.enums.DbType; +import org.apache.dolphinscheduler.common.utils.OSUtils; +import org.apache.dolphinscheduler.dao.entity.DataSource; +import org.apache.dolphinscheduler.dao.entity.ProcessInstance; +import org.apache.dolphinscheduler.server.worker.task.ShellCommandExecutor; +import org.apache.dolphinscheduler.server.worker.task.TaskProps; +import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; +import org.apache.dolphinscheduler.service.process.ProcessService; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.context.ApplicationContext; + +import java.util.Date; + +/** + * shell task test + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest(OSUtils.class) +public class ShellTaskTest { + + private static final Logger logger = LoggerFactory.getLogger(ShellTaskTest.class); + + private ShellTask shellTask; + + private ProcessService processService; + + private ShellCommandExecutor shellCommandExecutor; + + private ApplicationContext applicationContext; + + @Before + public void before() throws Exception { + PowerMockito.mockStatic(OSUtils.class); + processService = PowerMockito.mock(ProcessService.class); + shellCommandExecutor = PowerMockito.mock(ShellCommandExecutor.class); + + applicationContext = PowerMockito.mock(ApplicationContext.class); + SpringApplicationContext springApplicationContext = new SpringApplicationContext(); + springApplicationContext.setApplicationContext(applicationContext); + PowerMockito.when(applicationContext.getBean(ProcessService.class)).thenReturn(processService); + + TaskProps props = new TaskProps(); + props.setTaskDir("/tmp"); + props.setTaskAppId(String.valueOf(System.currentTimeMillis())); + props.setTaskInstId(1); + props.setTenantCode("1"); + props.setEnvFile(".dolphinscheduler_env.sh"); + props.setTaskStartTime(new Date()); + props.setTaskTimeout(0); + props.setTaskParams("{\"rawScript\": \" echo 'hello world!'\"}"); + shellTask = new ShellTask(props, logger); + shellTask.init(); + + PowerMockito.when(processService.findDataSourceById(1)).thenReturn(getDataSource()); + PowerMockito.when(processService.findDataSourceById(2)).thenReturn(getDataSource()); + PowerMockito.when(processService.findProcessInstanceByTaskId(1)).thenReturn(getProcessInstance()); + + String fileName = String.format("%s/%s_node.%s", props.getTaskDir(), props.getTaskAppId(), OSUtils.isWindows() ? "bat" : "sh"); + PowerMockito.when(shellCommandExecutor.run(fileName, processService)).thenReturn(0); + } + + private DataSource getDataSource() { + DataSource dataSource = new DataSource(); + dataSource.setType(DbType.MYSQL); + dataSource.setConnectionParams( + "{\"user\":\"root\",\"password\":\"123456\",\"address\":\"jdbc:mysql://127.0.0.1:3306\",\"database\":\"test\",\"jdbcUrl\":\"jdbc:mysql://127.0.0.1:3306/test\"}"); + dataSource.setUserId(1); + return dataSource; + } + + private ProcessInstance getProcessInstance() { + ProcessInstance processInstance = new ProcessInstance(); + processInstance.setCommandType(CommandType.START_PROCESS); + processInstance.setScheduleTime(new Date()); + return processInstance; + } + + @After + public void after() {} + + /** + * Method: ShellTask() + */ + @Test + public void testShellTask() + throws Exception { + TaskProps props = new TaskProps(); + props.setTaskDir("/tmp"); + props.setTaskAppId(String.valueOf(System.currentTimeMillis())); + props.setTaskInstId(1); + props.setTenantCode("1"); + ShellTask shellTaskTest = new ShellTask(props, logger); + Assert.assertNotNull(shellTaskTest); + } + + /** + * Method: init for Unix-like + */ + @Test + public void testInitForUnix() { + try { + PowerMockito.when(OSUtils.isWindows()).thenReturn(false); + shellTask.init(); + Assert.assertTrue(true); + } catch (Error | Exception e) { + logger.error(e.getMessage()); + } + } + + /** + * Method: init for Windows + */ + @Test + public void testInitForWindows() { + try { + PowerMockito.when(OSUtils.isWindows()).thenReturn(true); + shellTask.init(); + Assert.assertTrue(true); + } catch (Error | Exception e) { + logger.error(e.getMessage()); + } + } + + /** + * Method: handle() for Unix-like + */ + @Test + public void testHandleForUnix() throws Exception { + try { + PowerMockito.when(OSUtils.isWindows()).thenReturn(false); + shellTask.handle(); + Assert.assertTrue(true); + } catch (Error | Exception e) { + if (!e.getMessage().contains("process error . exitCode is : -1") + && !System.getProperty("os.name").startsWith("Windows")) { + logger.error(e.getMessage()); + } + } + } + + /** + * Method: handle() for Windows + */ + @Test + public void testHandleForWindows() throws Exception { + try { + PowerMockito.when(OSUtils.isWindows()).thenReturn(true); + shellTask.handle(); + Assert.assertTrue(true); + } catch (Error | Exception e) { + if (!e.getMessage().contains("process error . exitCode is : -1")) { + logger.error(e.getMessage()); + } + } + } + + /** + * Method: cancelApplication() + */ + @Test + public void testCancelApplication() throws Exception { + try { + shellTask.cancelApplication(true); + Assert.assertTrue(true); + } catch (Error | Exception e) { + logger.error(e.getMessage()); + } + } + +} diff --git a/pom.xml b/pom.xml index dee1dce8b2..a7feec0e14 100644 --- a/pom.xml +++ b/pom.xml @@ -684,6 +684,9 @@ **/common/utils/*.java + **/common/utils/process/ProcessBuilderForWin32Test.java + **/common/utils/process/ProcessEnvironmentForWin32Test.java + **/common/utils/process/ProcessImplForWin32Test.java **/common/log/*.java **/common/threadutils/*.java **/common/graph/*.java @@ -732,6 +735,7 @@ **/alert/template/AlertTemplateFactoryTest.java **/alert/template/impl/DefaultHTMLTemplateTest.java **/server/worker/task/datax/DataxTaskTest.java + **/server/worker/task/shell/ShellTaskTest.java **/server/worker/task/sqoop/SqoopTaskTest.java **/server/utils/DataxUtilsTest.java **/service/zk/DefaultEnsembleProviderTest.java From 4bb97dfb8b1c4aaee8e61a8d12793b1c337ee5f1 Mon Sep 17 00:00:00 2001 From: dailidong Date: Thu, 5 Mar 2020 23:53:12 +0800 Subject: [PATCH 065/221] fix actions/checkout@v2 problem (#2095) --- .github/workflows/ci_backend.yml | 12 ++++++++++++ .github/workflows/ci_e2e.yml | 6 ++++++ .github/workflows/ci_frontend.yml | 12 ++++++++++++ .github/workflows/ci_ut.yml | 6 ++++++ 4 files changed, 36 insertions(+) diff --git a/.github/workflows/ci_backend.yml b/.github/workflows/ci_backend.yml index 1ca15c2813..0273251e99 100644 --- a/.github/workflows/ci_backend.yml +++ b/.github/workflows/ci_backend.yml @@ -46,6 +46,12 @@ jobs: runs-on: ubuntu-latest steps: - uses: actions/checkout@v2 + # In the checkout@v2, it doesn't support git submodule. Execute the commands manually. + - name: checkout submodules + shell: bash + run: | + git submodule sync --recursive + git -c protocol.version=2 submodule update --init --force --recursive --depth=1 - name: Set up JDK 1.8 uses: actions/setup-java@v1 with: @@ -56,6 +62,12 @@ jobs: runs-on: ubuntu-latest steps: - uses: actions/checkout@v2 + # In the checkout@v2, it doesn't support git submodule. Execute the commands manually. + - name: checkout submodules + shell: bash + run: | + git submodule sync --recursive + git -c protocol.version=2 submodule update --init --force --recursive --depth=1 - name: Set up JDK 1.8 uses: actions/setup-java@v1 with: diff --git a/.github/workflows/ci_e2e.yml b/.github/workflows/ci_e2e.yml index 7d9c5e3e72..fe818d0668 100644 --- a/.github/workflows/ci_e2e.yml +++ b/.github/workflows/ci_e2e.yml @@ -30,6 +30,12 @@ jobs: steps: - uses: actions/checkout@v2 + # In the checkout@v2, it doesn't support git submodule. Execute the commands manually. + - name: checkout submodules + shell: bash + run: | + git submodule sync --recursive + git -c protocol.version=2 submodule update --init --force --recursive --depth=1 - uses: actions/cache@v1 with: path: ~/.m2/repository diff --git a/.github/workflows/ci_frontend.yml b/.github/workflows/ci_frontend.yml index 1f959b40eb..494d12dbae 100644 --- a/.github/workflows/ci_frontend.yml +++ b/.github/workflows/ci_frontend.yml @@ -35,6 +35,12 @@ jobs: os: [ubuntu-latest, macos-latest] steps: - uses: actions/checkout@v2 + # In the checkout@v2, it doesn't support git submodule. Execute the commands manually. + - name: checkout submodules + shell: bash + run: | + git submodule sync --recursive + git -c protocol.version=2 submodule update --init --force --recursive --depth=1 - name: Set up Node.js uses: actions/setup-node@v1 with: @@ -50,6 +56,12 @@ jobs: runs-on: ubuntu-latest steps: - uses: actions/checkout@v2 + # In the checkout@v2, it doesn't support git submodule. Execute the commands manually. + - name: checkout submodules + shell: bash + run: | + git submodule sync --recursive + git -c protocol.version=2 submodule update --init --force --recursive --depth=1 - name: Set up JDK 1.8 uses: actions/setup-java@v1 with: diff --git a/.github/workflows/ci_ut.yml b/.github/workflows/ci_ut.yml index e7e47437d4..48cd1e16ce 100644 --- a/.github/workflows/ci_ut.yml +++ b/.github/workflows/ci_ut.yml @@ -30,6 +30,12 @@ jobs: steps: - uses: actions/checkout@v2 + # In the checkout@v2, it doesn't support git submodule. Execute the commands manually. + - name: checkout submodules + shell: bash + run: | + git submodule sync --recursive + git -c protocol.version=2 submodule update --init --force --recursive --depth=1 - uses: actions/cache@v1 with: path: ~/.m2/repository From 0e1dd8d7d95b945c8982f18b802c88b216cf0179 Mon Sep 17 00:00:00 2001 From: "gabry.wu" Date: Sat, 7 Mar 2020 18:07:53 +0800 Subject: [PATCH 066/221] remove incorrect or unnecessary URL tag (#2099) --- dolphinscheduler-common/pom.xml | 2 +- dolphinscheduler-dao/pom.xml | 2 +- dolphinscheduler-remote/pom.xml | 2 -- dolphinscheduler-server/pom.xml | 2 +- 4 files changed, 3 insertions(+), 5 deletions(-) diff --git a/dolphinscheduler-common/pom.xml b/dolphinscheduler-common/pom.xml index e7789f724b..ca75a84a62 100644 --- a/dolphinscheduler-common/pom.xml +++ b/dolphinscheduler-common/pom.xml @@ -25,7 +25,7 @@ dolphinscheduler-common dolphinscheduler-common - http://maven.apache.org + jar UTF-8 diff --git a/dolphinscheduler-dao/pom.xml b/dolphinscheduler-dao/pom.xml index fd43458f87..3aea888f94 100644 --- a/dolphinscheduler-dao/pom.xml +++ b/dolphinscheduler-dao/pom.xml @@ -25,7 +25,7 @@ dolphinscheduler-dao ${project.artifactId} - http://maven.apache.org + UTF-8 diff --git a/dolphinscheduler-remote/pom.xml b/dolphinscheduler-remote/pom.xml index b67b033ffa..39c7c6a7c0 100644 --- a/dolphinscheduler-remote/pom.xml +++ b/dolphinscheduler-remote/pom.xml @@ -12,8 +12,6 @@ dolphinscheduler-remote dolphinscheduler-remote - - http://www.example.com UTF-8 diff --git a/dolphinscheduler-server/pom.xml b/dolphinscheduler-server/pom.xml index 86490197b6..e8e84297e2 100644 --- a/dolphinscheduler-server/pom.xml +++ b/dolphinscheduler-server/pom.xml @@ -25,7 +25,7 @@ dolphinscheduler-server dolphinscheduler-server - http://maven.apache.org + jar UTF-8 From 89b42db3b4da98295c898d4243577ef1e0a0291d Mon Sep 17 00:00:00 2001 From: liwenhe1993 <32166572+liwenhe1993@users.noreply.github.com> Date: Sun, 8 Mar 2020 15:56:24 +0800 Subject: [PATCH 067/221] Add docker internal server check script (#2092) * add environment variables add checkpoint.sh file to docker image 1. add `POSTGRESQL_DATABASE` environment variable 2. add `DOLPHINSCHEDULER_DATA_BASEDIR_PATH` environment variable 3. add `DOLPHINSCHEDULER_DATA_DOWNLOAD_BASEDIR_PATH` environment variable 4. add `DOLPHINSCHEDULER_PROCESS_EXEC_BASEPATH` environment variable 5. add checkpoint.sh file to check server process 6. modify `README.md` and `README_zh_CN.md` * add environment variables add checkpoint.sh file to docker image 1. add `POSTGRESQL_DATABASE` environment variable 2. add `DOLPHINSCHEDULER_DATA_BASEDIR_PATH` environment variable 3. add `DOLPHINSCHEDULER_DATA_DOWNLOAD_BASEDIR_PATH` environment variable 4. add `DOLPHINSCHEDULER_PROCESS_EXEC_BASEPATH` environment variable 5. add checkpoint.sh file to check server process 6. modify `README.md` and `README_zh_CN.md` * add logger port and modify dolphinscheduler_env to dolphinscheduler_env.sh --- dockerfile/Dockerfile | 13 +++--- dockerfile/README.md | 40 ++++++++++++++----- dockerfile/README_zh_CN.md | 38 ++++++++++++++---- dockerfile/checkpoint.sh | 27 +++++++++++++ .../application.properties.tpl | 2 +- .../dolphinscheduler/common.properties.tpl | 12 +++--- ...nscheduler_env => dolphinscheduler_env.sh} | 0 .../dolphinscheduler/quartz.properties.tpl | 2 +- dockerfile/startup-init-conf.sh | 4 ++ 9 files changed, 108 insertions(+), 30 deletions(-) create mode 100644 dockerfile/checkpoint.sh rename dockerfile/conf/dolphinscheduler/env/{dolphinscheduler_env => dolphinscheduler_env.sh} (100%) diff --git a/dockerfile/Dockerfile b/dockerfile/Dockerfile index 1fc064c489..8fa4886f31 100644 --- a/dockerfile/Dockerfile +++ b/dockerfile/Dockerfile @@ -63,19 +63,22 @@ RUN echo "daemon off;" >> /etc/nginx/nginx.conf && \ ADD ./conf/nginx/dolphinscheduler.conf /etc/nginx/conf.d #7. add configuration and modify permissions and set soft links +ADD ./checkpoint.sh /root/checkpoint.sh ADD ./startup-init-conf.sh /root/startup-init-conf.sh ADD ./startup.sh /root/startup.sh ADD ./conf/dolphinscheduler/*.tpl /opt/dolphinscheduler/conf/ -ADD ./conf/dolphinscheduler/env/dolphinscheduler_env /opt/dolphinscheduler/conf/env/ -RUN chmod +x /root/startup-init-conf.sh && \ +ADD conf/dolphinscheduler/env/dolphinscheduler_env.sh /opt/dolphinscheduler/conf/env/ +RUN chmod +x /root/checkpoint.sh && \ + chmod +x /root/startup-init-conf.sh && \ chmod +x /root/startup.sh && \ - chmod +x /opt/dolphinscheduler/conf/env/dolphinscheduler_env && \ + chmod +x /opt/dolphinscheduler/conf/env/dolphinscheduler_env.sh && \ chmod +x /opt/dolphinscheduler/script/*.sh && \ chmod +x /opt/dolphinscheduler/bin/*.sh && \ chmod +x /opt/zookeeper/bin/*.sh && \ + dos2unix /root/checkpoint.sh && \ dos2unix /root/startup-init-conf.sh && \ dos2unix /root/startup.sh && \ - dos2unix /opt/dolphinscheduler/conf/env/dolphinscheduler_env && \ + dos2unix /opt/dolphinscheduler/conf/env/dolphinscheduler_env.sh && \ dos2unix /opt/dolphinscheduler/script/*.sh && \ dos2unix /opt/dolphinscheduler/bin/*.sh && \ dos2unix /opt/zookeeper/bin/*.sh && \ @@ -87,6 +90,6 @@ RUN chmod +x /root/startup-init-conf.sh && \ RUN rm -rf /var/cache/apk/* #9. expose port -EXPOSE 2181 2888 3888 5432 12345 8888 +EXPOSE 2181 2888 3888 5432 12345 50051 8888 ENTRYPOINT ["/root/startup.sh"] \ No newline at end of file diff --git a/dockerfile/README.md b/dockerfile/README.md index b5a9d0d3aa..60af7fad8f 100644 --- a/dockerfile/README.md +++ b/dockerfile/README.md @@ -16,7 +16,7 @@ Official Website: https://dolphinscheduler.apache.org #### You can start a dolphinscheduler instance ``` $ docker run -dit --name dolphinscheduler \ --e POSTGRESQL_USERNAME=test -e POSTGRESQL_PASSWORD=test \ +-e POSTGRESQL_USERNAME=test -e POSTGRESQL_PASSWORD=test -e POSTGRESQL_DATABASE=dolphinscheduler \ -p 8888:8888 \ dolphinscheduler all ``` @@ -25,13 +25,13 @@ The default postgres user `root`, postgres password `root` and database `dolphin The default zookeeper is created in the `startup.sh`. -#### Or via Environment Variables **`POSTGRESQL_HOST`** **`POSTGRESQL_PORT`** **`ZOOKEEPER_QUORUM`** +#### Or via Environment Variables **`POSTGRESQL_HOST`** **`POSTGRESQL_PORT`** **`POSTGRESQL_DATABASE`** **`ZOOKEEPER_QUORUM`** You can specify **existing postgres service**. Example: ``` $ docker run -dit --name dolphinscheduler \ --e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" \ +-e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" -e POSTGRESQL_DATABASE="dolphinscheduler" \ -e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ -p 8888:8888 \ dolphinscheduler all @@ -42,7 +42,7 @@ You can specify **existing zookeeper service**. Example: ``` $ docker run -dit --name dolphinscheduler \ -e ZOOKEEPER_QUORUM="l92.168.x.x:2181" --e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ +-e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" -e POSTGRESQL_DATABASE="dolphinscheduler" \ -p 8888:8888 \ dolphinscheduler all ``` @@ -56,7 +56,7 @@ You can start a standalone dolphinscheduler server. ``` $ docker run -dit --name dolphinscheduler \ -e ZOOKEEPER_QUORUM="l92.168.x.x:2181" --e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" \ +-e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" -e POSTGRESQL_DATABASE="dolphinscheduler" \ -e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ dolphinscheduler master-server ``` @@ -66,7 +66,7 @@ dolphinscheduler master-server ``` $ docker run -dit --name dolphinscheduler \ -e ZOOKEEPER_QUORUM="l92.168.x.x:2181" --e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" \ +-e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" -e POSTGRESQL_DATABASE="dolphinscheduler" \ -e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ dolphinscheduler worker-server ``` @@ -75,7 +75,7 @@ dolphinscheduler worker-server ``` $ docker run -dit --name dolphinscheduler \ --e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" \ +-e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" -e POSTGRESQL_DATABASE="dolphinscheduler" \ -e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ -p 12345:12345 \ dolphinscheduler api-server @@ -85,7 +85,7 @@ dolphinscheduler api-server ``` $ docker run -dit --name dolphinscheduler \ --e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" \ +-e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" -e POSTGRESQL_DATABASE="dolphinscheduler" \ -e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ dolphinscheduler alert-server ``` @@ -99,7 +99,7 @@ $ docker run -dit --name dolphinscheduler \ dolphinscheduler frontend ``` -**Note**: You must be specify `POSTGRESQL_HOST` `POSTGRESQL_PORT` `ZOOKEEPER_QUORUM` when start a standalone dolphinscheduler server. +**Note**: You must be specify `POSTGRESQL_HOST` `POSTGRESQL_PORT` `POSTGRESQL_DATABASE` `POSTGRESQL_USERNAME` `POSTGRESQL_PASSWORD` `ZOOKEEPER_QUORUM` when start a standalone dolphinscheduler server. ## How to build a docker image @@ -140,14 +140,36 @@ This environment variable sets the port for PostgreSQL. The default value is `54 This environment variable sets the username for PostgreSQL. The default value is `root`. +**Note**: You must be specify it when start a standalone dolphinscheduler server. Like `master-server`, `worker-server`, `api-server`, `alert-server`. + **`POSTGRESQL_PASSWORD`** This environment variable sets the password for PostgreSQL. The default value is `root`. +**Note**: You must be specify it when start a standalone dolphinscheduler server. Like `master-server`, `worker-server`, `api-server`, `alert-server`. + +**`POSTGRESQL_DATABASE`** + +This environment variable sets the database for PostgreSQL. The default value is `dolphinscheduler`. + +**Note**: You must be specify it when start a standalone dolphinscheduler server. Like `master-server`, `worker-server`, `api-server`, `alert-server`. + **`DOLPHINSCHEDULER_ENV_PATH`** This environment variable sets the runtime environment for task. The default value is `/opt/dolphinscheduler/conf/env/dolphinscheduler_env.sh`. +**`DOLPHINSCHEDULER_DATA_BASEDIR_PATH`** + +User data directory path, self configuration, please make sure the directory exists and have read write permissions. The default value is `/tmp/dolphinscheduler` + +**`DOLPHINSCHEDULER_DATA_DOWNLOAD_BASEDIR_PATH`** + +Directory path for user data download. self configuration, please make sure the directory exists and have read write permissions. The default value is `/tmp/dolphinscheduler/download` + +**`DOLPHINSCHEDULER_PROCESS_EXEC_BASEPATH`** + +Process execute directory. self configuration, please make sure the directory exists and have read write permissions. The default value is `/tmp/dolphinscheduler/exec` + **`TASK_QUEUE`** This environment variable sets the task queue for `master-server` and `worker-serverr`. The default value is `zookeeper`. diff --git a/dockerfile/README_zh_CN.md b/dockerfile/README_zh_CN.md index 5830e30b07..900c8b50d9 100644 --- a/dockerfile/README_zh_CN.md +++ b/dockerfile/README_zh_CN.md @@ -16,7 +16,7 @@ Official Website: https://dolphinscheduler.apache.org #### 你可以运行一个dolphinscheduler实例 ``` $ docker run -dit --name dolphinscheduler \ --e POSTGRESQL_USERNAME=test -e POSTGRESQL_PASSWORD=test \ +-e POSTGRESQL_USERNAME=test -e POSTGRESQL_PASSWORD=test -e POSTGRESQL_DATABASE=dolphinscheduler \ -p 8888:8888 \ dolphinscheduler all ``` @@ -31,7 +31,7 @@ dolphinscheduler all ``` $ docker run -dit --name dolphinscheduler \ --e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" \ +-e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" -e POSTGRESQL_DATABASE="dolphinscheduler" \ -e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ -p 8888:8888 \ dolphinscheduler all @@ -42,7 +42,7 @@ dolphinscheduler all ``` $ docker run -dit --name dolphinscheduler \ -e ZOOKEEPER_QUORUM="l92.168.x.x:2181" --e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ +-e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" -e POSTGRESQL_DATABASE="dolphinscheduler" \ -p 8888:8888 \ dolphinscheduler all ``` @@ -56,7 +56,7 @@ dolphinscheduler all ``` $ docker run -dit --name dolphinscheduler \ -e ZOOKEEPER_QUORUM="l92.168.x.x:2181" --e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" \ +-e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" -e POSTGRESQL_DATABASE="dolphinscheduler" \ -e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ dolphinscheduler master-server ``` @@ -66,7 +66,7 @@ dolphinscheduler master-server ``` $ docker run -dit --name dolphinscheduler \ -e ZOOKEEPER_QUORUM="l92.168.x.x:2181" --e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" \ +-e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" -e POSTGRESQL_DATABASE="dolphinscheduler" \ -e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ dolphinscheduler worker-server ``` @@ -75,7 +75,7 @@ dolphinscheduler worker-server ``` $ docker run -dit --name dolphinscheduler \ --e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" \ +-e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" -e POSTGRESQL_DATABASE="dolphinscheduler" \ -e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ -p 12345:12345 \ dolphinscheduler api-server @@ -85,7 +85,7 @@ dolphinscheduler api-server ``` $ docker run -dit --name dolphinscheduler \ --e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" \ +-e POSTGRESQL_HOST="192.168.x.x" -e POSTGRESQL_PORT="5432" -e POSTGRESQL_DATABASE="dolphinscheduler" \ -e POSTGRESQL_USERNAME="test" -e POSTGRESQL_PASSWORD="test" \ dolphinscheduler alert-server ``` @@ -99,7 +99,7 @@ $ docker run -dit --name dolphinscheduler \ dolphinscheduler frontend ``` -**注意**: 当你运行dolphinscheduler中的部分服务时,你必须指定这些环境变量 `POSTGRESQL_HOST` `POSTGRESQL_PORT` `ZOOKEEPER_QUORUM`。 +**注意**: 当你运行dolphinscheduler中的部分服务时,你必须指定这些环境变量 `POSTGRESQL_HOST` `POSTGRESQL_PORT` `POSTGRESQL_DATABASE` `POSTGRESQL_USERNAME` `POSTGRESQL_PASSWORD` `ZOOKEEPER_QUORUM`。 ## 如何构建一个docker镜像 @@ -140,14 +140,36 @@ Dolphin Scheduler映像使用了几个容易遗漏的环境变量。虽然这些 配置`PostgreSQL`的`USERNAME`, 默认值 `root`。 +**注意**: 当运行`dolphinscheduler`中`master-server`、`worker-server`、`api-server`、`alert-server`这些服务时,必须指定这个环境变量,以便于你更好的搭建分布式服务。 + **`POSTGRESQL_PASSWORD`** 配置`PostgreSQL`的`PASSWORD`, 默认值 `root`。 +**注意**: 当运行`dolphinscheduler`中`master-server`、`worker-server`、`api-server`、`alert-server`这些服务时,必须指定这个环境变量,以便于你更好的搭建分布式服务。 + +**`POSTGRESQL_DATABASE`** + +配置`PostgreSQL`的`DATABASE`, 默认值 `dolphinscheduler`。 + +**注意**: 当运行`dolphinscheduler`中`master-server`、`worker-server`、`api-server`、`alert-server`这些服务时,必须指定这个环境变量,以便于你更好的搭建分布式服务。 + **`DOLPHINSCHEDULER_ENV_PATH`** 任务执行时的环境变量配置文件, 默认值 `/opt/dolphinscheduler/conf/env/dolphinscheduler_env.sh`。 +**`DOLPHINSCHEDULER_DATA_BASEDIR_PATH`** + +用户数据目录, 用户自己配置, 请确保这个目录存在并且用户读写权限, 默认值 `/tmp/dolphinscheduler`。 + +**`DOLPHINSCHEDULER_DATA_DOWNLOAD_BASEDIR_PATH`** + +用户数据下载目录, 用户自己配置, 请确保这个目录存在并且用户读写权限, 默认值 `/tmp/dolphinscheduler/download`。 + +**`DOLPHINSCHEDULER_PROCESS_EXEC_BASEPATH`** + +任务执行目录, 用户自己配置, 请确保这个目录存在并且用户读写权限, 默认值 `/tmp/dolphinscheduler/exec`。 + **`TASK_QUEUE`** 配置`master-server`和`worker-serverr`的`Zookeeper`任务队列名, 默认值 `zookeeper`。 diff --git a/dockerfile/checkpoint.sh b/dockerfile/checkpoint.sh new file mode 100644 index 0000000000..cd2774f9ce --- /dev/null +++ b/dockerfile/checkpoint.sh @@ -0,0 +1,27 @@ +#!/bin/bash +# +# 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. +# + +set -e + +if [ "$(ps -ef | grep java | grep -c $1)" -eq 0 ]; then + echo "[ERROR] $1 process not exits." + exit 1 +else + echo "[INFO] $1 process exits." + exit 0 +fi diff --git a/dockerfile/conf/dolphinscheduler/application.properties.tpl b/dockerfile/conf/dolphinscheduler/application.properties.tpl index c643c414cd..6dd8a18e11 100644 --- a/dockerfile/conf/dolphinscheduler/application.properties.tpl +++ b/dockerfile/conf/dolphinscheduler/application.properties.tpl @@ -19,7 +19,7 @@ spring.datasource.type=com.alibaba.druid.pool.DruidDataSource # postgre spring.datasource.driver-class-name=org.postgresql.Driver -spring.datasource.url=jdbc:postgresql://${POSTGRESQL_HOST}:${POSTGRESQL_PORT}/dolphinscheduler?characterEncoding=utf8 +spring.datasource.url=jdbc:postgresql://${POSTGRESQL_HOST}:${POSTGRESQL_PORT}/${POSTGRESQL_DATABASE}?characterEncoding=utf8 # mysql #spring.datasource.driver-class-name=com.mysql.jdbc.Driver #spring.datasource.url=jdbc:mysql://192.168.xx.xx:3306/dolphinscheduler?useUnicode=true&characterEncoding=UTF-8 diff --git a/dockerfile/conf/dolphinscheduler/common.properties.tpl b/dockerfile/conf/dolphinscheduler/common.properties.tpl index ea03e0b78e..8134fc7a9b 100644 --- a/dockerfile/conf/dolphinscheduler/common.properties.tpl +++ b/dockerfile/conf/dolphinscheduler/common.properties.tpl @@ -38,6 +38,12 @@ dolphinscheduler.env.path=${DOLPHINSCHEDULER_ENV_PATH} resource.view.suffixs=txt,log,sh,conf,cfg,py,java,sql,hql,xml,properties # is development state? default "false" development.state=true +# user data directory path, self configuration, please make sure the directory exists and have read write permissions +data.basedir.path=${DOLPHINSCHEDULER_DATA_BASEDIR_PATH} +# directory path for user data download. self configuration, please make sure the directory exists and have read write permissions +data.download.basedir.path=${DOLPHINSCHEDULER_DATA_DOWNLOAD_BASEDIR_PATH} +# process execute directory. self configuration, please make sure the directory exists and have read write permissions +process.exec.basepath=${DOLPHINSCHEDULER_PROCESS_EXEC_BASEPATH} # resource upload startup type : HDFS,S3,NONE res.upload.startup.type=NONE @@ -49,12 +55,6 @@ res.upload.startup.type=NONE hdfs.root.user=hdfs # data base dir, resource file will store to this hadoop hdfs path, self configuration, please make sure the directory exists on hdfs and have read write permissions。"/dolphinscheduler" is recommended data.store2hdfs.basepath=/dolphinscheduler -# user data directory path, self configuration, please make sure the directory exists and have read write permissions -data.basedir.path=/tmp/dolphinscheduler -# directory path for user data download. self configuration, please make sure the directory exists and have read write permissions -data.download.basedir.path=/tmp/dolphinscheduler/download -# process execute directory. self configuration, please make sure the directory exists and have read write permissions -process.exec.basepath=/tmp/dolphinscheduler/exec # whether kerberos starts hadoop.security.authentication.startup.state=false # java.security.krb5.conf path diff --git a/dockerfile/conf/dolphinscheduler/env/dolphinscheduler_env b/dockerfile/conf/dolphinscheduler/env/dolphinscheduler_env.sh similarity index 100% rename from dockerfile/conf/dolphinscheduler/env/dolphinscheduler_env rename to dockerfile/conf/dolphinscheduler/env/dolphinscheduler_env.sh diff --git a/dockerfile/conf/dolphinscheduler/quartz.properties.tpl b/dockerfile/conf/dolphinscheduler/quartz.properties.tpl index de5496bc3d..7c7c92e425 100644 --- a/dockerfile/conf/dolphinscheduler/quartz.properties.tpl +++ b/dockerfile/conf/dolphinscheduler/quartz.properties.tpl @@ -22,7 +22,7 @@ org.quartz.jobStore.driverDelegateClass = org.quartz.impl.jdbcjobstore.PostgreSQLDelegate # postgre org.quartz.dataSource.myDs.driver = org.postgresql.Driver -org.quartz.dataSource.myDs.URL = jdbc:postgresql://${POSTGRESQL_HOST}:${POSTGRESQL_PORT}/dolphinscheduler?characterEncoding=utf8 +org.quartz.dataSource.myDs.URL = jdbc:postgresql://${POSTGRESQL_HOST}:${POSTGRESQL_PORT}/${POSTGRESQL_DATABASE}?characterEncoding=utf8 org.quartz.dataSource.myDs.user = ${POSTGRESQL_USERNAME} org.quartz.dataSource.myDs.password = ${POSTGRESQL_PASSWORD} org.quartz.scheduler.instanceName = DolphinScheduler diff --git a/dockerfile/startup-init-conf.sh b/dockerfile/startup-init-conf.sh index db37976168..d2b50fa3e7 100644 --- a/dockerfile/startup-init-conf.sh +++ b/dockerfile/startup-init-conf.sh @@ -28,11 +28,15 @@ export POSTGRESQL_HOST=${POSTGRESQL_HOST:-"127.0.0.1"} export POSTGRESQL_PORT=${POSTGRESQL_PORT:-"5432"} export POSTGRESQL_USERNAME=${POSTGRESQL_USERNAME:-"root"} export POSTGRESQL_PASSWORD=${POSTGRESQL_PASSWORD:-"root"} +export POSTGRESQL_DATABASE=${POSTGRESQL_DATABASE:-"dolphinscheduler"} #============================================================================ # System #============================================================================ export DOLPHINSCHEDULER_ENV_PATH=${DOLPHINSCHEDULER_ENV_PATH:-"/opt/dolphinscheduler/conf/env/dolphinscheduler_env.sh"} +export DOLPHINSCHEDULER_DATA_BASEDIR_PATH=${DOLPHINSCHEDULER_DATA_BASEDIR_PATH:-"/tmp/dolphinscheduler"} +export DOLPHINSCHEDULER_DATA_DOWNLOAD_BASEDIR_PATH=${DOLPHINSCHEDULER_DATA_DOWNLOAD_BASEDIR_PATH:-"/tmp/dolphinscheduler/download"} +export DOLPHINSCHEDULER_PROCESS_EXEC_BASEPATH=${DOLPHINSCHEDULER_PROCESS_EXEC_BASEPATH:-"/tmp/dolphinscheduler/exec"} #============================================================================ # Zookeeper From b2268b1af9bdab3162a4c6378c85a936ce00827f Mon Sep 17 00:00:00 2001 From: tswstarplanet Date: Sun, 8 Mar 2020 15:57:06 +0800 Subject: [PATCH 068/221] use stringbuilder to append string (#2108) --- .../service/zk/AbstractZKClient.java | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/AbstractZKClient.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/AbstractZKClient.java index 135bfdabc6..fa1a0bfced 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/AbstractZKClient.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/zk/AbstractZKClient.java @@ -65,14 +65,16 @@ public abstract class AbstractZKClient extends ZookeeperCachedOperator { if (splits.length != Constants.HEARTBEAT_FOR_ZOOKEEPER_INFO_LENGTH){ return; } - String str = splits[0] + Constants.COMMA - + splits[1] + Constants.COMMA - + OSUtils.cpuUsage() + Constants.COMMA - + OSUtils.memoryUsage() + Constants.COMMA - + OSUtils.loadAverage() + Constants.COMMA - + splits[5] + Constants.COMMA - + DateUtils.dateToString(new Date()); - zkClient.setData().forPath(znode,str.getBytes()); + StringBuilder sb = new StringBuilder(); + sb.append(splits[0]).append(Constants.COMMA) + .append(splits[1]).append(Constants.COMMA) + .append(OSUtils.cpuUsage()).append(Constants.COMMA) + .append(OSUtils.memoryUsage()).append(Constants.COMMA) + .append(OSUtils.loadAverage()).append(Constants.COMMA) + .append(splits[5]).append(Constants.COMMA) + .append(DateUtils.dateToString(new Date())); + + zkClient.setData().forPath(znode, sb.toString().getBytes()); } catch (Exception e) { logger.error("heartbeat for zk failed", e); From 275df85250683b22691c658591ed2094d8a32ac3 Mon Sep 17 00:00:00 2001 From: Rubik-W <39549317+Rubik-W@users.noreply.github.com> Date: Sun, 8 Mar 2020 16:00:47 +0800 Subject: [PATCH 069/221] fix: remove redundant import class (#2110) --- .../org/apache/dolphinscheduler/remote/command/Command.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Command.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Command.java index 86ba79c884..ed46e1ff51 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Command.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Command.java @@ -16,8 +16,6 @@ */ package org.apache.dolphinscheduler.remote.command; -import com.sun.org.apache.regexp.internal.RE; - import java.io.Serializable; import java.util.concurrent.atomic.AtomicLong; From ad381fcb716632deb966fd8ca0627ff8c17d5641 Mon Sep 17 00:00:00 2001 From: "gabry.wu" Date: Sun, 8 Mar 2020 21:22:07 +0800 Subject: [PATCH 070/221] move UT class to reasonable package (#2116) --- .../dolphinscheduler/common}/utils/PreconditionsTest.java | 3 +-- .../dolphinscheduler/service/quartz}/cron/CronUtilsTest.java | 3 +-- .../dolphinscheduler/service}/queue/BaseTaskQueueTest.java | 2 +- .../dolphinscheduler/service}/queue/TaskQueueZKImplTest.java | 2 +- 4 files changed, 4 insertions(+), 6 deletions(-) rename {dolphinscheduler-service/src/test/java => dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common}/utils/PreconditionsTest.java (98%) rename dolphinscheduler-service/src/test/java/{ => org/apache/dolphinscheduler/service/quartz}/cron/CronUtilsTest.java (99%) rename dolphinscheduler-service/src/test/java/{ => org/apache/dolphinscheduler/service}/queue/BaseTaskQueueTest.java (96%) rename dolphinscheduler-service/src/test/java/{ => org/apache/dolphinscheduler/service}/queue/TaskQueueZKImplTest.java (99%) diff --git a/dolphinscheduler-service/src/test/java/utils/PreconditionsTest.java b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/PreconditionsTest.java similarity index 98% rename from dolphinscheduler-service/src/test/java/utils/PreconditionsTest.java rename to dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/PreconditionsTest.java index a1b85f1b12..47b24bb93c 100644 --- a/dolphinscheduler-service/src/test/java/utils/PreconditionsTest.java +++ b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/PreconditionsTest.java @@ -14,9 +14,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package utils; +package org.apache.dolphinscheduler.common.utils; -import org.apache.dolphinscheduler.common.utils.Preconditions; import org.junit.Assert; import org.junit.Test; import org.slf4j.Logger; diff --git a/dolphinscheduler-service/src/test/java/cron/CronUtilsTest.java b/dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/quartz/cron/CronUtilsTest.java similarity index 99% rename from dolphinscheduler-service/src/test/java/cron/CronUtilsTest.java rename to dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/quartz/cron/CronUtilsTest.java index 6a402b5e67..b4f864c5b4 100644 --- a/dolphinscheduler-service/src/test/java/cron/CronUtilsTest.java +++ b/dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/quartz/cron/CronUtilsTest.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cron; +package org.apache.dolphinscheduler.service.quartz.cron; import com.cronutils.builder.CronBuilder; import com.cronutils.model.Cron; @@ -25,7 +25,6 @@ import com.cronutils.model.field.CronFieldName; import com.cronutils.model.field.expression.*; import org.apache.dolphinscheduler.common.enums.CycleEnum; import org.apache.dolphinscheduler.common.utils.DateUtils; -import org.apache.dolphinscheduler.service.quartz.cron.CronUtils; import org.junit.Assert; import org.junit.Test; import org.slf4j.Logger; diff --git a/dolphinscheduler-service/src/test/java/queue/BaseTaskQueueTest.java b/dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/queue/BaseTaskQueueTest.java similarity index 96% rename from dolphinscheduler-service/src/test/java/queue/BaseTaskQueueTest.java rename to dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/queue/BaseTaskQueueTest.java index 97ab9969a3..17e2ae4056 100644 --- a/dolphinscheduler-service/src/test/java/queue/BaseTaskQueueTest.java +++ b/dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/queue/BaseTaskQueueTest.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package queue; +package org.apache.dolphinscheduler.service.queue; import org.apache.dolphinscheduler.service.queue.ITaskQueue; import org.apache.dolphinscheduler.service.queue.TaskQueueFactory; diff --git a/dolphinscheduler-service/src/test/java/queue/TaskQueueZKImplTest.java b/dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/queue/TaskQueueZKImplTest.java similarity index 99% rename from dolphinscheduler-service/src/test/java/queue/TaskQueueZKImplTest.java rename to dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/queue/TaskQueueZKImplTest.java index d29c5aa610..5d464ac3c9 100644 --- a/dolphinscheduler-service/src/test/java/queue/TaskQueueZKImplTest.java +++ b/dolphinscheduler-service/src/test/java/org/apache/dolphinscheduler/service/queue/TaskQueueZKImplTest.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package queue; +package org.apache.dolphinscheduler.service.queue; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.utils.IpUtils; From 450a1f56fc73f088fce89a343a0b008706f2088c Mon Sep 17 00:00:00 2001 From: "gabry.wu" Date: Sun, 8 Mar 2020 21:28:49 +0800 Subject: [PATCH 071/221] Adapting partial code(file name start with E) to the sonar cloud rule (#1999) * Adapting partial code(file name start with E) to the sonar cloud rule * remove isEmpty invoke * resolve conflicts --- .../manager/EnterpriseWeChatManager.java | 4 +- .../alert/utils/EnterpriseWeChatUtils.java | 38 +++++++++---------- .../utils/EnterpriseWeChatUtilsTest.java | 8 ++-- .../api/controller/ExecutorController.java | 2 +- .../api/service/ExecutorService.java | 15 +++----- .../dao/mapper/ErrorCommandMapperTest.java | 4 +- 6 files changed, 34 insertions(+), 37 deletions(-) diff --git a/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/manager/EnterpriseWeChatManager.java b/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/manager/EnterpriseWeChatManager.java index 9bcad56c24..bb06be6561 100644 --- a/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/manager/EnterpriseWeChatManager.java +++ b/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/manager/EnterpriseWeChatManager.java @@ -42,8 +42,8 @@ public class EnterpriseWeChatManager { public Map send(Alert alert, String token){ Map retMap = new HashMap<>(); retMap.put(Constants.STATUS, false); - String agentId = EnterpriseWeChatUtils.enterpriseWeChatAgentId; - String users = EnterpriseWeChatUtils.enterpriseWeChatUsers; + String agentId = EnterpriseWeChatUtils.ENTERPRISE_WE_CHAT_AGENT_ID; + String users = EnterpriseWeChatUtils.ENTERPRISE_WE_CHAT_USERS; List userList = Arrays.asList(users.split(",")); logger.info("send message {}",alert); String msg = EnterpriseWeChatUtils.makeUserSendMsg(userList, agentId,EnterpriseWeChatUtils.markdownByAlert(alert)); diff --git a/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/EnterpriseWeChatUtils.java b/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/EnterpriseWeChatUtils.java index ff8822421a..900c120cd4 100644 --- a/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/EnterpriseWeChatUtils.java +++ b/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/EnterpriseWeChatUtils.java @@ -43,24 +43,24 @@ public class EnterpriseWeChatUtils { public static final Logger logger = LoggerFactory.getLogger(EnterpriseWeChatUtils.class); - private static final String enterpriseWeChatCorpId = PropertyUtils.getString(Constants.ENTERPRISE_WECHAT_CORP_ID); + private static final String ENTERPRISE_WE_CHAT_CORP_ID = PropertyUtils.getString(Constants.ENTERPRISE_WECHAT_CORP_ID); - private static final String enterpriseWeChatSecret = PropertyUtils.getString(Constants.ENTERPRISE_WECHAT_SECRET); + private static final String ENTERPRISE_WE_CHAT_SECRET = PropertyUtils.getString(Constants.ENTERPRISE_WECHAT_SECRET); - private static final String enterpriseWeChatTokenUrl = PropertyUtils.getString(Constants.ENTERPRISE_WECHAT_TOKEN_URL); - private static String enterpriseWeChatTokenUrlReplace = enterpriseWeChatTokenUrl - .replaceAll("\\$corpId", enterpriseWeChatCorpId) - .replaceAll("\\$secret", enterpriseWeChatSecret); + private static final String ENTERPRISE_WE_CHAT_TOKEN_URL = PropertyUtils.getString(Constants.ENTERPRISE_WECHAT_TOKEN_URL); + private static final String ENTERPRISE_WE_CHAT_TOKEN_URL_REPLACE = ENTERPRISE_WE_CHAT_TOKEN_URL + .replaceAll("\\$corpId", ENTERPRISE_WE_CHAT_CORP_ID) + .replaceAll("\\$secret", ENTERPRISE_WE_CHAT_SECRET); - private static final String enterpriseWeChatPushUrl = PropertyUtils.getString(Constants.ENTERPRISE_WECHAT_PUSH_URL); + private static final String ENTERPRISE_WE_CHAT_PUSH_URL = PropertyUtils.getString(Constants.ENTERPRISE_WECHAT_PUSH_URL); - private static final String enterpriseWeChatTeamSendMsg = PropertyUtils.getString(Constants.ENTERPRISE_WECHAT_TEAM_SEND_MSG); + private static final String ENTERPRISE_WE_CHAT_TEAM_SEND_MSG = PropertyUtils.getString(Constants.ENTERPRISE_WECHAT_TEAM_SEND_MSG); - private static final String enterpriseWeChatUserSendMsg = PropertyUtils.getString(Constants.ENTERPRISE_WECHAT_USER_SEND_MSG); + private static final String ENTERPRISE_WE_CHAT_USER_SEND_MSG = PropertyUtils.getString(Constants.ENTERPRISE_WECHAT_USER_SEND_MSG); - public static final String enterpriseWeChatAgentId = PropertyUtils.getString(Constants.ENTERPRISE_WECHAT_AGENT_ID); + public static final String ENTERPRISE_WE_CHAT_AGENT_ID = PropertyUtils.getString(Constants.ENTERPRISE_WECHAT_AGENT_ID); - public static final String enterpriseWeChatUsers = PropertyUtils.getString(Constants.ENTERPRISE_WECHAT_USERS); + public static final String ENTERPRISE_WE_CHAT_USERS = PropertyUtils.getString(Constants.ENTERPRISE_WECHAT_USERS); /** * get Enterprise WeChat is enable @@ -87,7 +87,7 @@ public class EnterpriseWeChatUtils { CloseableHttpClient httpClient = HttpClients.createDefault(); try { - HttpGet httpGet = new HttpGet(enterpriseWeChatTokenUrlReplace); + HttpGet httpGet = new HttpGet(ENTERPRISE_WE_CHAT_TOKEN_URL_REPLACE); CloseableHttpResponse response = httpClient.execute(httpGet); try { HttpEntity entity = response.getEntity(); @@ -114,7 +114,7 @@ public class EnterpriseWeChatUtils { * @return Enterprise WeChat send message */ public static String makeTeamSendMsg(String toParty, String agentId, String msg) { - return enterpriseWeChatTeamSendMsg.replaceAll("\\$toParty", toParty) + return ENTERPRISE_WE_CHAT_TEAM_SEND_MSG.replaceAll("\\$toParty", toParty) .replaceAll("\\$agentId", agentId) .replaceAll("\\$msg", msg); } @@ -128,7 +128,7 @@ public class EnterpriseWeChatUtils { */ public static String makeTeamSendMsg(Collection toParty, String agentId, String msg) { String listParty = FuncUtils.mkString(toParty, "|"); - return enterpriseWeChatTeamSendMsg.replaceAll("\\$toParty", listParty) + return ENTERPRISE_WE_CHAT_TEAM_SEND_MSG.replaceAll("\\$toParty", listParty) .replaceAll("\\$agentId", agentId) .replaceAll("\\$msg", msg); } @@ -141,7 +141,7 @@ public class EnterpriseWeChatUtils { * @return Enterprise WeChat send message */ public static String makeUserSendMsg(String toUser, String agentId, String msg) { - return enterpriseWeChatUserSendMsg.replaceAll("\\$toUser", toUser) + return ENTERPRISE_WE_CHAT_USER_SEND_MSG.replaceAll("\\$toUser", toUser) .replaceAll("\\$agentId", agentId) .replaceAll("\\$msg", msg); } @@ -155,7 +155,7 @@ public class EnterpriseWeChatUtils { */ public static String makeUserSendMsg(Collection toUser, String agentId, String msg) { String listUser = FuncUtils.mkString(toUser, "|"); - return enterpriseWeChatUserSendMsg.replaceAll("\\$toUser", listUser) + return ENTERPRISE_WE_CHAT_USER_SEND_MSG.replaceAll("\\$toUser", listUser) .replaceAll("\\$agentId", agentId) .replaceAll("\\$msg", msg); } @@ -169,7 +169,7 @@ public class EnterpriseWeChatUtils { * @throws IOException the IOException */ public static String sendEnterpriseWeChat(String charset, String data, String token) throws IOException { - String enterpriseWeChatPushUrlReplace = enterpriseWeChatPushUrl.replaceAll("\\$token", token); + String enterpriseWeChatPushUrlReplace = ENTERPRISE_WE_CHAT_PUSH_URL.replaceAll("\\$token", token); CloseableHttpClient httpClient = HttpClients.createDefault(); try { @@ -184,8 +184,8 @@ public class EnterpriseWeChatUtils { } finally { response.close(); } - logger.info("Enterprise WeChat send [{}], param:{}, resp:{}", - enterpriseWeChatPushUrl, data, resp); + logger.info("Enterprise WeChat send [{}], param:{}, resp:{}", + ENTERPRISE_WE_CHAT_PUSH_URL, data, resp); return resp; } finally { httpClient.close(); diff --git a/dolphinscheduler-alert/src/test/java/org/apache/dolphinscheduler/alert/utils/EnterpriseWeChatUtilsTest.java b/dolphinscheduler-alert/src/test/java/org/apache/dolphinscheduler/alert/utils/EnterpriseWeChatUtilsTest.java index 3471f6efdd..15b92a622e 100644 --- a/dolphinscheduler-alert/src/test/java/org/apache/dolphinscheduler/alert/utils/EnterpriseWeChatUtilsTest.java +++ b/dolphinscheduler-alert/src/test/java/org/apache/dolphinscheduler/alert/utils/EnterpriseWeChatUtilsTest.java @@ -53,7 +53,7 @@ public class EnterpriseWeChatUtilsTest { String resp = EnterpriseWeChatUtils.sendEnterpriseWeChat("utf-8", msg, token); String errmsg = JSON.parseObject(resp).getString("errmsg"); - Assert.assertEquals(errmsg, "ok"); + Assert.assertEquals("ok",errmsg); } catch (IOException e) { e.printStackTrace(); } @@ -68,7 +68,7 @@ public class EnterpriseWeChatUtilsTest { String resp = EnterpriseWeChatUtils.sendEnterpriseWeChat("utf-8", msg, token); String errmsg = JSON.parseObject(resp).getString("errmsg"); - Assert.assertEquals(errmsg, "ok"); + Assert.assertEquals("ok",errmsg); } catch (IOException e) { e.printStackTrace(); } @@ -95,7 +95,7 @@ public class EnterpriseWeChatUtilsTest { String resp = EnterpriseWeChatUtils.sendEnterpriseWeChat("utf-8", msg, token); String errmsg = JSON.parseObject(resp).getString("errmsg"); - Assert.assertEquals(errmsg, "ok"); + Assert.assertEquals("ok",errmsg); } catch (IOException e) { e.printStackTrace(); } @@ -110,7 +110,7 @@ public class EnterpriseWeChatUtilsTest { String resp = EnterpriseWeChatUtils.sendEnterpriseWeChat("utf-8", msg, token); String errmsg = JSON.parseObject(resp).getString("errmsg"); - Assert.assertEquals(errmsg, "ok"); + Assert.assertEquals("ok",errmsg); } catch (IOException e) { e.printStackTrace(); } diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ExecutorController.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ExecutorController.java index cae4993942..ffedd5703c 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ExecutorController.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/ExecutorController.java @@ -149,7 +149,7 @@ public class ExecutorController extends BaseController { ) { try { logger.info("execute command, login user: {}, project:{}, process instance id:{}, execute type:{}", - loginUser.getUserName(), projectName, processInstanceId, executeType.toString()); + loginUser.getUserName(), projectName, processInstanceId, executeType); Map result = execService.execute(loginUser, projectName, processInstanceId, executeType); return returnDataList(result); } catch (Exception e) { diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java index 152292a21b..86b507f0a0 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ExecutorService.java @@ -259,10 +259,7 @@ public class ExecutorService extends BaseService{ // checkTenantExists(); Tenant tenant = processService.getTenantForProcess(processDefinition.getTenantId(), processDefinition.getUserId()); - if(tenant == null){ - return false; - } - return true; + return tenant != null; } /** @@ -298,6 +295,7 @@ public class ExecutorService extends BaseService{ if (executionStatus.typeIsPause()|| executionStatus.typeIsCancel()) { checkResult = true; } + break; default: break; } @@ -369,7 +367,7 @@ public class ExecutorService extends BaseService{ * @return check result code */ public Map startCheckByProcessDefinedId(int processDefineId) { - Map result = new HashMap(); + Map result = new HashMap<>(); if (processDefineId == 0){ logger.error("process definition id is null"); @@ -378,10 +376,9 @@ public class ExecutorService extends BaseService{ List ids = new ArrayList<>(); processService.recurseFindSubProcessId(processDefineId, ids); Integer[] idArray = ids.toArray(new Integer[ids.size()]); - if (ids.size() > 0){ - List processDefinitionList; - processDefinitionList = processDefinitionMapper.queryDefinitionListByIdList(idArray); - if (processDefinitionList != null && processDefinitionList.size() > 0){ + if (!ids.isEmpty()){ + List processDefinitionList = processDefinitionMapper.queryDefinitionListByIdList(idArray); + if (processDefinitionList != null){ for (ProcessDefinition processDefinition : processDefinitionList){ /** * if there is no online process, exit directly diff --git a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/ErrorCommandMapperTest.java b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/ErrorCommandMapperTest.java index 3653e6a53d..5fb7dfc09f 100644 --- a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/ErrorCommandMapperTest.java +++ b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/ErrorCommandMapperTest.java @@ -67,7 +67,7 @@ public class ErrorCommandMapperTest { //update errorCommand.setUpdateTime(new Date()); int update = errorCommandMapper.updateById(errorCommand); - Assert.assertEquals(update, 1); + Assert.assertEquals(1,update); errorCommandMapper.deleteById(errorCommand.getId()); } @@ -79,7 +79,7 @@ public class ErrorCommandMapperTest { ErrorCommand errorCommand = insertOne(); int delete = errorCommandMapper.deleteById(errorCommand.getId()); - Assert.assertEquals(delete, 1); + Assert.assertEquals(1,delete); } /** From 9224b49b58b756d22c75d8929108f716283282b4 Mon Sep 17 00:00:00 2001 From: tison Date: Mon, 9 Mar 2020 19:06:41 +0800 Subject: [PATCH 072/221] access field handle of FileDescriptor in ProcessImplForWin32 by reflection for portability (#2113) * access field handle of FileDescriptor in ProcessImplForWin32 by reflection for portability Current implementation relies on `sun.misc.JavaIOFileDescriptorAccess` which is only accessible on oraclejdk8. Basically the demand is getting & setting `handle` field of `FileDescriptor`, so we can directly do that with reflection. Though, I suspect the necessity we introduce ProcessImplForWin32. Maybe we could have a better way to support worker server to run bat script. * harden initialization of ProcessImplForWin32 * ignore ShellTaskTest#testHandleForWindows outside Windows --- .../common/utils/OSUtils.java | 16 ++++-- .../utils/process/ProcessImplForWin32.java | 57 +++++++++++++++---- .../worker/task/shell/ShellTaskTest.java | 3 +- 3 files changed, 58 insertions(+), 18 deletions(-) diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/OSUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/OSUtils.java index acfca77c8c..b011c0bc4e 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/OSUtils.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/OSUtils.java @@ -400,8 +400,7 @@ public class OSUtils { * @return true if mac */ public static boolean isMacOS() { - String os = System.getProperty("os.name"); - return os.startsWith("Mac"); + return getOSName().startsWith("Mac"); } @@ -409,9 +408,16 @@ public class OSUtils { * whether is windows * @return true if windows */ - public static boolean isWindows() { - String os = System.getProperty("os.name"); - return os.startsWith("Windows"); + public static boolean isWindows() { ; + return getOSName().startsWith("Windows"); + } + + /** + * get current OS name + * @return current OS name + */ + public static String getOSName() { + return System.getProperty("os.name"); } /** diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/process/ProcessImplForWin32.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/process/ProcessImplForWin32.java index 4583be8aff..9f2716a096 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/process/ProcessImplForWin32.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/process/ProcessImplForWin32.java @@ -19,6 +19,8 @@ package org.apache.dolphinscheduler.common.utils.process; import com.sun.jna.Pointer; import com.sun.jna.platform.win32.*; import com.sun.jna.ptr.IntByReference; +import java.lang.reflect.Field; +import org.apache.dolphinscheduler.common.utils.OSUtils; import sun.security.action.GetPropertyAction; import java.io.*; @@ -31,10 +33,25 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import static com.sun.jna.platform.win32.WinBase.STILL_ACTIVE; +import static java.util.Objects.requireNonNull; public class ProcessImplForWin32 extends Process { - private static final sun.misc.JavaIOFileDescriptorAccess fdAccess - = sun.misc.SharedSecrets.getJavaIOFileDescriptorAccess(); + + private static final Field FD_HANDLE; + + static { + if (!OSUtils.isWindows()) { + throw new RuntimeException("ProcessImplForWin32 can be only initialized in " + + "Windows environment, but current OS is " + OSUtils.getOSName()); + } + + try { + FD_HANDLE = requireNonNull(FileDescriptor.class.getDeclaredField("handle")); + FD_HANDLE.setAccessible(true); + } catch (NoSuchFieldException e) { + throw new RuntimeException(e); + } + } private static final int PIPE_SIZE = 4096 + 24; @@ -46,6 +63,22 @@ public class ProcessImplForWin32 extends Process { private static final WinNT.HANDLE JAVA_INVALID_HANDLE_VALUE = new WinNT.HANDLE(Pointer.createConstant(-1)); + private static void setHandle(FileDescriptor obj, long handle) { + try { + FD_HANDLE.set(obj, handle); + } catch (IllegalAccessException e) { + throw new RuntimeException(e); + } + } + + private static long getHandle(FileDescriptor obj) { + try { + return (Long) FD_HANDLE.get(obj); + } catch (IllegalAccessException e) { + throw new RuntimeException(e); + } + } + /** * Open a file for writing. If {@code append} is {@code true} then the file * is opened for atomic append directly and a FileOutputStream constructed @@ -63,7 +96,7 @@ public class ProcessImplForWin32 extends Process { sm.checkWrite(path); long handle = openForAtomicAppend(path); final FileDescriptor fd = new FileDescriptor(); - fdAccess.setHandle(fd, handle); + setHandle(fd, handle); return AccessController.doPrivileged( new PrivilegedAction() { public FileOutputStream run() { @@ -102,30 +135,30 @@ public class ProcessImplForWin32 extends Process { if (redirects[0] == ProcessBuilderForWin32.Redirect.PIPE) stdHandles[0] = -1L; else if (redirects[0] == ProcessBuilderForWin32.Redirect.INHERIT) - stdHandles[0] = fdAccess.getHandle(FileDescriptor.in); + stdHandles[0] = getHandle(FileDescriptor.in); else { f0 = new FileInputStream(redirects[0].file()); - stdHandles[0] = fdAccess.getHandle(f0.getFD()); + stdHandles[0] = getHandle(f0.getFD()); } if (redirects[1] == ProcessBuilderForWin32.Redirect.PIPE) stdHandles[1] = -1L; else if (redirects[1] == ProcessBuilderForWin32.Redirect.INHERIT) - stdHandles[1] = fdAccess.getHandle(FileDescriptor.out); + stdHandles[1] = getHandle(FileDescriptor.out); else { f1 = newFileOutputStream(redirects[1].file(), redirects[1].append()); - stdHandles[1] = fdAccess.getHandle(f1.getFD()); + stdHandles[1] = getHandle(f1.getFD()); } if (redirects[2] == ProcessBuilderForWin32.Redirect.PIPE) stdHandles[2] = -1L; else if (redirects[2] == ProcessBuilderForWin32.Redirect.INHERIT) - stdHandles[2] = fdAccess.getHandle(FileDescriptor.err); + stdHandles[2] = getHandle(FileDescriptor.err); else { f2 = newFileOutputStream(redirects[2].file(), redirects[2].append()); - stdHandles[2] = fdAccess.getHandle(f2.getFD()); + stdHandles[2] = getHandle(f2.getFD()); } } @@ -442,7 +475,7 @@ public class ProcessImplForWin32 extends Process { stdin_stream = ProcessBuilderForWin32.NullOutputStream.INSTANCE; else { FileDescriptor stdin_fd = new FileDescriptor(); - fdAccess.setHandle(stdin_fd, stdHandles[0]); + setHandle(stdin_fd, stdHandles[0]); stdin_stream = new BufferedOutputStream( new FileOutputStream(stdin_fd)); } @@ -451,7 +484,7 @@ public class ProcessImplForWin32 extends Process { stdout_stream = ProcessBuilderForWin32.NullInputStream.INSTANCE; else { FileDescriptor stdout_fd = new FileDescriptor(); - fdAccess.setHandle(stdout_fd, stdHandles[1]); + setHandle(stdout_fd, stdHandles[1]); stdout_stream = new BufferedInputStream( new FileInputStream(stdout_fd)); } @@ -460,7 +493,7 @@ public class ProcessImplForWin32 extends Process { stderr_stream = ProcessBuilderForWin32.NullInputStream.INSTANCE; else { FileDescriptor stderr_fd = new FileDescriptor(); - fdAccess.setHandle(stderr_fd, stdHandles[2]); + setHandle(stderr_fd, stdHandles[2]); stderr_stream = new FileInputStream(stderr_fd); } diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTaskTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTaskTest.java index 5536665e26..ebe90147d1 100644 --- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTaskTest.java +++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTaskTest.java @@ -27,6 +27,7 @@ import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.process.ProcessService; import org.junit.After; import org.junit.Assert; +import org.junit.Assume; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -172,7 +173,7 @@ public class ShellTaskTest { @Test public void testHandleForWindows() throws Exception { try { - PowerMockito.when(OSUtils.isWindows()).thenReturn(true); + Assume.assumeTrue(OSUtils.isWindows()); shellTask.handle(); Assert.assertTrue(true); } catch (Error | Exception e) { From ebf7a980d2922a5659d8f0da28f519556aa3a2f0 Mon Sep 17 00:00:00 2001 From: "gabry.wu" Date: Tue, 10 Mar 2020 18:59:45 +0800 Subject: [PATCH 073/221] Adapting partial code(file name start with F) to the sonar cloud rule (#2045) * Adapting partial code(file name start with F) to the sonar cloud rule * add more unit test * add License * add includes configuration to maven-surefire-plugin * fix getResourceFilesList incorrect logic --- .../alert/utils/FuncUtils.java | 2 +- .../alert/utils/FuncUtilsTest.java | 2 +- .../common/task/flink/FlinkParameters.java | 14 +++-- .../common/utils/FileUtils.java | 6 +- .../common/task/FlinkParametersTest.java | 55 +++++++++++++++++++ .../common/utils/FileUtilsTest.java | 17 +++--- .../server/utils/FlinkArgsUtils.java | 16 ++---- .../worker/runner/TaskScheduleThread.java | 4 +- .../server/utils/FlinkArgsUtilsTest.java | 24 ++++---- pom.xml | 1 + 10 files changed, 97 insertions(+), 44 deletions(-) create mode 100644 dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/task/FlinkParametersTest.java diff --git a/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/FuncUtils.java b/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/FuncUtils.java index dd6ca4b8a6..d68532a82b 100644 --- a/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/FuncUtils.java +++ b/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/FuncUtils.java @@ -20,7 +20,7 @@ import org.apache.dolphinscheduler.common.utils.StringUtils; public class FuncUtils { - static public String mkString(Iterable list, String split) { + public static String mkString(Iterable list, String split) { if (null == list || StringUtils.isEmpty(split)){ return null; diff --git a/dolphinscheduler-alert/src/test/java/org/apache/dolphinscheduler/alert/utils/FuncUtilsTest.java b/dolphinscheduler-alert/src/test/java/org/apache/dolphinscheduler/alert/utils/FuncUtilsTest.java index e2b25d961d..a4aeea9c0c 100644 --- a/dolphinscheduler-alert/src/test/java/org/apache/dolphinscheduler/alert/utils/FuncUtilsTest.java +++ b/dolphinscheduler-alert/src/test/java/org/apache/dolphinscheduler/alert/utils/FuncUtilsTest.java @@ -46,7 +46,7 @@ public class FuncUtilsTest { logger.info(result); //Expected result string - assertEquals(result, "user1|user2|user3"); + assertEquals("user1|user2|user3", result); //Null list expected return null result = FuncUtils.mkString(null, split); diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/flink/FlinkParameters.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/flink/FlinkParameters.java index 0638b3858e..1fbd9ab354 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/flink/FlinkParameters.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/flink/FlinkParameters.java @@ -20,6 +20,7 @@ import org.apache.dolphinscheduler.common.enums.ProgramType; import org.apache.dolphinscheduler.common.process.ResourceInfo; import org.apache.dolphinscheduler.common.task.AbstractParameters; +import java.util.Collections; import java.util.List; import java.util.stream.Collectors; @@ -207,12 +208,15 @@ public class FlinkParameters extends AbstractParameters { @Override public List getResourceFilesList() { - if(resourceList !=null ) { - this.resourceList.add(mainJar); - return resourceList.stream() - .map(p -> p.getRes()).collect(Collectors.toList()); + if(resourceList != null ) { + List resourceFiles = resourceList.stream() + .map(ResourceInfo::getRes).collect(Collectors.toList()); + if(mainJar != null) { + resourceFiles.add(mainJar.getRes()); + } + return resourceFiles; } - return null; + return Collections.emptyList(); } diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/FileUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/FileUtils.java index c84848fbae..dc60b04c59 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/FileUtils.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/FileUtils.java @@ -44,7 +44,7 @@ public class FileUtils { String fileSuffix = ""; if (StringUtils.isNotEmpty(filename)) { - int lastIndex = filename.lastIndexOf("."); + int lastIndex = filename.lastIndexOf('.'); if (lastIndex > 0) { fileSuffix = filename.substring(lastIndex + 1); } @@ -325,10 +325,8 @@ public class FileUtils { } } else { File parent = file.getParentFile(); - if (parent != null) { - if (!parent.mkdirs() && !parent.isDirectory()) { + if (parent != null && !parent.mkdirs() && !parent.isDirectory()) { throw new IOException("Directory '" + parent + "' could not be created"); - } } } return new FileOutputStream(file, append); diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/task/FlinkParametersTest.java b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/task/FlinkParametersTest.java new file mode 100644 index 0000000000..7ce00e875a --- /dev/null +++ b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/task/FlinkParametersTest.java @@ -0,0 +1,55 @@ +/* + * 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.common.task; + +import org.apache.dolphinscheduler.common.process.ResourceInfo; +import org.apache.dolphinscheduler.common.task.flink.FlinkParameters; +import org.junit.Assert; +import org.junit.Test; + +import java.util.LinkedList; +import java.util.List; + +public class FlinkParametersTest { + @Test + public void getResourceFilesList() { + FlinkParameters flinkParameters = new FlinkParameters(); + Assert.assertNotNull(flinkParameters.getResourceFilesList()); + Assert.assertTrue(flinkParameters.getResourceFilesList().isEmpty()); + + ResourceInfo mainResource = new ResourceInfo(); + mainResource.setRes("testFlinkMain-1.0.0-SNAPSHOT.jar"); + flinkParameters.setMainJar(mainResource); + + List resourceInfos = new LinkedList<>(); + ResourceInfo resourceInfo1 = new ResourceInfo(); + resourceInfo1.setRes("testFlinkParameters1.jar"); + resourceInfos.add(resourceInfo1); + + flinkParameters.setResourceList(resourceInfos); + Assert.assertNotNull(flinkParameters.getResourceFilesList()); + Assert.assertEquals(2, flinkParameters.getResourceFilesList().size()); + + ResourceInfo resourceInfo2 = new ResourceInfo(); + resourceInfo2.setRes("testFlinkParameters2.jar"); + resourceInfos.add(resourceInfo2); + + flinkParameters.setResourceList(resourceInfos); + Assert.assertNotNull(flinkParameters.getResourceFilesList()); + Assert.assertEquals(3, flinkParameters.getResourceFilesList().size()); + } +} diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/FileUtilsTest.java b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/FileUtilsTest.java index 89458f6f1c..96217842bf 100644 --- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/FileUtilsTest.java +++ b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/FileUtilsTest.java @@ -30,29 +30,32 @@ public class FileUtilsTest { @Test public void suffix() { - Assert.assertEquals(FileUtils.suffix("ninfor.java"),"java"); + Assert.assertEquals("java", FileUtils.suffix("ninfor.java")); + Assert.assertEquals("", FileUtils.suffix(null)); + Assert.assertEquals("", FileUtils.suffix("")); + Assert.assertEquals("", FileUtils.suffix("ninfor-java")); } @Test public void testGetDownloadFilename() { PowerMockito.mockStatic(DateUtils.class); PowerMockito.when(DateUtils.getCurrentTime(YYYYMMDDHHMMSS)).thenReturn("20190101101059"); - Assert.assertEquals(FileUtils.getDownloadFilename("test"), - "/tmp/dolphinscheduler/download/20190101101059/test"); + Assert.assertEquals("/tmp/dolphinscheduler/download/20190101101059/test", + FileUtils.getDownloadFilename("test")); } @Test public void testGetUploadFilename() { - Assert.assertEquals(FileUtils.getUploadFilename("aaa","bbb"), - "/tmp/dolphinscheduler/aaa/resources/bbb"); + Assert.assertEquals("/tmp/dolphinscheduler/aaa/resources/bbb", + FileUtils.getUploadFilename("aaa","bbb")); } @Test public void testGetProcessExecDir() { String dir = FileUtils.getProcessExecDir(1,2,3, 4); - Assert.assertEquals(dir, "/tmp/dolphinscheduler/exec/process/1/2/3/4"); + Assert.assertEquals("/tmp/dolphinscheduler/exec/process/1/2/3/4", dir); dir = FileUtils.getProcessExecDir(1,2,3); - Assert.assertEquals(dir, "/tmp/dolphinscheduler/exec/process/1/2/3"); + Assert.assertEquals("/tmp/dolphinscheduler/exec/process/1/2/3", dir); } @Test diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/FlinkArgsUtils.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/FlinkArgsUtils.java index 4c33ef8db2..12c7eb2d56 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/FlinkArgsUtils.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/FlinkArgsUtils.java @@ -17,12 +17,11 @@ package org.apache.dolphinscheduler.server.utils; +import org.apache.commons.lang.StringUtils; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.ProgramType; import org.apache.dolphinscheduler.common.process.ResourceInfo; import org.apache.dolphinscheduler.common.task.flink.FlinkParameters; -import org.apache.commons.lang.StringUtils; -import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.List; @@ -32,12 +31,7 @@ import java.util.List; * spark args utils */ public class FlinkArgsUtils { - - /** - * logger of FlinkArgsUtils - */ - private static final org.slf4j.Logger logger = LoggerFactory.getLogger(FlinkArgsUtils.class); - + private static final String LOCAL_DEPLOY_MODE = "local"; /** * build args * @param param flink parameters @@ -52,7 +46,7 @@ public class FlinkArgsUtils { deployMode = tmpDeployMode; } - if (!"local".equals(deployMode)) { + if (!LOCAL_DEPLOY_MODE.equals(deployMode)) { args.add(Constants.FLINK_RUN_MODE); //-m args.add(Constants.FLINK_YARN_CLUSTER); //yarn-cluster @@ -113,12 +107,12 @@ public class FlinkArgsUtils { String queue = param.getQueue(); if (StringUtils.isNotEmpty(others)) { - if (!others.contains(Constants.FLINK_QUEUE) && StringUtils.isNotEmpty(queue) && !deployMode.equals("local")) { + if (!others.contains(Constants.FLINK_QUEUE) && StringUtils.isNotEmpty(queue) && !deployMode.equals(LOCAL_DEPLOY_MODE)) { args.add(Constants.FLINK_QUEUE); args.add(param.getQueue()); } args.add(others); - } else if (StringUtils.isNotEmpty(queue) && !deployMode.equals("local")) { + } else if (StringUtils.isNotEmpty(queue) && !deployMode.equals(LOCAL_DEPLOY_MODE)) { args.add(Constants.FLINK_QUEUE); args.add(param.getQueue()); } diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/TaskScheduleThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/TaskScheduleThread.java index 5e68acf94e..21ee1dfa40 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/TaskScheduleThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/TaskScheduleThread.java @@ -296,9 +296,7 @@ public class TaskScheduleThread implements Runnable { if (baseParam != null) { List projectResourceFiles = baseParam.getResourceFilesList(); - if (projectResourceFiles != null) { - projectFiles.addAll(projectResourceFiles); - } + projectFiles.addAll(projectResourceFiles); } return new ArrayList<>(projectFiles); diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/utils/FlinkArgsUtilsTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/utils/FlinkArgsUtilsTest.java index 710d2c2505..2e4861e2a2 100644 --- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/utils/FlinkArgsUtilsTest.java +++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/utils/FlinkArgsUtilsTest.java @@ -87,35 +87,35 @@ public class FlinkArgsUtilsTest { } //Expected values and order - assertEquals(result.size(),20); + assertEquals(20, result.size()); - assertEquals(result.get(0),"-m"); - assertEquals(result.get(1),"yarn-cluster"); + assertEquals("-m", result.get(0)); + assertEquals("yarn-cluster", result.get(1)); - assertEquals(result.get(2),"-ys"); + assertEquals("-ys", result.get(2)); assertSame(Integer.valueOf(result.get(3)),slot); - assertEquals(result.get(4),"-ynm"); + assertEquals("-ynm",result.get(4)); assertEquals(result.get(5),appName); - assertEquals(result.get(6),"-yn"); + assertEquals("-yn", result.get(6)); assertSame(Integer.valueOf(result.get(7)),taskManager); - assertEquals(result.get(8),"-yjm"); + assertEquals("-yjm", result.get(8)); assertEquals(result.get(9),jobManagerMemory); - assertEquals(result.get(10),"-ytm"); + assertEquals("-ytm", result.get(10)); assertEquals(result.get(11),taskManagerMemory); - assertEquals(result.get(12),"-d"); + assertEquals("-d", result.get(12)); - assertEquals(result.get(13),"-c"); + assertEquals("-c", result.get(13)); assertEquals(result.get(14),mainClass); assertEquals(result.get(15),mainJar.getRes()); assertEquals(result.get(16),mainArgs); - assertEquals(result.get(17),"--qu"); + assertEquals("--qu", result.get(17)); assertEquals(result.get(18),queue); assertEquals(result.get(19),others); @@ -125,7 +125,7 @@ public class FlinkArgsUtilsTest { param1.setQueue(queue); param1.setDeployMode(mode); result = FlinkArgsUtils.buildArgs(param1); - assertEquals(result.size(),5); + assertEquals(5, result.size()); } } \ No newline at end of file diff --git a/pom.xml b/pom.xml index a7feec0e14..8b91add6d7 100644 --- a/pom.xml +++ b/pom.xml @@ -691,6 +691,7 @@ **/common/threadutils/*.java **/common/graph/*.java **/common/queue/*.java + **/common/task/FlinkParametersTest.java **/common/task/SqoopParameterEntityTest.java **/api/utils/CheckUtilsTest.java **/api/utils/FileUtilsTest.java From ce175310245cf78be91339545cf1f0b8cdd168d2 Mon Sep 17 00:00:00 2001 From: AhahaGe Date: Tue, 10 Mar 2020 22:00:40 +0800 Subject: [PATCH 074/221] add .asf.yaml file (#2136) Co-authored-by: guirong.ggr --- .asf.yaml | 10 ++++++++++ 1 file changed, 10 insertions(+) create mode 100644 .asf.yaml diff --git a/.asf.yaml b/.asf.yaml new file mode 100644 index 0000000000..fa71a46552 --- /dev/null +++ b/.asf.yaml @@ -0,0 +1,10 @@ +staging: + profile: ~ + whoami: dev + foo: trigger + +publish: + whoami: dev + +github: + description: “Dolphin Scheduler is a distributed and easy-to-extend visual DAG workflow scheduling system, dedicated to solving the complex dependencies in data processing, making the scheduling system out of the box for data processing.(分布式易扩展的可视化工作流任务调度)” \ No newline at end of file From 1db4ac6b63afedc2622aa7997cec701c2096e68f Mon Sep 17 00:00:00 2001 From: AhahaGe Date: Tue, 10 Mar 2020 22:28:40 +0800 Subject: [PATCH 075/221] edit README.md modify word from expand to extend (#2138) Co-authored-by: guirong.ggr --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 0a9a164b85..ebd620efee 100644 --- a/README.md +++ b/README.md @@ -17,7 +17,7 @@ Dolphin Scheduler Official Website ### Design features: -A distributed and easy-to-expand visual DAG workflow scheduling system. Dedicated to solving the complex dependencies in data processing, making the scheduling system `out of the box` for data processing. +A distributed and easy-to-extend visual DAG workflow scheduling system. Dedicated to solving the complex dependencies in data processing, making the scheduling system `out of the box` for data processing. Its main objectives are as follows: - Associate the Tasks according to the dependencies of the tasks in a DAG graph, which can visualize the running state of task in real time. From 84bcdc78967e5124865eedfa8d7587b92ef81310 Mon Sep 17 00:00:00 2001 From: break60 <790061044@qq.com> Date: Wed, 11 Mar 2020 11:03:47 +0800 Subject: [PATCH 076/221] Limit customization file content to no more than 3000 lines --- .../pages/file/pages/create/index.vue | 5 ++- .../resource/pages/file/pages/edit/index.vue | 38 ++++++++++++------- .../src/js/module/i18n/locale/en_US.js | 1 + .../src/js/module/i18n/locale/zh_CN.js | 1 + 4 files changed, 30 insertions(+), 15 deletions(-) diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/create/index.vue b/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/create/index.vue index caa31ed4d8..df84f0f292 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/create/index.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/create/index.vue @@ -132,7 +132,10 @@ this.$message.warning(`${i18n.$t('Please enter the resource content')}`) return false } - + if (editor.doc.size>3000) { + this.$message.warning(`${i18n.$t('Resource content cannot exceed 3000 lines')}`) + return false + } return true }, /** diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/edit/index.vue b/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/edit/index.vue index a9aee1c9fd..a0d1d7d187 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/edit/index.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/edit/index.vue @@ -45,6 +45,7 @@ @@ -257,4 +272,4 @@ } } } - \ No newline at end of file + diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/sqoop.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/sqoop.vue index eead745a06..6594ffcf74 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/sqoop.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/sqoop.vue @@ -65,7 +65,7 @@
    @@ -186,8 +186,8 @@
    @@ -385,7 +385,7 @@
    @@ -556,7 +556,8 @@ targetType:"HDFS", sourceMysqlParams:{ - srcDatasource:-1, + srcType:"MYSQL", + srcDatasource:"", srcTable:"", srcQueryType:"1", srcQuerySql:'', @@ -588,7 +589,8 @@ }, targetMysqlParams:{ - targetDatasource:-1, + targetType:"MYSQL", + targetDatasource:"", targetTable:"", targetColumns:"", fieldsTerminated:"", @@ -680,6 +682,7 @@ * return data source */ _onSourceDsData (o) { + this.sourceMysqlParams.srcType = o.type this.sourceMysqlParams.srcDatasource = o.datasource }, @@ -687,6 +690,7 @@ * return data source */ _onTargetDsData (o) { + this.targetMysqlParams.targetType = o.type this.targetMysqlParams.targetDatasource = o.datasource }, @@ -697,7 +701,7 @@ var params = null switch(this.sourceType){ case "MYSQL": - this.sourceMysqlParams.srcQuerySql = editor.getValue() + this.sourceMysqlParams.srcQuerySql = editor ? editor.getValue() : this.sourceMysqlParams.srcQuerySql params = JSON.stringify(this.sourceMysqlParams) break; case "ORACLE": @@ -879,7 +883,9 @@ * Processing code highlighting */ _handlerEditor () { - editor = codemirror('code-sql-mirror', { + this._destroyEditor() + + editor = codemirror('code-sqoop-mirror', { mode: 'sql', readOnly: this.isDetails }) @@ -892,9 +898,15 @@ } } + this.changes = () => { + this._cacheParams() + } + // Monitor keyboard editor.on('keypress', this.keypress) + editor.on('changes', this.changes) + editor.setValue(this.sourceMysqlParams.srcQuerySql) return editor @@ -906,6 +918,27 @@ _onLocalParams (a) { this.localParams = a }, + + _cacheParams () { + this.$emit('on-cache-params', { + concurrency:this.concurrency, + modelType:this.modelType, + sourceType:this.sourceType, + targetType:this.targetType, + sourceParams:this._handleSourceParams(), + targetParams:this._handleTargetParams(), + localParams:this.localParams + }); + }, + + _destroyEditor () { + if (editor) { + editor.toTextArea() // Uninstall + editor.off($('.code-sqoop-mirror'), 'keypress', this.keypress) + editor.off($('.code-sqoop-mirror'), 'changes', this.changes) + editor = null + } + }, }, watch: { // Listening to sqlType @@ -927,11 +960,12 @@ }, //Watch the cacheParams cacheParams (val) { - this.$emit('on-cache-params', val); + this._cacheParams() } }, created () { + this._destroyEditor() let o = this.backfillItem // Non-null objects represent backfill @@ -963,11 +997,28 @@ */ if (editor) { editor.toTextArea() // Uninstall - editor.off($('.code-sql-mirror'), 'keypress', this.keypress) + editor.off($('.code-sqoop-mirror'), 'keypress', this.keypress) + editor.off($('.code-sqoop-mirror'), 'changes', this.changes) + editor = null } }, computed: { + cacheParams () { + return { + concurrency:this.concurrency, + modelType:this.modelType, + sourceType:this.sourceType, + targetType:this.targetType, + localParams:this.localParams, + sourceMysqlParams:this.sourceMysqlParams, + sourceHdfsParams:this.sourceHdfsParams, + sourceHiveParams:this.sourceHiveParams, + targetHdfsParams:this.targetHdfsParams, + targetMysqlParams:this.targetMysqlParams, + targetHiveParams:this.targetHiveParams + } + } }, components: { mListBox, mDatasource, mLocalParams} } From 91ecc549a1d7cfae2ead6eb16b5e5559d13f54e0 Mon Sep 17 00:00:00 2001 From: Yelli Date: Thu, 12 Mar 2020 21:52:48 +0800 Subject: [PATCH 087/221] fix bug: data too long for column 'app_link' (#2146) * fix bug: data to long for app_link field * change app_link field length in 1.2.2 ddl * change app_link field type to text --- .../mysql/dolphinscheduler_ddl.sql | 22 ++++++++++++++++++- .../postgresql/dolphinscheduler_ddl.sql | 18 +++++++++++++++ 2 files changed, 39 insertions(+), 1 deletion(-) diff --git a/sql/upgrade/1.2.2_schema/mysql/dolphinscheduler_ddl.sql b/sql/upgrade/1.2.2_schema/mysql/dolphinscheduler_ddl.sql index e5785d9997..049484ce3a 100644 --- a/sql/upgrade/1.2.2_schema/mysql/dolphinscheduler_ddl.sql +++ b/sql/upgrade/1.2.2_schema/mysql/dolphinscheduler_ddl.sql @@ -54,4 +54,24 @@ d// delimiter ; CALL uc_dolphin_T_t_ds_task_instance_A_executor_id; -DROP PROCEDURE uc_dolphin_T_t_ds_task_instance_A_executor_id; \ No newline at end of file +DROP PROCEDURE uc_dolphin_T_t_ds_task_instance_A_executor_id; + +-- uc_dolphin_T_t_ds_task_instance_C_app_link +drop PROCEDURE if EXISTS uc_dolphin_T_t_ds_task_instance_C_app_link; +delimiter d// +CREATE PROCEDURE uc_dolphin_T_t_ds_task_instance_C_app_link() + BEGIN + IF NOT EXISTS (SELECT 1 FROM information_schema.COLUMNS + WHERE TABLE_NAME='t_ds_task_instance' + AND TABLE_SCHEMA=(SELECT DATABASE()) + AND COLUMN_NAME ='app_link') + THEN + ALTER TABLE t_ds_task_instance CHANGE COLUMN app_link app_link text COMMENT 'yarn app id'; + END IF; + END; + +d// + +delimiter ; +CALL uc_dolphin_T_t_ds_task_instance_C_app_link; +DROP PROCEDURE uc_dolphin_T_t_ds_task_instance_C_app_link; \ No newline at end of file diff --git a/sql/upgrade/1.2.2_schema/postgresql/dolphinscheduler_ddl.sql b/sql/upgrade/1.2.2_schema/postgresql/dolphinscheduler_ddl.sql index 70471cb26d..b1e0fd941c 100644 --- a/sql/upgrade/1.2.2_schema/postgresql/dolphinscheduler_ddl.sql +++ b/sql/upgrade/1.2.2_schema/postgresql/dolphinscheduler_ddl.sql @@ -49,3 +49,21 @@ d// delimiter ; SELECT uc_dolphin_T_t_ds_task_instance_A_executor_id(); DROP FUNCTION IF EXISTS uc_dolphin_T_t_ds_task_instance_A_executor_id(); + +-- uc_dolphin_T_t_ds_task_instance_C_app_link +delimiter d// +CREATE OR REPLACE FUNCTION uc_dolphin_T_t_ds_task_instance_C_app_link() RETURNS void AS $$ +BEGIN + IF NOT EXISTS (SELECT 1 FROM information_schema.COLUMNS + WHERE TABLE_NAME='t_ds_task_instance' + AND COLUMN_NAME ='app_link') + THEN + ALTER TABLE t_ds_task_instance ALTER COLUMN app_link type text; + END IF; +END; +$$ LANGUAGE plpgsql; +d// + +delimiter ; +SELECT uc_dolphin_T_t_ds_task_instance_C_app_link(); +DROP FUNCTION IF EXISTS uc_dolphin_T_t_ds_task_instance_C_app_link(); \ No newline at end of file From 455d8c7a7c36fca1259d0039910c07a716ff76dd Mon Sep 17 00:00:00 2001 From: dailidong Date: Fri, 13 Mar 2020 11:13:17 +0800 Subject: [PATCH 088/221] Delete .asf.yaml (#2158) --- .asf.yaml | 10 ---------- 1 file changed, 10 deletions(-) delete mode 100644 .asf.yaml diff --git a/.asf.yaml b/.asf.yaml deleted file mode 100644 index fa71a46552..0000000000 --- a/.asf.yaml +++ /dev/null @@ -1,10 +0,0 @@ -staging: - profile: ~ - whoami: dev - foo: trigger - -publish: - whoami: dev - -github: - description: “Dolphin Scheduler is a distributed and easy-to-extend visual DAG workflow scheduling system, dedicated to solving the complex dependencies in data processing, making the scheduling system out of the box for data processing.(分布式易扩展的可视化工作流任务调度)” \ No newline at end of file From 93aa160698a584bbcb8cdb45953efe1cc2b78915 Mon Sep 17 00:00:00 2001 From: "gabry.wu" Date: Fri, 13 Mar 2020 13:02:51 +0800 Subject: [PATCH 089/221] Adapting partial code(file name start with I) to the sonar cloud rule (#2155) --- .../common/utils/IOUtils.java | 19 +++++-------------- .../common/utils/IpUtils.java | 5 ----- .../common/utils/IpUtilsTest.java | 6 +++--- 3 files changed, 8 insertions(+), 22 deletions(-) diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/IOUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/IOUtils.java index 73df158aa3..ce551d8405 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/IOUtils.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/IOUtils.java @@ -19,26 +19,17 @@ package org.apache.dolphinscheduler.common.utils; +import java.io.Closeable; import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; public class IOUtils { - public static void closeQuietly(InputStream fis){ - if(fis != null){ + public static void closeQuietly(Closeable closeable){ + if(closeable != null){ try { - fis.close(); - } catch (IOException ignore) { - } - } - } - - public static void closeQuietly(InputStreamReader reader){ - if(reader != null){ - try { - reader.close(); + closeable.close(); } catch (IOException ignore) { + // nothing need to do } } } diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/IpUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/IpUtils.java index e7e0b34bdd..3b068c60d2 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/IpUtils.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/IpUtils.java @@ -17,16 +17,11 @@ package org.apache.dolphinscheduler.common.utils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - /** * http utils */ public class IpUtils { - private static final Logger logger = LoggerFactory.getLogger(IpUtils.class); public static final String DOT = "."; /** diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/IpUtilsTest.java b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/IpUtilsTest.java index e65bcd219b..ec6ffa35a7 100644 --- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/IpUtilsTest.java +++ b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/IpUtilsTest.java @@ -29,13 +29,13 @@ public class IpUtilsTest { long longNumber = IpUtils.ipToLong(ip); long longNumber2 = IpUtils.ipToLong(ip2); System.out.println(longNumber); - Assert.assertEquals(longNumber, 3232263681L); - Assert.assertEquals(longNumber2, 0L); + Assert.assertEquals(3232263681L, longNumber); + Assert.assertEquals(0L, longNumber2); String ip3 = "255.255.255.255"; long longNumber3 = IpUtils.ipToLong(ip3); System.out.println(longNumber3); - Assert.assertEquals(longNumber3, 4294967295L); + Assert.assertEquals(4294967295L, longNumber3); } From a0088c113726506b3b1c74ee2411a39ab184d2be Mon Sep 17 00:00:00 2001 From: gaojun2048 <32193458+gaojun2048@users.noreply.github.com> Date: Fri, 13 Mar 2020 13:04:33 +0800 Subject: [PATCH 090/221] fix rpmbuild and DS Ambari plugin bug (#2144) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Update quick-start.md * 更新demo的地址 * add rpm plugin * delete the git ignore files * 测试git ignore * 测试ignore * optimize the .gitignore file * add rpm package, delete unuse copy jar action in assembly * delete docs dir * add rpm plugin support * add rpm plugin support * add rpm plugin support * dolphinscheduler-common have not bin dir * dolphinscheduler-common have not bin dir * delete unuse config about assembly * add defineSatement * add rpm plugin support * add rpm plugin support * add rpm plugin support * add rpm support * update install dirname , make front and backend together * update rpm name * update rpm name * update rpm config * add jars excludes * add jars excludes * add rpm plugion * add rpm plugion * add rpm plugion * add auto create /opt/soft * The install dir can not be fixed to apache-dolphinscheduler-incubating-1.2.1 , its version is 1.2.2 * install from rpm will create a link * the ui install thrown rpm only support jetty not nginx * the ui install thrown rpm only support jetty not nginx * sql目录里加上soft_version文件 * env dir need in conf dir * remove no use module * add module * updaate * rpmbuild add dolphinscheduler-service module --- .../DOLPHIN/1.2.1/package/scripts/params.py | 2 +- dolphinscheduler-dist/pom.xml | 53 +++++++++--------- dolphinscheduler-ui/pom.xml | 55 +++++++++++++++++++ pom.xml | 2 +- 4 files changed, 83 insertions(+), 29 deletions(-) diff --git a/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/params.py b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/params.py index 230ad14565..3780f6c27e 100644 --- a/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/params.py +++ b/ambari_plugin/common-services/DOLPHIN/1.2.1/package/scripts/params.py @@ -30,7 +30,7 @@ sys.setdefaultencoding('utf-8') config = Script.get_config() # conf_dir = "/etc/" -dolphin_home = "/opt/soft/apache-dolphinscheduler-incubating-1.2.1" +dolphin_home = "/opt/soft/dolphinscheduler" dolphin_conf_dir = dolphin_home + "/conf" dolphin_log_dir = dolphin_home + "/logs" dolphin_bin_dir = dolphin_home + "/bin" diff --git a/dolphinscheduler-dist/pom.xml b/dolphinscheduler-dist/pom.xml index 80ddfec9b0..0ead2b9880 100644 --- a/dolphinscheduler-dist/pom.xml +++ b/dolphinscheduler-dist/pom.xml @@ -220,10 +220,7 @@ ${basedir}/../dolphinscheduler-alert/src/main/resources - **/*.properties - **/*.xml - **/*.json - **/*.ftl + **/*.* @@ -232,9 +229,7 @@ ${basedir}/../dolphinscheduler-common/src/main/resources - **/*.properties - **/*.xml - **/*.json + **/*.* @@ -243,10 +238,7 @@ ${basedir}/../dolphinscheduler-dao/src/main/resources - **/*.properties - **/*.xml - **/*.json - **/*.yml + **/*.* @@ -255,9 +247,7 @@ ${basedir}/../dolphinscheduler-api/src/main/resources - **/*.properties - **/*.xml - **/*.json + **/*.* @@ -266,13 +256,19 @@ ${basedir}/../dolphinscheduler-server/src/main/resources - **/*.properties - **/*.xml - **/*.json config/*.* + + + ${basedir}/../dolphinscheduler-service/src/main/resources + + + *.* + + + ${basedir}/../script @@ -342,14 +338,6 @@ - - - ${basedir}/../dolphinscheduler-ui - - - install-dolphinscheduler-ui.sh - - ${basedir}/release-docs @@ -362,7 +350,7 @@ - /opt/soft/${project.build.finalName}/dist + /opt/soft/${project.build.finalName}/ui 755 root root @@ -391,6 +379,14 @@ **/*.* + + + ${basedir}/../sql + + + soft_version + + @@ -405,7 +401,7 @@ ${basedir}/../script - **/*.* + *.sh @@ -416,6 +412,9 @@ + + + diff --git a/dolphinscheduler-ui/pom.xml b/dolphinscheduler-ui/pom.xml index 3fd9aa6650..78869ffbc4 100644 --- a/dolphinscheduler-ui/pom.xml +++ b/dolphinscheduler-ui/pom.xml @@ -89,6 +89,61 @@ + + rpmbuild + + + + com.github.eirslett + frontend-maven-plugin + ${frontend-maven-plugin.version} + + + install node and npm + + install-node-and-npm + + + ${node.version} + ${npm.version} + + + + npm install node-sass --unsafe-perm + + npm + + generate-resources + + install node-sass --unsafe-perm + + + + npm install + + npm + + generate-resources + + install + + + + npm run build:release + + npm + + + run build:release + + + + + + + + + nginx diff --git a/pom.xml b/pom.xml index 8b91add6d7..8910723117 100644 --- a/pom.xml +++ b/pom.xml @@ -885,4 +885,4 @@ dolphinscheduler-service - + \ No newline at end of file From dd2c50a10d13811f7eb71a3c1bb9d5ab712f8dcd Mon Sep 17 00:00:00 2001 From: liwenhe1993 <32166572+liwenhe1993@users.noreply.github.com> Date: Fri, 13 Mar 2020 23:38:38 +0800 Subject: [PATCH 091/221] Toolbar of the DAG has not prompt (#2161) * Toolbar of the DAG has not prompt * Remove duplication code --- dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.vue | 2 +- dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js | 1 - dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js | 1 - 3 files changed, 1 insertion(+), 3 deletions(-) diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.vue index 7d6f95d753..a1ccd39260 100755 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.vue @@ -25,7 +25,7 @@ :key="v" v-for="(item,v) in tasksTypeList" @mousedown="_getDagId(v)"> -
    +
    diff --git a/dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js b/dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js index 4507250aba..64ab298437 100755 --- a/dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js +++ b/dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js @@ -567,7 +567,6 @@ export default { 'Data Target': 'Data Target', 'All Columns': 'All Columns', 'Some Columns': 'Some Columns', - 'Modify User': 'Modify User', 'Branch flow': 'Branch flow', 'Cannot select the same node for successful branch flow and failed branch flow': 'Cannot select the same node for successful branch flow and failed branch flow' } diff --git a/dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js b/dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js index 132b4d8745..3363eea198 100755 --- a/dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js +++ b/dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js @@ -567,7 +567,6 @@ export default { 'Data Target': '数据目的', 'All Columns': '全表导入', 'Some Columns': '选择列', - 'Modify User': '修改用户', 'Branch flow': '分支流转', 'Cannot select the same node for successful branch flow and failed branch flow': '成功分支流转和失败分支流转不能选择同一个节点' } From daad5efbabe15f3d37193f60083a2f4a29da54b7 Mon Sep 17 00:00:00 2001 From: "gabry.wu" Date: Fri, 13 Mar 2020 23:53:22 +0800 Subject: [PATCH 092/221] use static method in static class JSON (#2156) --- .../alert/utils/JSONUtils.java | 6 ++--- .../api/service/DataSourceService.java | 25 ++++++++++--------- .../controller/ResourcesControllerTest.java | 23 +++++++++-------- .../common/model/TaskNode.java | 4 +-- .../common/utils/JSONUtils.java | 15 +++++------ .../common/utils/ParameterUtils.java | 4 +-- .../common/utils/JSONUtilsTest.java | 4 +-- .../common/utils/ParameterUtilsTest.java | 8 +++--- .../dao/entity/ProcessDefinition.java | 8 +++--- .../master/runner/MasterExecThread.java | 4 +-- .../master/runner/MasterTaskExecThread.java | 4 +-- .../worker/runner/TaskScheduleThread.java | 6 ++--- .../task/processdure/ProcedureTask.java | 4 +-- .../server/worker/task/sql/SqlTask.java | 9 ++++--- .../server/master/MasterExecThreadTest.java | 4 +-- .../shell/ShellCommandExecutorTest.java | 4 +-- .../server/worker/sql/SqlExecutorTest.java | 4 +-- .../worker/task/sqoop/SqoopTaskTest.java | 10 ++++---- .../service/process/ProcessService.java | 7 +++--- 19 files changed, 79 insertions(+), 74 deletions(-) diff --git a/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/JSONUtils.java b/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/JSONUtils.java index 1cd9f490b2..8d66255fc4 100644 --- a/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/JSONUtils.java +++ b/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/JSONUtils.java @@ -16,8 +16,8 @@ */ package org.apache.dolphinscheduler.alert.utils; +import com.alibaba.fastjson.JSON; import com.alibaba.fastjson.JSONArray; -import com.alibaba.fastjson.JSONObject; import org.apache.dolphinscheduler.common.utils.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,7 +38,7 @@ public class JSONUtils { */ public static String toJsonString(Object object) { try{ - return JSONObject.toJSONString(object,false); + return JSON.toJSONString(object,false); } catch (Exception e) { throw new RuntimeException("Json deserialization exception.", e); } @@ -57,7 +57,7 @@ public class JSONUtils { return null; } try { - return JSONArray.parseArray(json, clazz); + return JSON.parseArray(json, clazz); } catch (Exception e) { logger.error("JSONArray.parseArray exception!",e); } diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataSourceService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataSourceService.java index f6d8903dd8..6a732fed0e 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataSourceService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/DataSourceService.java @@ -16,6 +16,7 @@ */ package org.apache.dolphinscheduler.api.service; +import com.alibaba.fastjson.JSON; import org.apache.dolphinscheduler.api.enums.Status; import org.apache.dolphinscheduler.api.utils.PageInfo; import org.apache.dolphinscheduler.api.utils.Result; @@ -303,7 +304,7 @@ public class DataSourceService extends BaseService{ for (DataSource dataSource : dataSourceList) { String connectionParams = dataSource.getConnectionParams(); - JSONObject object = JSONObject.parseObject(connectionParams); + JSONObject object = JSON.parseObject(connectionParams); object.put(Constants.PASSWORD, Constants.XXXXXX); dataSource.setConnectionParams(JSONUtils.toJson(object)); @@ -367,11 +368,11 @@ public class DataSourceService extends BaseService{ try { switch (dbType) { case POSTGRESQL: - datasource = JSONObject.parseObject(parameter, PostgreDataSource.class); + datasource = JSON.parseObject(parameter, PostgreDataSource.class); Class.forName(Constants.ORG_POSTGRESQL_DRIVER); break; case MYSQL: - datasource = JSONObject.parseObject(parameter, MySQLDataSource.class); + datasource = JSON.parseObject(parameter, MySQLDataSource.class); Class.forName(Constants.COM_MYSQL_JDBC_DRIVER); break; case HIVE: @@ -386,26 +387,26 @@ public class DataSourceService extends BaseService{ getString(org.apache.dolphinscheduler.common.Constants.LOGIN_USER_KEY_TAB_PATH)); } if (dbType == DbType.HIVE){ - datasource = JSONObject.parseObject(parameter, HiveDataSource.class); + datasource = JSON.parseObject(parameter, HiveDataSource.class); }else if (dbType == DbType.SPARK){ - datasource = JSONObject.parseObject(parameter, SparkDataSource.class); + datasource = JSON.parseObject(parameter, SparkDataSource.class); } Class.forName(Constants.ORG_APACHE_HIVE_JDBC_HIVE_DRIVER); break; case CLICKHOUSE: - datasource = JSONObject.parseObject(parameter, ClickHouseDataSource.class); + datasource = JSON.parseObject(parameter, ClickHouseDataSource.class); Class.forName(Constants.COM_CLICKHOUSE_JDBC_DRIVER); break; case ORACLE: - datasource = JSONObject.parseObject(parameter, OracleDataSource.class); + datasource = JSON.parseObject(parameter, OracleDataSource.class); Class.forName(Constants.COM_ORACLE_JDBC_DRIVER); break; case SQLSERVER: - datasource = JSONObject.parseObject(parameter, SQLServerDataSource.class); + datasource = JSON.parseObject(parameter, SQLServerDataSource.class); Class.forName(Constants.COM_SQLSERVER_JDBC_DRIVER); break; case DB2: - datasource = JSONObject.parseObject(parameter, DB2ServerDataSource.class); + datasource = JSON.parseObject(parameter, DB2ServerDataSource.class); Class.forName(Constants.COM_DB2_JDBC_DRIVER); break; default: @@ -507,7 +508,7 @@ public class DataSourceService extends BaseService{ parameterMap.put(Constants.PRINCIPAL,principal); } if (other != null && !"".equals(other)) { - LinkedHashMap map = JSONObject.parseObject(other, new TypeReference>() { + LinkedHashMap map = JSON.parseObject(other, new TypeReference>() { }); if (map.size() > 0) { StringBuilder otherSb = new StringBuilder(); @@ -523,9 +524,9 @@ public class DataSourceService extends BaseService{ } if(logger.isDebugEnabled()){ - logger.info("parameters map-----" + JSONObject.toJSONString(parameterMap)); + logger.info("parameters map-----" + JSON.toJSONString(parameterMap)); } - return JSONObject.toJSONString(parameterMap); + return JSON.toJSONString(parameterMap); } diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/ResourcesControllerTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/ResourcesControllerTest.java index 40bdd5490d..a56e3f83ef 100644 --- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/ResourcesControllerTest.java +++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/ResourcesControllerTest.java @@ -16,6 +16,7 @@ */ package org.apache.dolphinscheduler.api.controller; +import com.alibaba.fastjson.JSON; import org.apache.dolphinscheduler.api.enums.Status; import org.apache.dolphinscheduler.api.utils.Result; import org.apache.dolphinscheduler.common.enums.ResourceType; @@ -54,7 +55,7 @@ public class ResourcesControllerTest extends AbstractControllerTest{ Result result = JSONUtils.parseObject(mvcResult.getResponse().getContentAsString(), Result.class); result.getCode().equals(Status.SUCCESS.getCode()); - JSONObject object = (JSONObject) JSONObject.parse(mvcResult.getResponse().getContentAsString()); + JSONObject object = (JSONObject) JSON.parse(mvcResult.getResponse().getContentAsString()); Assert.assertEquals(Status.SUCCESS.getCode(),result.getCode().intValue()); logger.info(mvcResult.getResponse().getContentAsString()); @@ -78,7 +79,7 @@ public class ResourcesControllerTest extends AbstractControllerTest{ Result result = JSONUtils.parseObject(mvcResult.getResponse().getContentAsString(), Result.class); result.getCode().equals(Status.SUCCESS.getCode()); - JSONObject object = (JSONObject) JSONObject.parse(mvcResult.getResponse().getContentAsString()); + JSONObject object = (JSONObject) JSON.parse(mvcResult.getResponse().getContentAsString()); Assert.assertEquals(Status.SUCCESS.getCode(),result.getCode().intValue()); logger.info(mvcResult.getResponse().getContentAsString()); @@ -281,7 +282,7 @@ public class ResourcesControllerTest extends AbstractControllerTest{ Result result = JSONUtils.parseObject(mvcResult.getResponse().getContentAsString(), Result.class); result.getCode().equals(Status.SUCCESS.getCode()); - JSONObject object = (JSONObject) JSONObject.parse(mvcResult.getResponse().getContentAsString()); + JSONObject object = (JSONObject) JSON.parse(mvcResult.getResponse().getContentAsString()); Assert.assertEquals(Status.SUCCESS.getCode(),result.getCode().intValue()); logger.info(mvcResult.getResponse().getContentAsString()); @@ -303,7 +304,7 @@ public class ResourcesControllerTest extends AbstractControllerTest{ Result result = JSONUtils.parseObject(mvcResult.getResponse().getContentAsString(), Result.class); result.getCode().equals(Status.SUCCESS.getCode()); - JSONObject object = (JSONObject) JSONObject.parse(mvcResult.getResponse().getContentAsString()); + JSONObject object = (JSONObject) JSON.parse(mvcResult.getResponse().getContentAsString()); Assert.assertEquals(Status.SUCCESS.getCode(),result.getCode().intValue()); logger.info(mvcResult.getResponse().getContentAsString()); @@ -324,7 +325,7 @@ public class ResourcesControllerTest extends AbstractControllerTest{ Result result = JSONUtils.parseObject(mvcResult.getResponse().getContentAsString(), Result.class); result.getCode().equals(Status.SUCCESS.getCode()); - JSONObject object = (JSONObject) JSONObject.parse(mvcResult.getResponse().getContentAsString()); + JSONObject object = (JSONObject) JSON.parse(mvcResult.getResponse().getContentAsString()); Assert.assertEquals(Status.SUCCESS.getCode(),result.getCode().intValue()); logger.info(mvcResult.getResponse().getContentAsString()); @@ -344,7 +345,7 @@ public class ResourcesControllerTest extends AbstractControllerTest{ Result result = JSONUtils.parseObject(mvcResult.getResponse().getContentAsString(), Result.class); result.getCode().equals(Status.SUCCESS.getCode()); - JSONObject object = (JSONObject) JSONObject.parse(mvcResult.getResponse().getContentAsString()); + JSONObject object = (JSONObject) JSON.parse(mvcResult.getResponse().getContentAsString()); Assert.assertEquals(Status.SUCCESS.getCode(),result.getCode().intValue()); logger.info(mvcResult.getResponse().getContentAsString()); @@ -365,7 +366,7 @@ public class ResourcesControllerTest extends AbstractControllerTest{ Result result = JSONUtils.parseObject(mvcResult.getResponse().getContentAsString(), Result.class); result.getCode().equals(Status.SUCCESS.getCode()); - JSONObject object = (JSONObject) JSONObject.parse(mvcResult.getResponse().getContentAsString()); + JSONObject object = (JSONObject) JSON.parse(mvcResult.getResponse().getContentAsString()); Assert.assertEquals(Status.SUCCESS.getCode(),result.getCode().intValue()); logger.info(mvcResult.getResponse().getContentAsString()); @@ -386,7 +387,7 @@ public class ResourcesControllerTest extends AbstractControllerTest{ Result result = JSONUtils.parseObject(mvcResult.getResponse().getContentAsString(), Result.class); result.getCode().equals(Status.SUCCESS.getCode()); - JSONObject object = (JSONObject) JSONObject.parse(mvcResult.getResponse().getContentAsString()); + JSONObject object = (JSONObject) JSON.parse(mvcResult.getResponse().getContentAsString()); Assert.assertEquals(Status.SUCCESS.getCode(),result.getCode().intValue()); logger.info(mvcResult.getResponse().getContentAsString()); @@ -406,7 +407,7 @@ public class ResourcesControllerTest extends AbstractControllerTest{ Result result = JSONUtils.parseObject(mvcResult.getResponse().getContentAsString(), Result.class); result.getCode().equals(Status.SUCCESS.getCode()); - JSONObject object = (JSONObject) JSONObject.parse(mvcResult.getResponse().getContentAsString()); + JSONObject object = (JSONObject) JSON.parse(mvcResult.getResponse().getContentAsString()); Assert.assertEquals(Status.SUCCESS.getCode(),result.getCode().intValue()); logger.info(mvcResult.getResponse().getContentAsString()); @@ -427,7 +428,7 @@ public class ResourcesControllerTest extends AbstractControllerTest{ Result result = JSONUtils.parseObject(mvcResult.getResponse().getContentAsString(), Result.class); result.getCode().equals(Status.SUCCESS.getCode()); - JSONObject object = (JSONObject) JSONObject.parse(mvcResult.getResponse().getContentAsString()); + JSONObject object = (JSONObject) JSON.parse(mvcResult.getResponse().getContentAsString()); Assert.assertEquals(Status.SUCCESS.getCode(),result.getCode().intValue()); logger.info(mvcResult.getResponse().getContentAsString()); @@ -446,7 +447,7 @@ public class ResourcesControllerTest extends AbstractControllerTest{ Result result = JSONUtils.parseObject(mvcResult.getResponse().getContentAsString(), Result.class); result.getCode().equals(Status.SUCCESS.getCode()); - JSONObject object = (JSONObject) JSONObject.parse(mvcResult.getResponse().getContentAsString()); + JSONObject object = (JSONObject) JSON.parse(mvcResult.getResponse().getContentAsString()); Assert.assertEquals(Status.SUCCESS.getCode(),result.getCode().intValue()); logger.info(mvcResult.getResponse().getContentAsString()); diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/TaskNode.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/TaskNode.java index b45bd8aeb8..c0ad907dca 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/TaskNode.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/model/TaskNode.java @@ -16,6 +16,7 @@ */ package org.apache.dolphinscheduler.common.model; +import com.alibaba.fastjson.JSON; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.Priority; import org.apache.dolphinscheduler.common.enums.TaskTimeoutStrategy; @@ -23,7 +24,6 @@ import org.apache.dolphinscheduler.common.enums.TaskType; import org.apache.dolphinscheduler.common.task.TaskTimeoutParameter; import org.apache.dolphinscheduler.common.utils.CollectionUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils; -import com.alibaba.fastjson.JSONObject; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.annotation.JsonDeserialize; import com.fasterxml.jackson.databind.annotation.JsonSerialize; @@ -294,7 +294,7 @@ public class TaskNode { if(StringUtils.isNotEmpty(this.getTimeout())){ String formatStr = String.format("%s,%s", TaskTimeoutStrategy.WARN.name(), TaskTimeoutStrategy.FAILED.name()); String timeout = this.getTimeout().replace(formatStr,TaskTimeoutStrategy.WARNFAILED.name()); - return JSONObject.parseObject(timeout,TaskTimeoutParameter.class); + return JSON.parseObject(timeout,TaskTimeoutParameter.class); } return new TaskTimeoutParameter(false); } diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/JSONUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/JSONUtils.java index ec523b1ff2..b6ed71f402 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/JSONUtils.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/JSONUtils.java @@ -16,6 +16,7 @@ */ package org.apache.dolphinscheduler.common.utils; +import com.alibaba.fastjson.JSON; import com.alibaba.fastjson.JSONArray; import com.alibaba.fastjson.JSONObject; import com.alibaba.fastjson.TypeReference; @@ -59,7 +60,7 @@ public class JSONUtils { */ public static String toJson(Object object) { try{ - return JSONObject.toJSONString(object,false); + return JSON.toJSONString(object,false); } catch (Exception e) { logger.error("object to json exception!",e); } @@ -89,7 +90,7 @@ public class JSONUtils { } try { - return JSONObject.parseObject(json, clazz); + return JSON.parseObject(json, clazz); } catch (Exception e) { logger.error("parse object exception!",e); } @@ -178,7 +179,7 @@ public class JSONUtils { } try { - return JSONObject.parseObject(json, new TypeReference>(){}); + return JSON.parseObject(json, new TypeReference>(){}); } catch (Exception e) { logger.error("json to map exception!",e); } @@ -203,7 +204,7 @@ public class JSONUtils { } try { - return JSONObject.parseObject(json, new TypeReference>() {}); + return JSON.parseObject(json, new TypeReference>() {}); } catch (Exception e) { logger.error("json to map exception!",e); } @@ -218,7 +219,7 @@ public class JSONUtils { */ public static String toJsonString(Object object) { try{ - return JSONObject.toJSONString(object,false); + return JSON.toJSONString(object,false); } catch (Exception e) { throw new RuntimeException("Json deserialization exception.", e); } @@ -226,7 +227,7 @@ public class JSONUtils { public static JSONObject parseObject(String text) { try{ - return JSONObject.parseObject(text); + return JSON.parseObject(text); } catch (Exception e) { throw new RuntimeException("Json deserialization exception.", e); } @@ -234,7 +235,7 @@ public class JSONUtils { public static JSONArray parseArray(String text) { try{ - return JSONObject.parseArray(text); + return JSON.parseArray(text); } catch (Exception e) { throw new RuntimeException("Json deserialization exception.", e); } diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/ParameterUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/ParameterUtils.java index 9492b49cb1..7a4553aaf5 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/ParameterUtils.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/ParameterUtils.java @@ -16,6 +16,7 @@ */ package org.apache.dolphinscheduler.common.utils; +import com.alibaba.fastjson.JSON; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.CommandType; import org.apache.dolphinscheduler.common.enums.DataType; @@ -23,7 +24,6 @@ import org.apache.dolphinscheduler.common.process.Property; import org.apache.dolphinscheduler.common.utils.placeholder.BusinessTimeUtils; import org.apache.dolphinscheduler.common.utils.placeholder.PlaceholderUtils; import org.apache.dolphinscheduler.common.utils.placeholder.TimePlaceholderUtils; -import com.alibaba.fastjson.JSONObject; import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.time.DateUtils; import org.slf4j.Logger; @@ -157,7 +157,7 @@ public class ParameterUtils { property.setValue(val); } } - return JSONObject.toJSONString(globalParamList); + return JSON.toJSONString(globalParamList); } diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/JSONUtilsTest.java b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/JSONUtilsTest.java index bd924e4852..d249314819 100644 --- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/JSONUtilsTest.java +++ b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/JSONUtilsTest.java @@ -16,10 +16,10 @@ */ package org.apache.dolphinscheduler.common.utils; +import com.alibaba.fastjson.JSON; import org.apache.dolphinscheduler.common.enums.DataType; import org.apache.dolphinscheduler.common.enums.Direct; import org.apache.dolphinscheduler.common.process.Property; -import com.alibaba.fastjson.JSONObject; import com.fasterxml.jackson.databind.node.ArrayNode; import com.fasterxml.jackson.databind.node.JsonNodeFactory; import org.junit.Assert; @@ -53,7 +53,7 @@ public class JSONUtilsTest { property.setType(DataType.VARCHAR); property.setValue("sssssss"); String str = "{\"direct\":\"IN\",\"prop\":\"ds\",\"type\":\"VARCHAR\",\"value\":\"sssssss\"}"; - Property property1 = JSONObject.parseObject(str, Property.class); + Property property1 = JSON.parseObject(str, Property.class); Direct direct = property1.getDirect(); Assert.assertEquals(direct , Direct.IN); } diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/ParameterUtilsTest.java b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/ParameterUtilsTest.java index 8bb64b03c8..abdc15cc6e 100644 --- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/ParameterUtilsTest.java +++ b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/ParameterUtilsTest.java @@ -16,7 +16,7 @@ */ package org.apache.dolphinscheduler.common.utils; -import com.alibaba.fastjson.JSONObject; +import com.alibaba.fastjson.JSON; import org.apache.commons.lang.time.DateUtils; import org.apache.dolphinscheduler.common.enums.CommandType; import org.apache.dolphinscheduler.common.enums.DataType; @@ -91,13 +91,13 @@ public class ParameterUtilsTest { globalParamList.add(property); String result2 = ParameterUtils.curingGlobalParams(null,globalParamList,CommandType.START_CURRENT_TASK_PROCESS,scheduleTime); - Assert.assertEquals(result2, JSONObject.toJSONString(globalParamList)); + Assert.assertEquals(result2, JSON.toJSONString(globalParamList)); String result3 = ParameterUtils.curingGlobalParams(globalParamMap,globalParamList,CommandType.START_CURRENT_TASK_PROCESS,null); - Assert.assertEquals(result3, JSONObject.toJSONString(globalParamList)); + Assert.assertEquals(result3, JSON.toJSONString(globalParamList)); String result4 = ParameterUtils.curingGlobalParams(globalParamMap, globalParamList, CommandType.START_CURRENT_TASK_PROCESS, scheduleTime); - Assert.assertEquals(result4, JSONObject.toJSONString(globalParamList)); + Assert.assertEquals(result4, JSON.toJSONString(globalParamList)); //test var $ startsWith globalParamMap.put("bizDate","${system.biz.date}"); diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessDefinition.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessDefinition.java index dbb880c025..f59d11f3fe 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessDefinition.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessDefinition.java @@ -16,10 +16,10 @@ */ package org.apache.dolphinscheduler.dao.entity; +import com.alibaba.fastjson.JSON; import org.apache.dolphinscheduler.common.enums.Flag; import org.apache.dolphinscheduler.common.enums.ReleaseState; import org.apache.dolphinscheduler.common.process.Property; -import com.alibaba.fastjson.JSONObject; import com.baomidou.mybatisplus.annotation.IdType; import com.baomidou.mybatisplus.annotation.TableField; import com.baomidou.mybatisplus.annotation.TableId; @@ -266,7 +266,7 @@ public class ProcessDefinition { } public void setGlobalParams(String globalParams) { - this.globalParamList = JSONObject.parseArray(globalParams, Property.class); + this.globalParamList = JSON.parseArray(globalParams, Property.class); this.globalParams = globalParams; } @@ -275,7 +275,7 @@ public class ProcessDefinition { } public void setGlobalParamList(List globalParamList) { - this.globalParams = JSONObject.toJSONString(globalParamList); + this.globalParams = JSON.toJSONString(globalParamList); this.globalParamList = globalParamList; } @@ -283,7 +283,7 @@ public class ProcessDefinition { List propList; if (globalParamMap == null && StringUtils.isNotEmpty(globalParams)) { - propList = JSONObject.parseArray(globalParams, Property.class); + propList = JSON.parseArray(globalParams, Property.class); globalParamMap = propList.stream().collect(Collectors.toMap(Property::getProp, Property::getValue)); } diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java index 0b81a30191..849bbd5680 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java @@ -16,7 +16,7 @@ */ package org.apache.dolphinscheduler.server.master.runner; -import com.alibaba.fastjson.JSONObject; +import com.alibaba.fastjson.JSON; import com.google.common.collect.Lists; import org.apache.commons.io.FileUtils; import org.apache.dolphinscheduler.common.Constants; @@ -454,7 +454,7 @@ public class MasterExecThread implements Runnable { // process instance id taskInstance.setProcessInstanceId(processInstance.getId()); // task instance node json - taskInstance.setTaskJson(JSONObject.toJSONString(taskNode)); + taskInstance.setTaskJson(JSON.toJSONString(taskNode)); // task instance type taskInstance.setTaskType(taskNode.getType()); // task instance whether alert diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecThread.java index 37262c05e2..3cdb8a0271 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecThread.java @@ -16,6 +16,7 @@ */ package org.apache.dolphinscheduler.server.master.runner; +import com.alibaba.fastjson.JSON; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.enums.TaskTimeoutStrategy; @@ -25,7 +26,6 @@ import org.apache.dolphinscheduler.common.thread.Stopper; import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; import org.apache.dolphinscheduler.dao.entity.ProcessInstance; import org.apache.dolphinscheduler.dao.entity.TaskInstance; -import com.alibaba.fastjson.JSONObject; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -176,7 +176,7 @@ public class MasterTaskExecThread extends MasterBaseTaskExecThread { */ private TaskTimeoutParameter getTaskTimeoutParameter(){ String taskJson = taskInstance.getTaskJson(); - TaskNode taskNode = JSONObject.parseObject(taskJson, TaskNode.class); + TaskNode taskNode = JSON.parseObject(taskJson, TaskNode.class); return taskNode.getTaskTimeoutParameter(); } diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/TaskScheduleThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/TaskScheduleThread.java index 21ee1dfa40..d36d4de5b4 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/TaskScheduleThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/runner/TaskScheduleThread.java @@ -19,7 +19,7 @@ package org.apache.dolphinscheduler.server.worker.runner; import ch.qos.logback.classic.LoggerContext; import ch.qos.logback.classic.sift.SiftingAppender; -import com.alibaba.fastjson.JSONObject; +import com.alibaba.fastjson.JSON; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.AuthorizationType; import org.apache.dolphinscheduler.common.enums.ExecutionStatus; @@ -93,7 +93,7 @@ public class TaskScheduleThread implements Runnable { logger.info("script path : {}", taskInstance.getExecutePath()); // task node - TaskNode taskNode = JSONObject.parseObject(taskInstance.getTaskJson(), TaskNode.class); + TaskNode taskNode = JSON.parseObject(taskInstance.getTaskJson(), TaskNode.class); // get resource files List resourceFiles = createProjectResFiles(taskNode); @@ -176,7 +176,7 @@ public class TaskScheduleThread implements Runnable { String globalParamsStr = taskInstance.getProcessInstance().getGlobalParams(); if (globalParamsStr != null) { - List globalParamsList = JSONObject.parseArray(globalParamsStr, Property.class); + List globalParamsList = JSON.parseArray(globalParamsStr, Property.class); globalParamsMap.putAll(globalParamsList.stream().collect(Collectors.toMap(Property::getProp, Property::getValue))); } return globalParamsMap; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java index fb881453e9..fd00e517b5 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java @@ -16,7 +16,7 @@ */ package org.apache.dolphinscheduler.server.worker.task.processdure; -import com.alibaba.fastjson.JSONObject; +import com.alibaba.fastjson.JSON; import com.cronutils.utils.StringUtils; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.DataType; @@ -75,7 +75,7 @@ public class ProcedureTask extends AbstractTask { logger.info("procedure task params {}", taskProps.getTaskParams()); - this.procedureParameters = JSONObject.parseObject(taskProps.getTaskParams(), ProcedureParameters.class); + this.procedureParameters = JSON.parseObject(taskProps.getTaskParams(), ProcedureParameters.class); // check parameters if (!procedureParameters.checkParameters()) { diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java index aae11f5530..82034740fc 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java @@ -16,6 +16,7 @@ */ package org.apache.dolphinscheduler.server.worker.task.sql; +import com.alibaba.fastjson.JSON; import com.alibaba.fastjson.JSONArray; import com.alibaba.fastjson.JSONObject; import com.alibaba.fastjson.serializer.SerializerFeature; @@ -92,7 +93,7 @@ public class SqlTask extends AbstractTask { super(taskProps, logger); logger.info("sql task params {}", taskProps.getTaskParams()); - this.sqlParameters = JSONObject.parseObject(taskProps.getTaskParams(), SqlParameters.class); + this.sqlParameters = JSON.parseObject(taskProps.getTaskParams(), SqlParameters.class); if (!sqlParameters.checkParameters()) { throw new RuntimeException("sql task params is not valid"); @@ -308,16 +309,16 @@ public class SqlTask extends AbstractTask { } resultJSONArray.add(mapOfColValues); } - logger.debug("execute sql : {}", JSONObject.toJSONString(resultJSONArray, SerializerFeature.WriteMapNullValue)); + logger.debug("execute sql : {}", JSON.toJSONString(resultJSONArray, SerializerFeature.WriteMapNullValue)); // if there is a result set if ( !resultJSONArray.isEmpty() ) { if (StringUtils.isNotEmpty(sqlParameters.getTitle())) { sendAttachment(sqlParameters.getTitle(), - JSONObject.toJSONString(resultJSONArray, SerializerFeature.WriteMapNullValue)); + JSON.toJSONString(resultJSONArray, SerializerFeature.WriteMapNullValue)); }else{ sendAttachment(taskProps.getNodeName() + " query resultsets ", - JSONObject.toJSONString(resultJSONArray, SerializerFeature.WriteMapNullValue)); + JSON.toJSONString(resultJSONArray, SerializerFeature.WriteMapNullValue)); } } diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterExecThreadTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterExecThreadTest.java index d2a0fb2407..af312d9601 100644 --- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterExecThreadTest.java +++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/MasterExecThreadTest.java @@ -16,7 +16,7 @@ */ package org.apache.dolphinscheduler.server.master; -import com.alibaba.fastjson.JSONObject; +import com.alibaba.fastjson.JSON; import org.apache.dolphinscheduler.common.enums.*; import org.apache.dolphinscheduler.common.graph.DAG; import org.apache.dolphinscheduler.common.utils.DateUtils; @@ -85,7 +85,7 @@ public class MasterExecThreadTest { Map cmdParam = new HashMap<>(); cmdParam.put(CMDPARAM_COMPLEMENT_DATA_START_DATE, "2020-01-01 00:00:00"); cmdParam.put(CMDPARAM_COMPLEMENT_DATA_END_DATE, "2020-01-31 23:00:00"); - Mockito.when(processInstance.getCommandParam()).thenReturn(JSONObject.toJSONString(cmdParam)); + Mockito.when(processInstance.getCommandParam()).thenReturn(JSON.toJSONString(cmdParam)); ProcessDefinition processDefinition = new ProcessDefinition(); processDefinition.setGlobalParamMap(Collections.EMPTY_MAP); processDefinition.setGlobalParamList(Collections.EMPTY_LIST); diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/shell/ShellCommandExecutorTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/shell/ShellCommandExecutorTest.java index 5d4263644b..250c8a2680 100644 --- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/shell/ShellCommandExecutorTest.java +++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/shell/ShellCommandExecutorTest.java @@ -16,7 +16,7 @@ */ package org.apache.dolphinscheduler.server.worker.shell; -import com.alibaba.fastjson.JSONObject; +import com.alibaba.fastjson.JSON; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.model.TaskNode; @@ -68,7 +68,7 @@ public class ShellCommandExecutorTest { TaskInstance taskInstance = processService.findTaskInstanceById(7657); String taskJson = taskInstance.getTaskJson(); - TaskNode taskNode = JSONObject.parseObject(taskJson, TaskNode.class); + TaskNode taskNode = JSON.parseObject(taskJson, TaskNode.class); taskProps.setTaskParams(taskNode.getParams()); diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/sql/SqlExecutorTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/sql/SqlExecutorTest.java index c395eabe51..07b700239b 100644 --- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/sql/SqlExecutorTest.java +++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/sql/SqlExecutorTest.java @@ -16,7 +16,7 @@ */ package org.apache.dolphinscheduler.server.worker.sql; -import com.alibaba.fastjson.JSONObject; +import com.alibaba.fastjson.JSON; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.CommandType; import org.apache.dolphinscheduler.common.enums.ExecutionStatus; @@ -112,7 +112,7 @@ public class SqlExecutorTest { TaskInstance taskInstance = processService.findTaskInstanceById(taskInstId); String taskJson = taskInstance.getTaskJson(); - TaskNode taskNode = JSONObject.parseObject(taskJson, TaskNode.class); + TaskNode taskNode = JSON.parseObject(taskJson, TaskNode.class); taskProps.setTaskParams(taskNode.getParams()); diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/sqoop/SqoopTaskTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/sqoop/SqoopTaskTest.java index 511102e4b5..5c7afa8155 100644 --- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/sqoop/SqoopTaskTest.java +++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/task/sqoop/SqoopTaskTest.java @@ -16,7 +16,7 @@ */ package org.apache.dolphinscheduler.server.worker.task.sqoop; -import com.alibaba.fastjson.JSONObject; +import com.alibaba.fastjson.JSON; import org.apache.dolphinscheduler.common.enums.DbType; import org.apache.dolphinscheduler.common.task.sqoop.SqoopParameters; import org.apache.dolphinscheduler.dao.entity.DataSource; @@ -74,7 +74,7 @@ public class SqoopTaskTest { @Test public void testGenerator(){ String data1 = "{\"concurrency\":1,\"modelType\":\"import\",\"sourceType\":\"MYSQL\",\"targetType\":\"HDFS\",\"sourceParams\":\"{\\\"srcDatasource\\\":2,\\\"srcTable\\\":\\\"person_2\\\",\\\"srcQueryType\\\":\\\"0\\\",\\\"srcQuerySql\\\":\\\"\\\",\\\"srcColumnType\\\":\\\"0\\\",\\\"srcColumns\\\":\\\"\\\",\\\"srcConditionList\\\":[],\\\"mapColumnHive\\\":[],\\\"mapColumnJava\\\":[]}\",\"targetParams\":\"{\\\"targetPath\\\":\\\"/ods/tmp/test/person7\\\",\\\"deleteTargetDir\\\":true,\\\"fileType\\\":\\\"--as-textfile\\\",\\\"compressionCodec\\\":\\\"\\\",\\\"fieldsTerminated\\\":\\\"@\\\",\\\"linesTerminated\\\":\\\"\\\\\\\\n\\\"}\",\"localParams\":[]}"; - SqoopParameters sqoopParameters1 = JSONObject.parseObject(data1,SqoopParameters.class); + SqoopParameters sqoopParameters1 = JSON.parseObject(data1,SqoopParameters.class); SqoopJobGenerator generator = new SqoopJobGenerator(); String script = generator.generateSqoopJob(sqoopParameters1); @@ -82,21 +82,21 @@ public class SqoopTaskTest { Assert.assertEquals(expected, script); String data2 = "{\"concurrency\":1,\"modelType\":\"export\",\"sourceType\":\"HDFS\",\"targetType\":\"MYSQL\",\"sourceParams\":\"{\\\"exportDir\\\":\\\"/ods/tmp/test/person7\\\"}\",\"targetParams\":\"{\\\"targetDatasource\\\":2,\\\"targetTable\\\":\\\"person_3\\\",\\\"targetColumns\\\":\\\"id,name,age,sex,create_time\\\",\\\"preQuery\\\":\\\"\\\",\\\"isUpdate\\\":true,\\\"targetUpdateKey\\\":\\\"id\\\",\\\"targetUpdateMode\\\":\\\"allowinsert\\\",\\\"fieldsTerminated\\\":\\\"@\\\",\\\"linesTerminated\\\":\\\"\\\\\\\\n\\\"}\",\"localParams\":[]}"; - SqoopParameters sqoopParameters2 = JSONObject.parseObject(data2,SqoopParameters.class); + SqoopParameters sqoopParameters2 = JSON.parseObject(data2,SqoopParameters.class); String script2 = generator.generateSqoopJob(sqoopParameters2); String expected2 = "sqoop export -m 1 --export-dir /ods/tmp/test/person7 --connect jdbc:mysql://192.168.0.111:3306/test --username kylo --password 123456 --table person_3 --columns id,name,age,sex,create_time --fields-terminated-by '@' --lines-terminated-by '\\n' --update-key id --update-mode allowinsert"; Assert.assertEquals(expected2, script2); String data3 = "{\"concurrency\":1,\"modelType\":\"export\",\"sourceType\":\"HIVE\",\"targetType\":\"MYSQL\",\"sourceParams\":\"{\\\"hiveDatabase\\\":\\\"stg\\\",\\\"hiveTable\\\":\\\"person_internal\\\",\\\"hivePartitionKey\\\":\\\"date\\\",\\\"hivePartitionValue\\\":\\\"2020-02-17\\\"}\",\"targetParams\":\"{\\\"targetDatasource\\\":2,\\\"targetTable\\\":\\\"person_3\\\",\\\"targetColumns\\\":\\\"\\\",\\\"preQuery\\\":\\\"\\\",\\\"isUpdate\\\":false,\\\"targetUpdateKey\\\":\\\"\\\",\\\"targetUpdateMode\\\":\\\"allowinsert\\\",\\\"fieldsTerminated\\\":\\\"@\\\",\\\"linesTerminated\\\":\\\"\\\\\\\\n\\\"}\",\"localParams\":[]}"; - SqoopParameters sqoopParameters3 = JSONObject.parseObject(data3,SqoopParameters.class); + SqoopParameters sqoopParameters3 = JSON.parseObject(data3,SqoopParameters.class); String script3 = generator.generateSqoopJob(sqoopParameters3); String expected3 = "sqoop export -m 1 --hcatalog-database stg --hcatalog-table person_internal --hcatalog-partition-keys date --hcatalog-partition-values 2020-02-17 --connect jdbc:mysql://192.168.0.111:3306/test --username kylo --password 123456 --table person_3 --fields-terminated-by '@' --lines-terminated-by '\\n'"; Assert.assertEquals(expected3, script3); String data4 = "{\"concurrency\":1,\"modelType\":\"import\",\"sourceType\":\"MYSQL\",\"targetType\":\"HIVE\",\"sourceParams\":\"{\\\"srcDatasource\\\":2,\\\"srcTable\\\":\\\"person_2\\\",\\\"srcQueryType\\\":\\\"1\\\",\\\"srcQuerySql\\\":\\\"SELECT * FROM person_2\\\",\\\"srcColumnType\\\":\\\"0\\\",\\\"srcColumns\\\":\\\"\\\",\\\"srcConditionList\\\":[],\\\"mapColumnHive\\\":[],\\\"mapColumnJava\\\":[{\\\"prop\\\":\\\"id\\\",\\\"direct\\\":\\\"IN\\\",\\\"type\\\":\\\"VARCHAR\\\",\\\"value\\\":\\\"Integer\\\"}]}\",\"targetParams\":\"{\\\"hiveDatabase\\\":\\\"stg\\\",\\\"hiveTable\\\":\\\"person_internal_2\\\",\\\"createHiveTable\\\":true,\\\"dropDelimiter\\\":false,\\\"hiveOverWrite\\\":true,\\\"replaceDelimiter\\\":\\\"\\\",\\\"hivePartitionKey\\\":\\\"date\\\",\\\"hivePartitionValue\\\":\\\"2020-02-16\\\"}\",\"localParams\":[]}"; - SqoopParameters sqoopParameters4 = JSONObject.parseObject(data4,SqoopParameters.class); + SqoopParameters sqoopParameters4 = JSON.parseObject(data4,SqoopParameters.class); String script4 = generator.generateSqoopJob(sqoopParameters4); String expected4 = "sqoop import -m 1 --connect jdbc:mysql://192.168.0.111:3306/test --username kylo --password 123456 --query 'SELECT * FROM person_2 WHERE $CONDITIONS' --map-column-java id=Integer --hive-import --hive-table stg.person_internal_2 --create-hive-table --hive-overwrite -delete-target-dir --hive-partition-key date --hive-partition-value 2020-02-16"; diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java index 8bcd64f1fd..c848ec5197 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java @@ -16,6 +16,7 @@ */ package org.apache.dolphinscheduler.service.process; +import com.alibaba.fastjson.JSON; import com.alibaba.fastjson.JSONObject; import com.cronutils.model.Cron; import org.apache.commons.lang.ArrayUtils; @@ -207,7 +208,7 @@ public class ProcessService { CommandType commandType = command.getCommandType(); if(cmdTypeMap.containsKey(commandType)){ - JSONObject cmdParamObj = (JSONObject) JSONObject.parse(command.getCommandParam()); + JSONObject cmdParamObj = (JSONObject) JSON.parse(command.getCommandParam()); JSONObject tempObj; int processInstanceId = cmdParamObj.getInteger(CMDPARAM_RECOVER_PROCESS_ID_STRING); @@ -215,7 +216,7 @@ public class ProcessService { // for all commands for (Command tmpCommand:commands){ if(cmdTypeMap.containsKey(tmpCommand.getCommandType())){ - tempObj = (JSONObject) JSONObject.parse(tmpCommand.getCommandParam()); + tempObj = (JSONObject) JSON.parse(tmpCommand.getCommandParam()); if(tempObj != null && processInstanceId == tempObj.getInteger(CMDPARAM_RECOVER_PROCESS_ID_STRING)){ isNeedCreate = false; break; @@ -309,7 +310,7 @@ public class ProcessService { for (TaskNode taskNode : taskNodeList){ String parameter = taskNode.getParams(); if (parameter.contains(CMDPARAM_SUB_PROCESS_DEFINE_ID)){ - SubProcessParameters subProcessParam = JSONObject.parseObject(parameter, SubProcessParameters.class); + SubProcessParameters subProcessParam = JSON.parseObject(parameter, SubProcessParameters.class); ids.add(subProcessParam.getProcessDefinitionId()); recurseFindSubProcessId(subProcessParam.getProcessDefinitionId(),ids); } From 205af16d55f6e16a7db9f20e8076b4658934eed3 Mon Sep 17 00:00:00 2001 From: DS Date: Sat, 14 Mar 2020 08:30:41 +0800 Subject: [PATCH 093/221] Create .asf.yaml (#2166) --- .asf.yaml | 4 ++++ 1 file changed, 4 insertions(+) create mode 100644 .asf.yaml diff --git a/.asf.yaml b/.asf.yaml new file mode 100644 index 0000000000..650e3c7e75 --- /dev/null +++ b/.asf.yaml @@ -0,0 +1,4 @@ +# Web site staging services: + +github: + description: "Dolphin Scheduler is a distributed and easy-to-extend visual workflow scheduling platform, dedicated to solving the complex dependencies in data processing, making the scheduling system out of the box for data processing.(分布式易扩展的可视化工作流任务调度)" From e98c6f3008930e7f96943d55636a50e3ff2c05d3 Mon Sep 17 00:00:00 2001 From: DS Date: Sat, 14 Mar 2020 08:42:55 +0800 Subject: [PATCH 094/221] Update .asf.yaml (#2167) --- .asf.yaml | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/.asf.yaml b/.asf.yaml index 650e3c7e75..7e950a0a53 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -1,4 +1,7 @@ -# Web site staging services: +staging: + profile: ~ + whoami: master + foo: trigger github: description: "Dolphin Scheduler is a distributed and easy-to-extend visual workflow scheduling platform, dedicated to solving the complex dependencies in data processing, making the scheduling system out of the box for data processing.(分布式易扩展的可视化工作流任务调度)" From 2e63dc9c96b1fe68407ba7adf2cb237f785f7831 Mon Sep 17 00:00:00 2001 From: DS Date: Sat, 14 Mar 2020 08:47:23 +0800 Subject: [PATCH 095/221] Delete .asf.yaml (#2168) --- .asf.yaml | 7 ------- 1 file changed, 7 deletions(-) delete mode 100644 .asf.yaml diff --git a/.asf.yaml b/.asf.yaml deleted file mode 100644 index 7e950a0a53..0000000000 --- a/.asf.yaml +++ /dev/null @@ -1,7 +0,0 @@ -staging: - profile: ~ - whoami: master - foo: trigger - -github: - description: "Dolphin Scheduler is a distributed and easy-to-extend visual workflow scheduling platform, dedicated to solving the complex dependencies in data processing, making the scheduling system out of the box for data processing.(分布式易扩展的可视化工作流任务调度)" From 80c6ce5711ab77e104df83ea70853d86916135b7 Mon Sep 17 00:00:00 2001 From: Rubik-W <39549317+Rubik-W@users.noreply.github.com> Date: Sat, 14 Mar 2020 18:10:27 +0800 Subject: [PATCH 096/221] The master and worker server exit exception #2163 (#2176) * fix: #2163 * fix: format --- .../common/thread/ThreadUtils.java | 17 +++++-- .../server/master/MasterServer.java | 33 +++++++------- .../server/worker/WorkerServer.java | 45 +++++++------------ 3 files changed, 46 insertions(+), 49 deletions(-) diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/thread/ThreadUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/thread/ThreadUtils.java index 0a4ed9b5ac..d8ef0bb38d 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/thread/ThreadUtils.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/thread/ThreadUtils.java @@ -120,12 +120,24 @@ public class ThreadUtils { /** * Wrapper over ScheduledThreadPoolExecutor + * @param threadName * @param corePoolSize * @return */ - public static ScheduledExecutorService newDaemonThreadScheduledExecutor(String threadName,int corePoolSize) { + public static ScheduledExecutorService newDaemonThreadScheduledExecutor(String threadName, int corePoolSize) { + return newThreadScheduledExecutor(threadName, corePoolSize, true); + } + + /** + * Wrapper over ScheduledThreadPoolExecutor + * @param threadName + * @param corePoolSize + * @param isDaemon + * @return + */ + public static ScheduledExecutorService newThreadScheduledExecutor(String threadName, int corePoolSize, boolean isDaemon) { ThreadFactory threadFactory = new ThreadFactoryBuilder() - .setDaemon(true) + .setDaemon(isDaemon) .setNameFormat(threadName) .build(); ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(corePoolSize, threadFactory); @@ -135,7 +147,6 @@ public class ThreadUtils { return executor; } - public static ThreadInfo getThreadInfo(Thread t) { long tid = t.getId(); return threadBean.getThreadInfo(tid, STACK_DEPTH); diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java index 6b5063cba4..9512b1a1c2 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java @@ -25,6 +25,7 @@ import org.apache.dolphinscheduler.common.utils.OSUtils; import org.apache.dolphinscheduler.common.utils.StringUtils; import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.runner.MasterSchedulerThread; +import org.apache.dolphinscheduler.server.worker.WorkerServer; import org.apache.dolphinscheduler.server.zk.ZKMasterClient; import org.apache.dolphinscheduler.service.bean.SpringApplicationContext; import org.apache.dolphinscheduler.service.process.ProcessService; @@ -37,8 +38,10 @@ import org.springframework.beans.factory.annotation.Autowired; import org.springframework.boot.WebApplicationType; import org.springframework.boot.builder.SpringApplicationBuilder; import org.springframework.context.annotation.ComponentScan; +import org.springframework.context.annotation.FilterType; import javax.annotation.PostConstruct; +import javax.annotation.PreDestroy; import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -46,7 +49,9 @@ import java.util.concurrent.TimeUnit; /** * master server */ -@ComponentScan("org.apache.dolphinscheduler") +@ComponentScan(value = "org.apache.dolphinscheduler", excludeFilters = { + @ComponentScan.Filter(type = FilterType.ASSIGNABLE_TYPE, classes = {WorkerServer.class}) +}) public class MasterServer implements IStoppable { /** @@ -112,7 +117,7 @@ public class MasterServer implements IStoppable { masterSchedulerService = ThreadUtils.newDaemonSingleThreadExecutor("Master-Scheduler-Thread"); - heartbeatMasterService = ThreadUtils.newDaemonThreadScheduledExecutor("Master-Main-Thread",Constants.DEFAULT_MASTER_HEARTBEAT_THREAD_NUM); + heartbeatMasterService = ThreadUtils.newThreadScheduledExecutor("Master-Main-Thread",Constants.DEFAULT_MASTER_HEARTBEAT_THREAD_NUM, false); // heartbeat thread implement Runnable heartBeatThread = heartBeatThread(); @@ -147,23 +152,17 @@ public class MasterServer implements IStoppable { } logger.error("start Quartz failed", e); } - - - /** - * register hooks, which are called before the process exits - */ - Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() { - @Override - public void run() { - if (zkMasterClient.getActiveMasterNum() <= 1) { - zkMasterClient.getAlertDao().sendServerStopedAlert( - 1, OSUtils.getHost(), "Master-Server"); - } - stop("shutdownhook"); - } - })); } + @PreDestroy + public void destroy() { + // master server exit alert + if (zkMasterClient.getActiveMasterNum() <= 1) { + zkMasterClient.getAlertDao().sendServerStopedAlert( + 1, OSUtils.getHost(), "Master-Server"); + } + stop("shutdownhook"); + } /** * gracefully stop diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/WorkerServer.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/WorkerServer.java index ace93079ff..86bb7d3e07 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/WorkerServer.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/WorkerServer.java @@ -29,6 +29,7 @@ import org.apache.dolphinscheduler.common.utils.CollectionUtils; import org.apache.dolphinscheduler.common.utils.OSUtils; import org.apache.dolphinscheduler.dao.AlertDao; import org.apache.dolphinscheduler.dao.entity.TaskInstance; +import org.apache.dolphinscheduler.server.master.MasterServer; import org.apache.dolphinscheduler.server.utils.ProcessUtils; import org.apache.dolphinscheduler.server.worker.config.WorkerConfig; import org.apache.dolphinscheduler.server.worker.runner.FetchTaskThread; @@ -43,10 +44,13 @@ import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; import org.springframework.beans.factory.annotation.Value; import org.springframework.boot.WebApplicationType; +import org.springframework.boot.autoconfigure.SpringBootApplication; import org.springframework.boot.builder.SpringApplicationBuilder; import org.springframework.context.annotation.ComponentScan; +import org.springframework.context.annotation.FilterType; import javax.annotation.PostConstruct; +import javax.annotation.PreDestroy; import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; @@ -56,7 +60,10 @@ import java.util.concurrent.TimeUnit; /** * worker server */ -@ComponentScan("org.apache.dolphinscheduler") +@SpringBootApplication +@ComponentScan(value = "org.apache.dolphinscheduler", excludeFilters = { + @ComponentScan.Filter(type = FilterType.ASSIGNABLE_TYPE, classes = {MasterServer.class}) +}) public class WorkerServer implements IStoppable { /** @@ -104,11 +111,6 @@ public class WorkerServer implements IStoppable { */ private ExecutorService fetchTaskExecutorService; - /** - * CountDownLatch latch - */ - private CountDownLatch latch; - @Value("${server.is-combined-server:false}") private Boolean isCombinedServer; @@ -149,7 +151,7 @@ public class WorkerServer implements IStoppable { this.fetchTaskExecutorService = ThreadUtils.newDaemonSingleThreadExecutor("Worker-Fetch-Thread-Executor"); - heartbeatWorkerService = ThreadUtils.newDaemonThreadScheduledExecutor("Worker-Heartbeat-Thread-Executor", Constants.DEFAUL_WORKER_HEARTBEAT_THREAD_NUM); + heartbeatWorkerService = ThreadUtils.newThreadScheduledExecutor("Worker-Heartbeat-Thread-Executor", Constants.DEFAUL_WORKER_HEARTBEAT_THREAD_NUM, false); // heartbeat thread implement Runnable heartBeatThread = heartBeatThread(); @@ -171,29 +173,15 @@ public class WorkerServer implements IStoppable { // submit fetch task thread fetchTaskExecutorService.execute(fetchTaskThread); + } - /** - * register hooks, which are called before the process exits - */ - Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() { - @Override - public void run() { - // worker server exit alert - if (zkWorkerClient.getActiveMasterNum() <= 1) { - alertDao.sendServerStopedAlert(1, OSUtils.getHost(), "Worker-Server"); - } - stop("shutdownhook"); - } - })); - - //let the main thread await - latch = new CountDownLatch(1); - if (!isCombinedServer) { - try { - latch.await(); - } catch (InterruptedException ignore) { - } + @PreDestroy + public void destroy() { + // worker server exit alert + if (zkWorkerClient.getActiveMasterNum() <= 1) { + alertDao.sendServerStopedAlert(1, OSUtils.getHost(), "Worker-Server"); } + stop("shutdownhook"); } @Override @@ -251,7 +239,6 @@ public class WorkerServer implements IStoppable { }catch (Exception e){ logger.warn("zookeeper service stopped exception:{}",e.getMessage()); } - latch.countDown(); logger.info("zookeeper service stopped"); } catch (Exception e) { From a076ae4c3658b23afee1e5b516b073cac0866756 Mon Sep 17 00:00:00 2001 From: "gabry.wu" Date: Sat, 14 Mar 2020 18:12:14 +0800 Subject: [PATCH 097/221] Adapting partial code(file name start with J) to the sonar cloud rule (#2165) * Adapting partial code(file name start with J) to the sonar cloud rule * fix unit failure --- .../alert/utils/JSONUtils.java | 7 ++--- .../alert/utils/JSONUtilsTest.java | 27 ++++++++++--------- .../common/utils/JSONUtils.java | 10 ++----- .../common/utils/JSONUtilsTest.java | 18 ++++++------- 4 files changed, 29 insertions(+), 33 deletions(-) diff --git a/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/JSONUtils.java b/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/JSONUtils.java index 8d66255fc4..5f8371de2d 100644 --- a/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/JSONUtils.java +++ b/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/JSONUtils.java @@ -22,6 +22,7 @@ import org.apache.dolphinscheduler.common.utils.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Collections; import java.util.List; /** @@ -50,11 +51,11 @@ public class JSONUtils { * @param json the json * @param clazz c * @param the generic clazz - * @return the result list + * @return the result list or empty list */ public static List toList(String json, Class clazz) { if (StringUtils.isEmpty(json)) { - return null; + return Collections.emptyList(); } try { return JSON.parseArray(json, clazz); @@ -62,7 +63,7 @@ public class JSONUtils { logger.error("JSONArray.parseArray exception!",e); } - return null; + return Collections.emptyList(); } } diff --git a/dolphinscheduler-alert/src/test/java/org/apache/dolphinscheduler/alert/utils/JSONUtilsTest.java b/dolphinscheduler-alert/src/test/java/org/apache/dolphinscheduler/alert/utils/JSONUtilsTest.java index 277c42b5bd..a151abc714 100644 --- a/dolphinscheduler-alert/src/test/java/org/apache/dolphinscheduler/alert/utils/JSONUtilsTest.java +++ b/dolphinscheduler-alert/src/test/java/org/apache/dolphinscheduler/alert/utils/JSONUtilsTest.java @@ -26,8 +26,7 @@ import java.util.ArrayList; import java.util.LinkedHashMap; import java.util.List; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; +import static org.junit.Assert.*; public class JSONUtilsTest { @@ -73,7 +72,7 @@ public class JSONUtilsTest { result = JSONUtils.toJsonString(null); logger.info(result); - assertEquals(result,"null"); + assertEquals("null", result); } @@ -86,25 +85,27 @@ public class JSONUtilsTest { //Invoke toList List result = JSONUtils.toList(expected ,LinkedHashMap.class); //Equal list size=1 - assertEquals(result.size(),1); + assertEquals(1,result.size()); //Transform entity to LinkedHashMap LinkedHashMap entity = result.get(0); //Equal expected values - assertEquals(entity.get("mysql service name"),"mysql200"); - assertEquals(entity.get("mysql address"),"192.168.xx.xx"); - assertEquals(entity.get("port"),"3306"); - assertEquals(entity.get("no index of number"),"80"); - assertEquals(entity.get("database client connections"),"190"); + assertEquals("mysql200",entity.get("mysql service name")); + assertEquals("192.168.xx.xx", entity.get("mysql address")); + assertEquals("3306", entity.get("port")); + assertEquals("80", entity.get("no index of number")); + assertEquals("190", entity.get("database client connections")); - //If param is null, then return null + //If param is null, then return empty list result = JSONUtils.toList(null ,LinkedHashMap.class); - assertNull(result); + assertNotNull(result); + assertTrue(result.isEmpty()); - //If param is incorrect, then return null and log error message + //If param is incorrect, then return empty list and log error message result = JSONUtils.toList("}{" ,LinkedHashMap.class); - assertNull(result); + assertNotNull(result); + assertTrue(result.isEmpty()); } diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/JSONUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/JSONUtils.java index b6ed71f402..f0aed91a0d 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/JSONUtils.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/JSONUtils.java @@ -42,12 +42,6 @@ public class JSONUtils { */ private static final ObjectMapper objectMapper = new ObjectMapper(); - /** - * init - */ - private static final JSONUtils instance = new JSONUtils(); - - private JSONUtils() { //Feature that determines whether encountering of unknown properties, false means not analyzer unknown properties objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false).setTimeZone(TimeZone.getDefault()); @@ -221,7 +215,7 @@ public class JSONUtils { try{ return JSON.toJSONString(object,false); } catch (Exception e) { - throw new RuntimeException("Json deserialization exception.", e); + throw new RuntimeException("Object json deserialization exception.", e); } } @@ -229,7 +223,7 @@ public class JSONUtils { try{ return JSON.parseObject(text); } catch (Exception e) { - throw new RuntimeException("Json deserialization exception.", e); + throw new RuntimeException("String json deserialization exception.", e); } } diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/JSONUtilsTest.java b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/JSONUtilsTest.java index d249314819..8ce60349ed 100644 --- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/JSONUtilsTest.java +++ b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/JSONUtilsTest.java @@ -40,8 +40,8 @@ public class JSONUtilsTest { String jsonStr = "{\"id\":\"1001\",\"name\":\"Jobs\"}"; Map models = JSONUtils.toMap(jsonStr); - Assert.assertEquals(models.get("id"), "1001"); - Assert.assertEquals(models.get("name"), "Jobs"); + Assert.assertEquals("1001", models.get("id")); + Assert.assertEquals("Jobs", models.get("name")); } @@ -55,7 +55,7 @@ public class JSONUtilsTest { String str = "{\"direct\":\"IN\",\"prop\":\"ds\",\"type\":\"VARCHAR\",\"value\":\"sssssss\"}"; Property property1 = JSON.parseObject(str, Property.class); Direct direct = property1.getDirect(); - Assert.assertEquals(direct , Direct.IN); + Assert.assertEquals(Direct.IN, direct); } @@ -66,12 +66,12 @@ public class JSONUtilsTest { List maps = JSONUtils.toList(str, LinkedHashMap.class); - Assert.assertEquals(maps.size(), 1); - Assert.assertEquals(maps.get(0).get("mysql service name"), "mysql200"); - Assert.assertEquals(maps.get(0).get("mysql address"), "192.168.xx.xx"); - Assert.assertEquals(maps.get(0).get("port"), "3306"); - Assert.assertEquals(maps.get(0).get("no index of number"), "80"); - Assert.assertEquals(maps.get(0).get("database client connections"), "190"); + Assert.assertEquals(1, maps.size()); + Assert.assertEquals("mysql200", maps.get(0).get("mysql service name")); + Assert.assertEquals("192.168.xx.xx", maps.get(0).get("mysql address")); + Assert.assertEquals("3306", maps.get(0).get("port")); + Assert.assertEquals("80", maps.get(0).get("no index of number")); + Assert.assertEquals("190", maps.get(0).get("database client connections")); } public String list2String(){ From e0389aebf6416c90f31ec0f6a4ff5a5d94cfecfb Mon Sep 17 00:00:00 2001 From: Rubik-W <39549317+Rubik-W@users.noreply.github.com> Date: Sat, 14 Mar 2020 19:20:46 +0800 Subject: [PATCH 098/221] fix: NullPointException in the case of task queuing (#2175) --- .../dolphinscheduler/server/master/runner/MasterExecThread.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java index 849bbd5680..0c44b7a77e 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java @@ -1062,7 +1062,7 @@ public class MasterExecThread implements Runnable { TaskInstance taskInstance = taskExecThread.getTaskInstance(); taskInstance = processService.findTaskInstanceById(taskInstance.getId()); - if(taskInstance.getState().typeIsFinished()){ + if(taskInstance != null && taskInstance.getState().typeIsFinished()){ continue; } From 61ebbf29463809449734ff00b00e6d9627170d10 Mon Sep 17 00:00:00 2001 From: "gabry.wu" Date: Sat, 14 Mar 2020 19:22:24 +0800 Subject: [PATCH 099/221] Adapting partial code(file name start with L) to the sonar cloud rule (#2171) --- .../interceptor/LoginHandlerInterceptor.java | 11 ---------- .../common/utils/LoggerUtils.java | 2 +- .../server/log/LoggerRequestProcessor.java | 21 +++++-------------- .../service/log/LogClientService.java | 8 +++---- 4 files changed, 10 insertions(+), 32 deletions(-) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/interceptor/LoginHandlerInterceptor.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/interceptor/LoginHandlerInterceptor.java index 380eea5774..98bac42f72 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/interceptor/LoginHandlerInterceptor.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/interceptor/LoginHandlerInterceptor.java @@ -27,7 +27,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.beans.factory.annotation.Autowired; import org.springframework.web.servlet.HandlerInterceptor; -import org.springframework.web.servlet.ModelAndView; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; @@ -90,14 +89,4 @@ public class LoginHandlerInterceptor implements HandlerInterceptor { return true; } - @Override - public void postHandle(HttpServletRequest request, HttpServletResponse response, Object handler, ModelAndView modelAndView) throws Exception { - - } - - @Override - public void afterCompletion(HttpServletRequest request, HttpServletResponse response, Object handler, Exception ex) throws Exception { - - } - } diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/LoggerUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/LoggerUtils.java index fc08eb645b..191df335c5 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/LoggerUtils.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/LoggerUtils.java @@ -79,7 +79,7 @@ public class LoggerUtils { */ public static List getAppIds(String log, Logger logger) { - List appIds = new ArrayList(); + List appIds = new ArrayList<>(); Matcher matcher = APPLICATION_REGEX.matcher(log); diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/log/LoggerRequestProcessor.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/log/LoggerRequestProcessor.java index 4e4404ea1c..4b21d5f4a2 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/log/LoggerRequestProcessor.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/log/LoggerRequestProcessor.java @@ -17,6 +17,7 @@ package org.apache.dolphinscheduler.server.log; import io.netty.channel.Channel; +import org.apache.dolphinscheduler.common.utils.IOUtils; import org.apache.dolphinscheduler.remote.command.Command; import org.apache.dolphinscheduler.remote.command.CommandType; import org.apache.dolphinscheduler.remote.command.log.*; @@ -116,16 +117,8 @@ public class LoggerRequestProcessor implements NettyRequestProcessor { }catch (IOException e){ logger.error("get file bytes error",e); }finally { - if (bos != null){ - try { - bos.close(); - } catch (IOException ignore) {} - } - if (in != null){ - try { - in.close(); - } catch (IOException ignore) {} - } + IOUtils.closeQuietly(bos); + IOUtils.closeQuietly(in); } return new byte[0]; } @@ -146,7 +139,7 @@ public class LoggerRequestProcessor implements NettyRequestProcessor { } catch (IOException e) { logger.error("read file error",e); } - return Collections.EMPTY_LIST; + return Collections.emptyList(); } /** @@ -168,11 +161,7 @@ public class LoggerRequestProcessor implements NettyRequestProcessor { }catch (IOException e){ logger.error("read file error",e); }finally { - try { - if (br != null){ - br.close(); - } - } catch (IOException ignore) {} + IOUtils.closeQuietly(br); } return ""; } diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java index 01a49910df..163cd3696f 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/log/LogClientService.java @@ -40,7 +40,7 @@ public class LogClientService { /** * request time out */ - private final long logRequestTimeout = 10 * 1000; + private static final long LOG_REQUEST_TIMEOUT = 10 * 1000L; /** * construct client @@ -75,7 +75,7 @@ public class LogClientService { final Address address = new Address(host, port); try { Command command = request.convert2Command(); - Command response = this.client.sendSync(address, command, logRequestTimeout); + Command response = this.client.sendSync(address, command, LOG_REQUEST_TIMEOUT); if(response != null){ RollViewLogResponseCommand rollReviewLog = FastJsonSerializer.deserialize( response.getBody(), RollViewLogResponseCommand.class); @@ -103,7 +103,7 @@ public class LogClientService { final Address address = new Address(host, port); try { Command command = request.convert2Command(); - Command response = this.client.sendSync(address, command, logRequestTimeout); + Command response = this.client.sendSync(address, command, LOG_REQUEST_TIMEOUT); if(response != null){ ViewLogResponseCommand viewLog = FastJsonSerializer.deserialize( response.getBody(), ViewLogResponseCommand.class); @@ -131,7 +131,7 @@ public class LogClientService { final Address address = new Address(host, port); try { Command command = request.convert2Command(); - Command response = this.client.sendSync(address, command, logRequestTimeout); + Command response = this.client.sendSync(address, command, LOG_REQUEST_TIMEOUT); if(response != null){ GetLogBytesResponseCommand getLog = FastJsonSerializer.deserialize( response.getBody(), GetLogBytesResponseCommand.class); From fcc57320d6b281e81fd9068f4dbb3d86ec6978f1 Mon Sep 17 00:00:00 2001 From: "gabry.wu" Date: Sun, 15 Mar 2020 19:05:33 +0800 Subject: [PATCH 100/221] Adapting partial code(file name start with M #1) to the sonar cloud rule (#2173) * Adapting partial code(file name start with M #1) to the sonar cloud rule * remove unused import --- .../alert/utils/MailUtils.java | 52 ++++++++-------- .../common/task/mr/MapreduceParameters.java | 14 +++-- .../dao/entity/ProcessInstance.java | 2 +- .../dao/entity/TaskInstance.java | 4 +- .../master/runner/MasterExecThread.java | 61 +++++++++---------- .../server/worker/task/mr/MapReduceTask.java | 17 +++--- 6 files changed, 72 insertions(+), 78 deletions(-) diff --git a/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/MailUtils.java b/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/MailUtils.java index 130ad8ade6..66a63dcff4 100644 --- a/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/MailUtils.java +++ b/dolphinscheduler-alert/src/main/java/org/apache/dolphinscheduler/alert/utils/MailUtils.java @@ -39,29 +39,29 @@ public class MailUtils { public static final Logger logger = LoggerFactory.getLogger(MailUtils.class); - public static final String mailProtocol = PropertyUtils.getString(Constants.MAIL_PROTOCOL); + public static final String MAIL_PROTOCOL = PropertyUtils.getString(Constants.MAIL_PROTOCOL); - public static final String mailServerHost = PropertyUtils.getString(Constants.MAIL_SERVER_HOST); + public static final String MAIL_SERVER_HOST = PropertyUtils.getString(Constants.MAIL_SERVER_HOST); - public static final Integer mailServerPort = PropertyUtils.getInt(Constants.MAIL_SERVER_PORT); + public static final Integer MAIL_SERVER_PORT = PropertyUtils.getInt(Constants.MAIL_SERVER_PORT); - public static final String mailSender = PropertyUtils.getString(Constants.MAIL_SENDER); + public static final String MAIL_SENDER = PropertyUtils.getString(Constants.MAIL_SENDER); - public static final String mailUser = PropertyUtils.getString(Constants.MAIL_USER); + public static final String MAIL_USER = PropertyUtils.getString(Constants.MAIL_USER); - public static final String mailPasswd = PropertyUtils.getString(Constants.MAIL_PASSWD); + public static final String MAIL_PASSWD = PropertyUtils.getString(Constants.MAIL_PASSWD); - public static final Boolean mailUseStartTLS = PropertyUtils.getBoolean(Constants.MAIL_SMTP_STARTTLS_ENABLE); + public static final Boolean MAIL_USE_START_TLS = PropertyUtils.getBoolean(Constants.MAIL_SMTP_STARTTLS_ENABLE); - public static final Boolean mailUseSSL = PropertyUtils.getBoolean(Constants.MAIL_SMTP_SSL_ENABLE); + public static final Boolean MAIL_USE_SSL = PropertyUtils.getBoolean(Constants.MAIL_SMTP_SSL_ENABLE); - public static final String xlsFilePath = PropertyUtils.getString(Constants.XLS_FILE_PATH); + public static final String XLS_FILE_PATH = PropertyUtils.getString(Constants.XLS_FILE_PATH); - public static final String starttlsEnable = PropertyUtils.getString(Constants.MAIL_SMTP_STARTTLS_ENABLE); + public static final String STARTTLS_ENABLE = PropertyUtils.getString(Constants.MAIL_SMTP_STARTTLS_ENABLE); - public static final String sslEnable = PropertyUtils.getString(Constants.MAIL_SMTP_SSL_ENABLE); + public static final String SSL_ENABLE = PropertyUtils.getString(Constants.MAIL_SMTP_SSL_ENABLE); - public static final String sslTrust = PropertyUtils.getString(Constants.MAIL_SMTP_SSL_TRUST); + public static final String SSL_TRUST = PropertyUtils.getString(Constants.MAIL_SMTP_SSL_TRUST); public static final AlertTemplate alertTemplate = AlertTemplateFactory.getMessageTemplate(); @@ -105,7 +105,7 @@ public class MailUtils { try { Session session = getSession(); email.setMailSession(session); - email.setFrom(mailSender); + email.setFrom(MAIL_SENDER); email.setCharset(Constants.UTF_8); if (CollectionUtils.isNotEmpty(receivers)){ // receivers mail @@ -199,10 +199,10 @@ public class MailUtils { // 2. creating mail: Creating a MimeMessage MimeMessage msg = new MimeMessage(session); // 3. set sender - msg.setFrom(new InternetAddress(mailSender)); + msg.setFrom(new InternetAddress(MAIL_SENDER)); // 4. set receivers for (String receiver : receivers) { - msg.addRecipients(MimeMessage.RecipientType.TO, InternetAddress.parse(receiver)); + msg.addRecipients(Message.RecipientType.TO, InternetAddress.parse(receiver)); } return msg; } @@ -213,19 +213,19 @@ public class MailUtils { */ private static Session getSession() { Properties props = new Properties(); - props.setProperty(Constants.MAIL_HOST, mailServerHost); - props.setProperty(Constants.MAIL_PORT, String.valueOf(mailServerPort)); + props.setProperty(Constants.MAIL_HOST, MAIL_SERVER_HOST); + props.setProperty(Constants.MAIL_PORT, String.valueOf(MAIL_SERVER_PORT)); props.setProperty(Constants.MAIL_SMTP_AUTH, Constants.STRING_TRUE); - props.setProperty(Constants.MAIL_TRANSPORT_PROTOCOL, mailProtocol); - props.setProperty(Constants.MAIL_SMTP_STARTTLS_ENABLE, starttlsEnable); - props.setProperty(Constants.MAIL_SMTP_SSL_ENABLE, sslEnable); - props.setProperty(Constants.MAIL_SMTP_SSL_TRUST, sslTrust); + props.setProperty(Constants.MAIL_TRANSPORT_PROTOCOL, MAIL_PROTOCOL); + props.setProperty(Constants.MAIL_SMTP_STARTTLS_ENABLE, STARTTLS_ENABLE); + props.setProperty(Constants.MAIL_SMTP_SSL_ENABLE, SSL_ENABLE); + props.setProperty(Constants.MAIL_SMTP_SSL_TRUST, SSL_TRUST); Authenticator auth = new Authenticator() { @Override protected PasswordAuthentication getPasswordAuthentication() { // mail username and password - return new PasswordAuthentication(mailUser, mailPasswd); + return new PasswordAuthentication(MAIL_USER, MAIL_PASSWD); } }; @@ -248,12 +248,10 @@ public class MailUtils { */ if(CollectionUtils.isNotEmpty(receiversCc)){ for (String receiverCc : receiversCc){ - msg.addRecipients(MimeMessage.RecipientType.CC, InternetAddress.parse(receiverCc)); + msg.addRecipients(Message.RecipientType.CC, InternetAddress.parse(receiverCc)); } } - // set receivers type to cc - // msg.addRecipients(MimeMessage.RecipientType.CC, InternetAddress.parse(propMap.get("${CC}"))); // set subject msg.setSubject(title); MimeMultipart partList = new MimeMultipart(); @@ -263,8 +261,8 @@ public class MailUtils { // set attach file MimeBodyPart part2 = new MimeBodyPart(); // make excel file - ExcelUtils.genExcelFile(content,title,xlsFilePath); - File file = new File(xlsFilePath + Constants.SINGLE_SLASH + title + Constants.EXCEL_SUFFIX_XLS); + ExcelUtils.genExcelFile(content,title, XLS_FILE_PATH); + File file = new File(XLS_FILE_PATH + Constants.SINGLE_SLASH + title + Constants.EXCEL_SUFFIX_XLS); part2.attachFile(file); part2.setFileName(MimeUtility.encodeText(title + Constants.EXCEL_SUFFIX_XLS,Constants.UTF_8,"B")); // add components to collection diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/mr/MapreduceParameters.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/mr/MapreduceParameters.java index b8fd6ebcbf..31c9c7292f 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/mr/MapreduceParameters.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/task/mr/MapreduceParameters.java @@ -20,6 +20,7 @@ import org.apache.dolphinscheduler.common.enums.ProgramType; import org.apache.dolphinscheduler.common.process.ResourceInfo; import org.apache.dolphinscheduler.common.task.AbstractParameters; +import java.util.Collections; import java.util.List; import java.util.stream.Collectors; @@ -125,12 +126,15 @@ public class MapreduceParameters extends AbstractParameters { @Override public List getResourceFilesList() { - if (resourceList != null) { - this.resourceList.add(mainJar); - return resourceList.stream() - .map(p -> p.getRes()).collect(Collectors.toList()); + if(resourceList != null ) { + List resourceFiles = resourceList.stream() + .map(ResourceInfo::getRes).collect(Collectors.toList()); + if(mainJar != null) { + resourceFiles.add(mainJar.getRes()); + } + return resourceFiles; } - return null; + return Collections.emptyList(); } @Override diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessInstance.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessInstance.java index fb5d0cda42..fd9653a36c 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessInstance.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/ProcessInstance.java @@ -506,7 +506,7 @@ public class ProcessInstance { * check this process is start complement data * @return whether complement data */ - public Boolean isComplementData(){ + public boolean isComplementData(){ if(!StringUtils.isNotEmpty(this.historyCmd)){ return false; } diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java index c692575e3a..702897fba9 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java @@ -373,7 +373,7 @@ public class TaskInstance { } - public Boolean isSubProcess(){ + public boolean isSubProcess(){ return TaskType.SUB_PROCESS.getDescp().equals(this.taskType); } @@ -442,7 +442,7 @@ public class TaskInstance { this.executorName = executorName; } - public Boolean isTaskComplete() { + public boolean isTaskComplete() { return this.getState().typeIsPause() || this.getState().typeIsSuccess() diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java index 0c44b7a77e..26457a386a 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterExecThread.java @@ -68,7 +68,7 @@ public class MasterExecThread implements Runnable { /** * runing TaskNode */ - private final Map> activeTaskNode = new ConcurrentHashMap>(); + private final Map> activeTaskNode = new ConcurrentHashMap<>(); /** * task exec service @@ -78,7 +78,7 @@ public class MasterExecThread implements Runnable { /** * submit failure nodes */ - private Boolean taskFailedSubmit = false; + private boolean taskFailedSubmit = false; /** * recover node id list @@ -652,7 +652,7 @@ public class MasterExecThread implements Runnable { continue; } if(task.getState().typeIsPause() || task.getState().typeIsCancel()){ - logger.info("task {} stopped, the state is {}", task.getName(), task.getState().toString()); + logger.info("task {} stopped, the state is {}", task.getName(), task.getState()); }else{ addTaskToStandByList(task); } @@ -685,11 +685,12 @@ public class MasterExecThread implements Runnable { } ExecutionStatus depTaskState = completeTaskList.get(depsNode).getState(); // conditions task would not return failed. - if(depTaskState.typeIsFailure()){ - if(!haveConditionsAfterNode(depsNode) && !dag.getNode(depsNode).isConditionsTask()){ - return DependResult.FAILED; - } + if(depTaskState.typeIsFailure() + && !haveConditionsAfterNode(depsNode) + && !dag.getNode(depsNode).isConditionsTask()){ + return DependResult.FAILED; } + if(depTaskState.typeIsPause() || depTaskState.typeIsCancel()){ return DependResult.WAITING; } @@ -737,7 +738,7 @@ public class MasterExecThread implements Runnable { * * @return Boolean whether has failed task */ - private Boolean hasFailedTask(){ + private boolean hasFailedTask(){ if(this.taskFailedSubmit){ return true; @@ -753,7 +754,7 @@ public class MasterExecThread implements Runnable { * * @return Boolean whether process instance failed */ - private Boolean processFailed(){ + private boolean processFailed(){ if(hasFailedTask()) { if(processInstance.getFailureStrategy() == FailureStrategy.END){ return true; @@ -769,9 +770,9 @@ public class MasterExecThread implements Runnable { * whether task for waiting thread * @return Boolean whether has waiting thread task */ - private Boolean hasWaitingThreadTask(){ + private boolean hasWaitingThreadTask(){ List waitingList = getCompleteTaskByState(ExecutionStatus.WAITTING_THREAD); - return waitingList.size() > 0; + return CollectionUtils.isNotEmpty(waitingList); } /** @@ -787,7 +788,7 @@ public class MasterExecThread implements Runnable { } List pauseList = getCompleteTaskByState(ExecutionStatus.PAUSE); - if(pauseList.size() > 0 + if(CollectionUtils.isNotEmpty(pauseList) || !isComplementEnd() || readyToSubmitTaskList.size() > 0){ return ExecutionStatus.PAUSE; @@ -827,7 +828,8 @@ public class MasterExecThread implements Runnable { if(state == ExecutionStatus.READY_STOP){ List stopList = getCompleteTaskByState(ExecutionStatus.STOP); List killList = getCompleteTaskByState(ExecutionStatus.KILL); - if(stopList.size() > 0 || killList.size() > 0 || !isComplementEnd()){ + if(CollectionUtils.isNotEmpty(stopList) + || CollectionUtils.isNotEmpty(killList) || !isComplementEnd()){ return ExecutionStatus.STOP; }else{ return ExecutionStatus.SUCCESS; @@ -852,7 +854,7 @@ public class MasterExecThread implements Runnable { * whether complement end * @return Boolean whether is complement end */ - private Boolean isComplementEnd() { + private boolean isComplementEnd() { if(!processInstance.isComplementData()){ return true; } @@ -877,8 +879,8 @@ public class MasterExecThread implements Runnable { logger.info( "work flow process instance [id: {}, name:{}], state change from {} to {}, cmd type: {}", processInstance.getId(), processInstance.getName(), - processInstance.getState().toString(), state.toString(), - processInstance.getCommandType().toString()); + processInstance.getState(), state, + processInstance.getCommandType()); processInstance.setState(state); ProcessInstance instance = processService.findProcessInstanceById(processInstance.getId()); instance.setState(state); @@ -894,8 +896,7 @@ public class MasterExecThread implements Runnable { * @return DependResult */ private DependResult getDependResultForTask(TaskInstance taskInstance){ - DependResult inner = isTaskDepsComplete(taskInstance.getName()); - return inner; + return isTaskDepsComplete(taskInstance.getName()); } /** @@ -920,7 +921,7 @@ public class MasterExecThread implements Runnable { * has retry task in standby * @return Boolean whether has retry task in standby */ - private Boolean hasRetryTaskInStandBy(){ + private boolean hasRetryTaskInStandBy(){ for (Map.Entry entry: readyToSubmitTaskList.entrySet()) { if(entry.getValue().getState().typeIsFailure()){ return true; @@ -958,7 +959,7 @@ public class MasterExecThread implements Runnable { continue; } logger.info("task :{}, id:{} complete, state is {} ", - task.getName(), task.getId(), task.getState().toString()); + task.getName(), task.getId(), task.getState()); // node success , post node submit if(task.getState() == ExecutionStatus.SUCCESS){ completeTaskList.put(task.getName(), task); @@ -990,7 +991,7 @@ public class MasterExecThread implements Runnable { completeTaskList.put(task.getName(), task); } // send alert - if(this.recoverToleranceFaultTaskList.size() > 0){ + if(CollectionUtils.isNotEmpty(this.recoverToleranceFaultTaskList)){ alertManager.sendAlertWorkerToleranceFault(processInstance, recoverToleranceFaultTaskList); this.recoverToleranceFaultTaskList.clear(); } @@ -1034,10 +1035,7 @@ public class MasterExecThread implements Runnable { Date now = new Date(); long runningTime = DateUtils.diffMin(now, processInstance.getStartTime()); - if(runningTime > processInstance.getTimeout()){ - return true; - } - return false; + return runningTime > processInstance.getTimeout(); } /** @@ -1081,22 +1079,19 @@ public class MasterExecThread implements Runnable { * @param taskInstance task instance * @return Boolean */ - private Boolean retryTaskIntervalOverTime(TaskInstance taskInstance){ + private boolean retryTaskIntervalOverTime(TaskInstance taskInstance){ if(taskInstance.getState() != ExecutionStatus.FAILURE){ - return Boolean.TRUE; + return true; } if(taskInstance.getId() == 0 || taskInstance.getMaxRetryTimes() ==0 || taskInstance.getRetryInterval() == 0 ){ - return Boolean.TRUE; + return true; } Date now = new Date(); long failedTimeInterval = DateUtils.differSec(now, taskInstance.getEndTime()); // task retry does not over time, return false - if(taskInstance.getRetryInterval() * SEC_2_MINUTES_TIME_UNIT >= failedTimeInterval){ - return Boolean.FALSE; - } - return Boolean.TRUE; + return taskInstance.getRetryInterval() * SEC_2_MINUTES_TIME_UNIT < failedTimeInterval; } /** @@ -1189,7 +1184,7 @@ public class MasterExecThread implements Runnable { */ private List getRecoveryNodeNameList(){ List recoveryNodeNameList = new ArrayList<>(); - if(recoverNodeIdList.size() > 0) { + if(CollectionUtils.isNotEmpty(recoverNodeIdList)) { for (TaskInstance task : recoverNodeIdList) { recoveryNodeNameList.add(task.getName()); } diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/mr/MapReduceTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/mr/MapReduceTask.java index b86ff9952e..7f6baad427 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/mr/MapReduceTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/mr/MapReduceTask.java @@ -122,22 +122,19 @@ public class MapReduceTask extends AbstractYarnTask { } // main class - if(mapreduceParameters.getProgramType() !=null ){ - if(mapreduceParameters.getProgramType()!= ProgramType.PYTHON){ - if(StringUtils.isNotEmpty(mapreduceParameters.getMainClass())){ - result.add(mapreduceParameters.getMainClass()); - } - } + if(!ProgramType.PYTHON.equals(mapreduceParameters.getProgramType()) + && StringUtils.isNotEmpty(mapreduceParameters.getMainClass())){ + result.add(mapreduceParameters.getMainClass()); } // others if (StringUtils.isNotEmpty(mapreduceParameters.getOthers())) { String others = mapreduceParameters.getOthers(); - if(!others.contains(Constants.MR_QUEUE)){ - if (StringUtils.isNotEmpty(mapreduceParameters.getQueue())) { - result.add(String.format("%s %s=%s", Constants.D, Constants.MR_QUEUE, mapreduceParameters.getQueue())); - } + if (!others.contains(Constants.MR_QUEUE) + && StringUtils.isNotEmpty(mapreduceParameters.getQueue())) { + result.add(String.format("%s %s=%s", Constants.D, Constants.MR_QUEUE, mapreduceParameters.getQueue())); } + result.add(mapreduceParameters.getOthers()); }else if (StringUtils.isNotEmpty(mapreduceParameters.getQueue())) { result.add(String.format("%s %s=%s", Constants.D, Constants.MR_QUEUE, mapreduceParameters.getQueue())); From e0fc3171da3b731433a4e8da3403070323007cce Mon Sep 17 00:00:00 2001 From: tswstarplanet Date: Sun, 15 Mar 2020 21:14:43 +0800 Subject: [PATCH 101/221] Cache HadoopUtils instance with specific days expire time (#2181) * Cache HadoopUtils instance with 7 days expire time * solve sonar issue * add kerberos expire time config * move KERBEROS_EXPIRE_TIME to Constants.java --- .../dolphinscheduler/common/Constants.java | 5 + .../common/utils/HadoopUtils.java | 234 +++++++++--------- .../src/main/resources/common.properties | 2 +- 3 files changed, 124 insertions(+), 117 deletions(-) diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java index 6af0e6445f..b0a7b74d39 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java @@ -813,6 +813,11 @@ public final class Constants { */ public static final String KERBEROS = "kerberos"; + /** + * kerberos expire time + */ + public static final String KERBEROS_EXPIRE_TIME = "kerberos.expire.time"; + /** * java.security.krb5.conf */ diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/HadoopUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/HadoopUtils.java index 6cb58a4324..e767911580 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/HadoopUtils.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/HadoopUtils.java @@ -16,6 +16,9 @@ */ package org.apache.dolphinscheduler.common.utils; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.enums.ResUploadType; @@ -37,6 +40,7 @@ import java.security.PrivilegedExceptionAction; import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -48,30 +52,37 @@ public class HadoopUtils implements Closeable { private static final Logger logger = LoggerFactory.getLogger(HadoopUtils.class); - private static HadoopUtils instance = new HadoopUtils(); + private static final String HADOOP_UTILS_KEY = "HADOOP_UTILS_KEY"; + + private static final LoadingCache cache = CacheBuilder + .newBuilder() + .expireAfterWrite(PropertyUtils.getInt(Constants.KERBEROS_EXPIRE_TIME, 7), TimeUnit.DAYS) + .build(new CacheLoader() { + @Override + public HadoopUtils load(String key) throws Exception { + return new HadoopUtils(); + } + }); + private static Configuration configuration; private static FileSystem fs; - private String hdfsUser; + private static String hdfsUser = PropertyUtils.getString(Constants.HDFS_ROOT_USER); - private HadoopUtils(){ - hdfsUser = PropertyUtils.getString(Constants.HDFS_ROOT_USER); + private HadoopUtils() { init(); initHdfsPath(); } - public static HadoopUtils getInstance(){ - // if kerberos startup , renew HadoopUtils - if (CommonUtils.getKerberosStartupState()){ - return new HadoopUtils(); - } - return instance; + public static HadoopUtils getInstance() { + + return cache.getUnchecked(HADOOP_UTILS_KEY); } /** * init dolphinscheduler root path in hdfs */ - private void initHdfsPath(){ + private void initHdfsPath() { String hdfsPath = PropertyUtils.getString(Constants.DATA_STORE_2_HDFS_BASEPATH); Path path = new Path(hdfsPath); @@ -80,7 +91,7 @@ public class HadoopUtils implements Closeable { fs.mkdirs(path); } } catch (Exception e) { - logger.error(e.getMessage(),e); + logger.error(e.getMessage(), e); } } @@ -88,82 +99,75 @@ public class HadoopUtils implements Closeable { /** * init hadoop configuration */ - private void init() { - if (configuration == null) { - synchronized (HadoopUtils.class) { - if (configuration == null) { - try { - configuration = new Configuration(); - - String resUploadStartupType = PropertyUtils.getString(Constants.RES_UPLOAD_STARTUP_TYPE); - ResUploadType resUploadType = ResUploadType.valueOf(resUploadStartupType); - - if (resUploadType == ResUploadType.HDFS){ - if (PropertyUtils.getBoolean(Constants.HADOOP_SECURITY_AUTHENTICATION_STARTUP_STATE)){ - System.setProperty(Constants.JAVA_SECURITY_KRB5_CONF, - PropertyUtils.getString(Constants.JAVA_SECURITY_KRB5_CONF_PATH)); - configuration.set(Constants.HADOOP_SECURITY_AUTHENTICATION,"kerberos"); - UserGroupInformation.setConfiguration(configuration); - UserGroupInformation.loginUserFromKeytab(PropertyUtils.getString(Constants.LOGIN_USER_KEY_TAB_USERNAME), - PropertyUtils.getString(Constants.LOGIN_USER_KEY_TAB_PATH)); - } + private static void init() { + try { + configuration = new Configuration(); + + String resUploadStartupType = PropertyUtils.getString(Constants.RES_UPLOAD_STARTUP_TYPE); + ResUploadType resUploadType = ResUploadType.valueOf(resUploadStartupType); + + if (resUploadType == ResUploadType.HDFS) { + if (PropertyUtils.getBoolean(Constants.HADOOP_SECURITY_AUTHENTICATION_STARTUP_STATE)) { + System.setProperty(Constants.JAVA_SECURITY_KRB5_CONF, + PropertyUtils.getString(Constants.JAVA_SECURITY_KRB5_CONF_PATH)); + configuration.set(Constants.HADOOP_SECURITY_AUTHENTICATION, "kerberos"); + UserGroupInformation.setConfiguration(configuration); + UserGroupInformation.loginUserFromKeytab(PropertyUtils.getString(Constants.LOGIN_USER_KEY_TAB_USERNAME), + PropertyUtils.getString(Constants.LOGIN_USER_KEY_TAB_PATH)); + } - String defaultFS = configuration.get(Constants.FS_DEFAULTFS); - //first get key from core-site.xml hdfs-site.xml ,if null ,then try to get from properties file - // the default is the local file system - if(defaultFS.startsWith("file")){ - String defaultFSProp = PropertyUtils.getString(Constants.FS_DEFAULTFS); - if(StringUtils.isNotBlank(defaultFSProp)){ - Map fsRelatedProps = PropertyUtils.getPrefixedProperties("fs."); - configuration.set(Constants.FS_DEFAULTFS,defaultFSProp); - fsRelatedProps.forEach((key, value) -> configuration.set(key, value)); - }else{ - logger.error("property:{} can not to be empty, please set!", Constants.FS_DEFAULTFS ); - throw new RuntimeException( - String.format("property: %s can not to be empty, please set!", Constants.FS_DEFAULTFS) - ); - } - }else{ - logger.info("get property:{} -> {}, from core-site.xml hdfs-site.xml ", Constants.FS_DEFAULTFS, defaultFS); - } + String defaultFS = configuration.get(Constants.FS_DEFAULTFS); + //first get key from core-site.xml hdfs-site.xml ,if null ,then try to get from properties file + // the default is the local file system + if (defaultFS.startsWith("file")) { + String defaultFSProp = PropertyUtils.getString(Constants.FS_DEFAULTFS); + if (StringUtils.isNotBlank(defaultFSProp)) { + Map fsRelatedProps = PropertyUtils.getPrefixedProperties("fs."); + configuration.set(Constants.FS_DEFAULTFS, defaultFSProp); + fsRelatedProps.forEach((key, value) -> configuration.set(key, value)); + } else { + logger.error("property:{} can not to be empty, please set!", Constants.FS_DEFAULTFS); + throw new RuntimeException( + String.format("property: %s can not to be empty, please set!", Constants.FS_DEFAULTFS) + ); + } + } else { + logger.info("get property:{} -> {}, from core-site.xml hdfs-site.xml ", Constants.FS_DEFAULTFS, defaultFS); + } - if (fs == null) { - if(StringUtils.isNotEmpty(hdfsUser)){ - UserGroupInformation ugi = UserGroupInformation.createRemoteUser(hdfsUser); - ugi.doAs(new PrivilegedExceptionAction() { - @Override - public Boolean run() throws Exception { - fs = FileSystem.get(configuration); - return true; - } - }); - }else{ - logger.warn("hdfs.root.user is not set value!"); - fs = FileSystem.get(configuration); - } + if (fs == null) { + if (StringUtils.isNotEmpty(hdfsUser)) { + UserGroupInformation ugi = UserGroupInformation.createRemoteUser(hdfsUser); + ugi.doAs(new PrivilegedExceptionAction() { + @Override + public Boolean run() throws Exception { + fs = FileSystem.get(configuration); + return true; } - }else if (resUploadType == ResUploadType.S3){ - configuration.set(Constants.FS_DEFAULTFS, PropertyUtils.getString(Constants.FS_DEFAULTFS)); - configuration.set(Constants.FS_S3A_ENDPOINT, PropertyUtils.getString(Constants.FS_S3A_ENDPOINT)); - configuration.set(Constants.FS_S3A_ACCESS_KEY, PropertyUtils.getString(Constants.FS_S3A_ACCESS_KEY)); - configuration.set(Constants.FS_S3A_SECRET_KEY, PropertyUtils.getString(Constants.FS_S3A_SECRET_KEY)); - fs = FileSystem.get(configuration); - } - - - String rmHaIds = PropertyUtils.getString(Constants.YARN_RESOURCEMANAGER_HA_RM_IDS); - String appAddress = PropertyUtils.getString(Constants.YARN_APPLICATION_STATUS_ADDRESS); - if (!StringUtils.isEmpty(rmHaIds)) { - appAddress = getAppAddress(appAddress, rmHaIds); - logger.info("appAddress : {}", appAddress); - } - configuration.set(Constants.YARN_APPLICATION_STATUS_ADDRESS, appAddress); - } catch (Exception e) { - logger.error(e.getMessage(), e); + }); + } else { + logger.warn("hdfs.root.user is not set value!"); + fs = FileSystem.get(configuration); } - } + } else if (resUploadType == ResUploadType.S3) { + configuration.set(Constants.FS_DEFAULTFS, PropertyUtils.getString(Constants.FS_DEFAULTFS)); + configuration.set(Constants.FS_S3A_ENDPOINT, PropertyUtils.getString(Constants.FS_S3A_ENDPOINT)); + configuration.set(Constants.FS_S3A_ACCESS_KEY, PropertyUtils.getString(Constants.FS_S3A_ACCESS_KEY)); + configuration.set(Constants.FS_S3A_SECRET_KEY, PropertyUtils.getString(Constants.FS_S3A_SECRET_KEY)); + fs = FileSystem.get(configuration); + } + + + String rmHaIds = PropertyUtils.getString(Constants.YARN_RESOURCEMANAGER_HA_RM_IDS); + String appAddress = PropertyUtils.getString(Constants.YARN_APPLICATION_STATUS_ADDRESS); + if (!StringUtils.isEmpty(rmHaIds)) { + appAddress = getAppAddress(appAddress, rmHaIds); + logger.info("appAddress : {}", appAddress); } + configuration.set(Constants.YARN_APPLICATION_STATUS_ADDRESS, appAddress); + } catch (Exception e) { + logger.error(e.getMessage(), e); } } @@ -187,14 +191,14 @@ public class HadoopUtils implements Closeable { /** * cat file on hdfs * - * @param hdfsFilePath hdfs file path + * @param hdfsFilePath hdfs file path * @return byte[] byte array * @throws IOException errors */ public byte[] catFile(String hdfsFilePath) throws IOException { - if(StringUtils.isBlank(hdfsFilePath)){ - logger.error("hdfs file path:{} is blank",hdfsFilePath); + if (StringUtils.isBlank(hdfsFilePath)) { + logger.error("hdfs file path:{} is blank", hdfsFilePath); return new byte[0]; } @@ -203,29 +207,28 @@ public class HadoopUtils implements Closeable { } - /** * cat file on hdfs * - * @param hdfsFilePath hdfs file path - * @param skipLineNums skip line numbers - * @param limit read how many lines + * @param hdfsFilePath hdfs file path + * @param skipLineNums skip line numbers + * @param limit read how many lines * @return content of file * @throws IOException errors */ public List catFile(String hdfsFilePath, int skipLineNums, int limit) throws IOException { - if (StringUtils.isBlank(hdfsFilePath)){ - logger.error("hdfs file path:{} is blank",hdfsFilePath); + if (StringUtils.isBlank(hdfsFilePath)) { + logger.error("hdfs file path:{} is blank", hdfsFilePath); return Collections.emptyList(); } - try (FSDataInputStream in = fs.open(new Path(hdfsFilePath))){ + try (FSDataInputStream in = fs.open(new Path(hdfsFilePath))) { BufferedReader br = new BufferedReader(new InputStreamReader(in)); Stream stream = br.lines().skip(skipLineNums).limit(limit); return stream.collect(Collectors.toList()); } - + } /** @@ -258,17 +261,17 @@ public class HadoopUtils implements Closeable { /** * the src file is on the local disk. Add it to FS at * the given dst name. - - * @param srcFile local file - * @param dstHdfsPath destination hdfs path - * @param deleteSource whether to delete the src - * @param overwrite whether to overwrite an existing file + * + * @param srcFile local file + * @param dstHdfsPath destination hdfs path + * @param deleteSource whether to delete the src + * @param overwrite whether to overwrite an existing file * @return if success or not * @throws IOException errors */ public boolean copyLocalToHdfs(String srcFile, String dstHdfsPath, boolean deleteSource, boolean overwrite) throws IOException { Path srcPath = new Path(srcFile); - Path dstPath= new Path(dstHdfsPath); + Path dstPath = new Path(dstHdfsPath); fs.copyFromLocalFile(deleteSource, overwrite, srcPath, dstPath); @@ -278,10 +281,10 @@ public class HadoopUtils implements Closeable { /** * copy hdfs file to local * - * @param srcHdfsFilePath source hdfs file path - * @param dstFile destination file - * @param deleteSource delete source - * @param overwrite overwrite + * @param srcHdfsFilePath source hdfs file path + * @param dstFile destination file + * @param deleteSource delete source + * @param overwrite overwrite * @return result of copy hdfs file to local * @throws IOException errors */ @@ -299,7 +302,7 @@ public class HadoopUtils implements Closeable { } } - if(!dstPath.getParentFile().exists()){ + if (!dstPath.getParentFile().exists()) { dstPath.getParentFile().mkdirs(); } @@ -307,14 +310,13 @@ public class HadoopUtils implements Closeable { } /** - * * delete a file * * @param hdfsFilePath the path to delete. - * @param recursive if path is a directory and set to - * true, the directory is deleted else throws an exception. In - * case of a file the recursive can be set to either true or false. - * @return true if delete is successful else false. + * @param recursive if path is a directory and set to + * true, the directory is deleted else throws an exception. In + * case of a file the recursive can be set to either true or false. + * @return true if delete is successful else false. * @throws IOException errors */ public boolean delete(String hdfsFilePath, boolean recursive) throws IOException { @@ -339,7 +341,7 @@ public class HadoopUtils implements Closeable { * @return {@link FileStatus} file status * @throws Exception errors */ - public FileStatus[] listFileStatus(String filePath)throws Exception{ + public FileStatus[] listFileStatus(String filePath) throws Exception { try { return fs.listStatus(new Path(filePath)); } catch (IOException e) { @@ -351,10 +353,11 @@ public class HadoopUtils implements Closeable { /** * Renames Path src to Path dst. Can take place on local fs * or remote DFS. + * * @param src path to be renamed * @param dst new path after rename - * @throws IOException on failure * @return true if rename is successful + * @throws IOException on failure */ public boolean rename(String src, String dst) throws IOException { return fs.rename(new Path(src), new Path(dst)); @@ -400,7 +403,6 @@ public class HadoopUtils implements Closeable { } /** - * * @return data hdfs path */ public static String getHdfsDataBasePath() { @@ -427,11 +429,11 @@ public class HadoopUtils implements Closeable { * hdfs user dir * * @param tenantCode tenant code - * @param userId user id + * @param userId user id * @return hdfs resource dir */ - public static String getHdfsUserDir(String tenantCode,int userId) { - return String.format("%s/home/%d", getHdfsTenantDir(tenantCode),userId); + public static String getHdfsUserDir(String tenantCode, int userId) { + return String.format("%s/home/%d", getHdfsTenantDir(tenantCode), userId); } /** @@ -479,7 +481,7 @@ public class HadoopUtils implements Closeable { * getAppAddress * * @param appAddress app address - * @param rmHa resource manager ha + * @param rmHa resource manager ha * @return app address */ public static String getAppAddress(String appAddress, String rmHa) { diff --git a/dolphinscheduler-common/src/main/resources/common.properties b/dolphinscheduler-common/src/main/resources/common.properties index 5a4aa1441f..8391a9e2d6 100644 --- a/dolphinscheduler-common/src/main/resources/common.properties +++ b/dolphinscheduler-common/src/main/resources/common.properties @@ -91,4 +91,4 @@ yarn.resourcemanager.ha.rm.ids=192.168.xx.xx,192.168.xx.xx # If it is a single resourcemanager, you only need to configure one host name. If it is resourcemanager HA, the default configuration is fine yarn.application.status.address=http://ark1:8088/ws/v1/cluster/apps/%s - +kerberos.expire.time=7 From 1658176ef52e5a31414525f59d3d1fa2f1319bac Mon Sep 17 00:00:00 2001 From: "gabry.wu" Date: Sun, 15 Mar 2020 21:25:52 +0800 Subject: [PATCH 102/221] Adapting partial code(file name start with P #1) to the sonar cloud rule (#2171) (#2187) --- .../utils/placeholder/PlaceholderUtils.java | 6 +-- .../utils/process/ProcessBuilderForWin32.java | 10 ++++- .../dao/datasource/PostgreDataSource.java | 2 +- .../dao/upgrade/PostgresqlUpgradeDao.java | 39 +++---------------- .../dao/utils/PostgrePerformance.java | 3 +- .../mapper/ProcessDefinitionMapperTest.java | 6 +-- .../dolphinscheduler/remote/command/Ping.java | 4 +- .../dolphinscheduler/remote/command/Pong.java | 4 +- .../server/utils/ParamUtils.java | 22 +++++------ .../task/processdure/ProcedureTask.java | 27 ++++++------- .../service/permission/PermissionCheck.java | 2 +- 11 files changed, 49 insertions(+), 76 deletions(-) diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/placeholder/PlaceholderUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/placeholder/PlaceholderUtils.java index 0c756cb0b3..53a0d5af5b 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/placeholder/PlaceholderUtils.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/placeholder/PlaceholderUtils.java @@ -31,12 +31,12 @@ public class PlaceholderUtils { /** * Prefix of the position to be replaced */ - public static final String placeholderPrefix = "${"; + public static final String PLACEHOLDER_PREFIX = "${"; /** * The suffix of the position to be replaced */ - public static final String placeholderSuffix = "}"; + public static final String PLACEHOLDER_SUFFIX = "}"; /** @@ -68,7 +68,7 @@ public class PlaceholderUtils { */ public static PropertyPlaceholderHelper getPropertyPlaceholderHelper(boolean ignoreUnresolvablePlaceholders) { - return new PropertyPlaceholderHelper(placeholderPrefix, placeholderSuffix, null, ignoreUnresolvablePlaceholders); + return new PropertyPlaceholderHelper(PLACEHOLDER_PREFIX, PLACEHOLDER_SUFFIX, null, ignoreUnresolvablePlaceholders); } /** diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/process/ProcessBuilderForWin32.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/process/ProcessBuilderForWin32.java index 4fb5f94616..eee456d019 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/process/ProcessBuilderForWin32.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/process/ProcessBuilderForWin32.java @@ -426,6 +426,7 @@ public class ProcessBuilderForWin32 { static final ProcessBuilderForWin32.NullInputStream INSTANCE = new ProcessBuilderForWin32.NullInputStream(); private NullInputStream() {} public int read() { return -1; } + @Override public int available() { return 0; } } @@ -462,7 +463,7 @@ public class ProcessBuilderForWin32 { * * @since 1.7 */ - public static abstract class Redirect { + public abstract static class Redirect { /** * The type of a {@link ProcessBuilderForWin32.Redirect}. */ @@ -494,7 +495,7 @@ public class ProcessBuilderForWin32 { * {@link ProcessBuilderForWin32.Redirect#appendTo Redirect.appendTo(File)}. */ APPEND - }; + } /** * Returns the type of this {@code Redirect}. @@ -568,6 +569,7 @@ public class ProcessBuilderForWin32 { throw new NullPointerException(); return new ProcessBuilderForWin32.Redirect() { public Type type() { return Type.READ; } + @Override public File file() { return file; } public String toString() { return "redirect to read from file \"" + file + "\""; @@ -595,10 +597,12 @@ public class ProcessBuilderForWin32 { throw new NullPointerException(); return new ProcessBuilderForWin32.Redirect() { public Type type() { return Type.WRITE; } + @Override public File file() { return file; } public String toString() { return "redirect to write to file \"" + file + "\""; } + @Override boolean append() { return false; } }; } @@ -626,10 +630,12 @@ public class ProcessBuilderForWin32 { throw new NullPointerException(); return new ProcessBuilderForWin32.Redirect() { public Type type() { return Type.APPEND; } + @Override public File file() { return file; } public String toString() { return "redirect to append to file \"" + file + "\""; } + @Override boolean append() { return true; } }; } diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/PostgreDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/PostgreDataSource.java index 176cba2587..c01dfa4dc0 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/PostgreDataSource.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/PostgreDataSource.java @@ -40,7 +40,7 @@ public class PostgreDataSource extends BaseDataSource { @Override public String getJdbcUrl() { String jdbcUrl = getAddress(); - if (jdbcUrl.lastIndexOf("/") != (jdbcUrl.length() - 1)) { + if (jdbcUrl.lastIndexOf('/') != (jdbcUrl.length() - 1)) { jdbcUrl += "/"; } diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/upgrade/PostgresqlUpgradeDao.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/upgrade/PostgresqlUpgradeDao.java index 5db273642a..c58521de1d 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/upgrade/PostgresqlUpgradeDao.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/upgrade/PostgresqlUpgradeDao.java @@ -30,16 +30,8 @@ import java.sql.SQLException; */ public class PostgresqlUpgradeDao extends UpgradeDao { - public static final Logger logger = LoggerFactory.getLogger(UpgradeDao.class); - private static final String schema = getSchema(); - - /** - * init - */ - @Override - protected void init() { - - } + public static final Logger logger = LoggerFactory.getLogger(PostgresqlUpgradeDao.class); + private static final String SCHEMA = getSchema(); /** * postgresql upgrade dao holder @@ -58,16 +50,6 @@ public class PostgresqlUpgradeDao extends UpgradeDao { return PostgresqlUpgradeDaoHolder.INSTANCE; } - - /** - * init schema - * @param initSqlPath initSqlPath - */ - @Override - public void initSchema(String initSqlPath) { - super.initSchema(initSqlPath); - } - /** * getSchema * @return schema @@ -107,13 +89,9 @@ public class PostgresqlUpgradeDao extends UpgradeDao { try { conn = dataSource.getConnection(); - rs = conn.getMetaData().getTables(null, schema, tableName, null); - if (rs.next()) { - return true; - } else { - return false; - } + rs = conn.getMetaData().getTables(null, SCHEMA, tableName, null); + return rs.next(); } catch (SQLException e) { logger.error(e.getMessage(),e); throw new RuntimeException(e.getMessage(),e); @@ -135,13 +113,8 @@ public class PostgresqlUpgradeDao extends UpgradeDao { ResultSet rs = null; try { conn = dataSource.getConnection(); - rs = conn.getMetaData().getColumns(null,schema,tableName,columnName); - if (rs.next()) { - return true; - } else { - return false; - } - + rs = conn.getMetaData().getColumns(null, SCHEMA,tableName,columnName); + return rs.next(); } catch (SQLException e) { logger.error(e.getMessage(),e); throw new RuntimeException(e.getMessage(),e); diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/PostgrePerformance.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/PostgrePerformance.java index 031fd00681..b1cdf6f179 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/PostgrePerformance.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/PostgrePerformance.java @@ -24,7 +24,6 @@ import java.util.Date; import org.apache.dolphinscheduler.common.enums.DbType; import org.apache.dolphinscheduler.common.enums.Flag; -import org.apache.dolphinscheduler.dao.MonitorDBDao; import org.apache.dolphinscheduler.dao.entity.MonitorRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -34,7 +33,7 @@ import org.slf4j.LoggerFactory; */ public class PostgrePerformance extends BaseDBPerformance { - private static Logger logger = LoggerFactory.getLogger(MonitorDBDao.class); + private static Logger logger = LoggerFactory.getLogger(PostgrePerformance.class); /** * get monitor record diff --git a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/ProcessDefinitionMapperTest.java b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/ProcessDefinitionMapperTest.java index 1b3db55608..775d10656a 100644 --- a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/ProcessDefinitionMapperTest.java +++ b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/ProcessDefinitionMapperTest.java @@ -77,7 +77,7 @@ public class ProcessDefinitionMapperTest { //update processDefinition.setUpdateTime(new Date()); int update = processDefinitionMapper.updateById(processDefinition); - Assert.assertEquals(update, 1); + Assert.assertEquals(1, update); processDefinitionMapper.deleteById(processDefinition.getId()); } @@ -88,7 +88,7 @@ public class ProcessDefinitionMapperTest { public void testDelete(){ ProcessDefinition processDefinition = insertOne(); int delete = processDefinitionMapper.deleteById(processDefinition.getId()); - Assert.assertEquals(delete, 1); + Assert.assertEquals(1, delete); } /** @@ -189,7 +189,7 @@ public class ProcessDefinitionMapperTest { List processDefinitions = processDefinitionMapper.queryDefinitionListByIdList(array); processDefinitionMapper.deleteById(processDefinition.getId()); processDefinitionMapper.deleteById(processDefinition1.getId()); - Assert.assertEquals(processDefinitions.size(), 2); + Assert.assertEquals(2, processDefinitions.size()); } diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Ping.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Ping.java index c5e4d075af..f90d3fff18 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Ping.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Ping.java @@ -30,12 +30,12 @@ public class Ping implements Serializable { /** * ping body */ - protected static ByteBuf EMPTY_BODY = Unpooled.EMPTY_BUFFER; + protected static final ByteBuf EMPTY_BODY = Unpooled.EMPTY_BUFFER; /** * request command body */ - private static byte[] EMPTY_BODY_ARRAY = new byte[0]; + private static final byte[] EMPTY_BODY_ARRAY = new byte[0]; private static final ByteBuf PING_BUF; diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Pong.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Pong.java index e52cef6d92..1b51373bff 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Pong.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/command/Pong.java @@ -30,12 +30,12 @@ public class Pong implements Serializable { /** * pong body */ - protected static ByteBuf EMPTY_BODY = Unpooled.EMPTY_BUFFER; + protected static final ByteBuf EMPTY_BODY = Unpooled.EMPTY_BUFFER; /** * pong command body */ - private static byte[] EMPTY_BODY_ARRAY = new byte[0]; + private static final byte[] EMPTY_BODY_ARRAY = new byte[0]; /** * ping byte buffer diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ParamUtils.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ParamUtils.java index 1d7a80daf0..3040cd198d 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ParamUtils.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ParamUtils.java @@ -19,6 +19,7 @@ package org.apache.dolphinscheduler.server.utils; import org.apache.dolphinscheduler.common.enums.CommandType; import org.apache.dolphinscheduler.common.process.Property; import org.apache.dolphinscheduler.common.utils.ParameterUtils; +import org.apache.dolphinscheduler.common.utils.StringUtils; import org.apache.dolphinscheduler.common.utils.placeholder.BusinessTimeUtils; import java.util.Date; @@ -70,17 +71,16 @@ public class ParamUtils { Map.Entry en = iter.next(); Property property = en.getValue(); - if (property.getValue() != null && property.getValue().length() > 0){ - if (property.getValue().startsWith("$")){ - /** - * local parameter refers to global parameter with the same name - * note: the global parameters of the process instance here are solidified parameters, - * and there are no variables in them. - */ - String val = property.getValue(); - val = ParameterUtils.convertParameterPlaceholders(val, timeParams); - property.setValue(val); - } + if (StringUtils.isNotEmpty(property.getValue()) + && property.getValue().startsWith("$")){ + /** + * local parameter refers to global parameter with the same name + * note: the global parameters of the process instance here are solidified parameters, + * and there are no variables in them. + */ + String val = property.getValue(); + val = ParameterUtils.convertParameterPlaceholders(val, timeParams); + property.setValue(val); } } diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java index fd00e517b5..5a8afdb7bb 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/processdure/ProcedureTask.java @@ -60,11 +60,6 @@ public class ProcedureTask extends AbstractTask { */ private ProcessService processService; - /** - * base datasource - */ - private BaseDataSource baseDataSource; - /** * constructor * @param taskProps task props @@ -117,7 +112,7 @@ public class ProcedureTask extends AbstractTask { // load class DataSourceFactory.loadClass(dataSource.getType()); // get datasource - baseDataSource = DataSourceFactory.getDatasource(dataSource.getType(), + BaseDataSource baseDataSource = DataSourceFactory.getDatasource(dataSource.getType(), dataSource.getConnectionParams()); // get jdbc connection @@ -163,7 +158,7 @@ public class ProcedureTask extends AbstractTask { stmt.setQueryTimeout(taskProps.getTaskTimeout()); } Map outParameterMap = new HashMap<>(); - if (userDefParamsList != null && userDefParamsList.size() > 0){ + if (CollectionUtils.isNotEmpty(userDefParamsList)){ int index = 1; for (Property property : userDefParamsList){ logger.info("localParams : prop : {} , dirct : {} , type : {} , value : {}" @@ -237,31 +232,31 @@ public class ProcedureTask extends AbstractTask { private void getOutputParameter(CallableStatement stmt, int index, String prop, DataType dataType) throws SQLException { switch (dataType){ case VARCHAR: - logger.info("out prameter key : {} , value : {}",prop,stmt.getString(index)); + logger.info("out prameter varchar key : {} , value : {}",prop,stmt.getString(index)); break; case INTEGER: - logger.info("out prameter key : {} , value : {}", prop, stmt.getInt(index)); + logger.info("out prameter integer key : {} , value : {}", prop, stmt.getInt(index)); break; case LONG: - logger.info("out prameter key : {} , value : {}",prop,stmt.getLong(index)); + logger.info("out prameter long key : {} , value : {}",prop,stmt.getLong(index)); break; case FLOAT: - logger.info("out prameter key : {} , value : {}",prop,stmt.getFloat(index)); + logger.info("out prameter float key : {} , value : {}",prop,stmt.getFloat(index)); break; case DOUBLE: - logger.info("out prameter key : {} , value : {}",prop,stmt.getDouble(index)); + logger.info("out prameter double key : {} , value : {}",prop,stmt.getDouble(index)); break; case DATE: - logger.info("out prameter key : {} , value : {}",prop,stmt.getDate(index)); + logger.info("out prameter date key : {} , value : {}",prop,stmt.getDate(index)); break; case TIME: - logger.info("out prameter key : {} , value : {}",prop,stmt.getTime(index)); + logger.info("out prameter time key : {} , value : {}",prop,stmt.getTime(index)); break; case TIMESTAMP: - logger.info("out prameter key : {} , value : {}",prop,stmt.getTimestamp(index)); + logger.info("out prameter timestamp key : {} , value : {}",prop,stmt.getTimestamp(index)); break; case BOOLEAN: - logger.info("out prameter key : {} , value : {}",prop, stmt.getBoolean(index)); + logger.info("out prameter boolean key : {} , value : {}",prop, stmt.getBoolean(index)); break; default: break; diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/permission/PermissionCheck.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/permission/PermissionCheck.java index 027666f053..e53fae6e86 100644 --- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/permission/PermissionCheck.java +++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/permission/PermissionCheck.java @@ -147,7 +147,7 @@ public class PermissionCheck { List unauthorizedList = processService.listUnauthorized(userId,needChecks,authorizationType); // if exist unauthorized resource if(CollectionUtils.isNotEmpty(unauthorizedList)){ - logger.error("user {} didn't has permission of {}: {}", user.getUserName(), authorizationType.getDescp(),unauthorizedList.toString()); + logger.error("user {} didn't has permission of {}: {}", user.getUserName(), authorizationType.getDescp(),unauthorizedList); throw new RuntimeException(String.format("user %s didn't has permission of %s %s", user.getUserName(), authorizationType.getDescp(), unauthorizedList.get(0))); } } From f6f29f4abb9f8034441a05ea4b7f85eef4f9f100 Mon Sep 17 00:00:00 2001 From: t1mon <178317391@qq.com> Date: Mon, 16 Mar 2020 17:10:55 +0800 Subject: [PATCH 103/221] Replace Charset.forName("UTF-8") with StandardCharsets.UTF_8. (#2193) --- .../dolphinscheduler/api/service/ProcessInstanceService.java | 5 +++-- .../org/apache/dolphinscheduler/remote/utils/Constants.java | 3 ++- .../apache/dolphinscheduler/server/utils/ProcessUtils.java | 4 ++-- .../dolphinscheduler/server/worker/task/datax/DataxTask.java | 4 ++-- 4 files changed, 9 insertions(+), 7 deletions(-) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessInstanceService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessInstanceService.java index 4f81d89505..3530c66ff6 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessInstanceService.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/ProcessInstanceService.java @@ -16,6 +16,7 @@ */ package org.apache.dolphinscheduler.api.service; +import java.nio.charset.StandardCharsets; import org.apache.dolphinscheduler.api.dto.gantt.GanttDto; import org.apache.dolphinscheduler.api.dto.gantt.Task; import org.apache.dolphinscheduler.api.enums.Status; @@ -49,7 +50,6 @@ import java.io.BufferedReader; import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.InputStreamReader; -import java.nio.charset.Charset; import java.text.ParseException; import java.util.*; import java.util.stream.Collectors; @@ -273,7 +273,8 @@ public class ProcessInstanceService extends BaseDAGService { return resultMap; } - BufferedReader br = new BufferedReader(new InputStreamReader(new ByteArrayInputStream(log.getBytes(Charset.forName("utf8"))), Charset.forName("utf8"))); + BufferedReader br = new BufferedReader(new InputStreamReader(new ByteArrayInputStream(log.getBytes( + StandardCharsets.UTF_8)), StandardCharsets.UTF_8)); String line; while ((line = br.readLine()) != null) { if(line.contains(DEPENDENT_SPLIT)){ diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Constants.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Constants.java index 5733b17790..f4791715b8 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Constants.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/Constants.java @@ -17,6 +17,7 @@ package org.apache.dolphinscheduler.remote.utils; import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; /** @@ -31,7 +32,7 @@ public class Constants { /** * charset */ - public static final Charset UTF8 = Charset.forName("UTF-8"); + public static final Charset UTF8 = StandardCharsets.UTF_8; /** * cpus diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ProcessUtils.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ProcessUtils.java index e0c00c55d9..f3f8a79489 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ProcessUtils.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/utils/ProcessUtils.java @@ -16,6 +16,7 @@ */ package org.apache.dolphinscheduler.server.utils; +import java.nio.charset.StandardCharsets; import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.utils.CommonUtils; import org.apache.dolphinscheduler.common.utils.LoggerUtils; @@ -29,7 +30,6 @@ import org.slf4j.LoggerFactory; import java.io.File; import java.io.IOException; -import java.nio.charset.Charset; import java.util.ArrayList; import java.util.List; import java.util.regex.Matcher; @@ -297,7 +297,7 @@ public class ProcessUtils { File f = new File(commandFile); if (!f.exists()) { - FileUtils.writeStringToFile(new File(commandFile), sb.toString(), Charset.forName("UTF-8")); + FileUtils.writeStringToFile(new File(commandFile), sb.toString(), StandardCharsets.UTF_8); } String runCmd = "sh " + commandFile; diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java index 7537ca2edc..952030ea62 100755 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/datax/DataxTask.java @@ -18,7 +18,7 @@ package org.apache.dolphinscheduler.server.worker.task.datax; import java.io.File; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.StandardOpenOption; @@ -209,7 +209,7 @@ public class DataxTask extends AbstractTask { logger.debug("datax job json : {}", root.toString()); // create datax json file - FileUtils.writeStringToFile(new File(fileName), root.toString(), Charset.forName("UTF-8")); + FileUtils.writeStringToFile(new File(fileName), root.toString(), StandardCharsets.UTF_8); return fileName; } From a3767642a546a4dac11975b0032eb80766871134 Mon Sep 17 00:00:00 2001 From: tswstarplanet Date: Mon, 16 Mar 2020 18:06:13 +0800 Subject: [PATCH 104/221] change static field to instance field (#2190) --- .../apache/dolphinscheduler/common/utils/HadoopUtils.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/HadoopUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/HadoopUtils.java index e767911580..6c42704b47 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/HadoopUtils.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/HadoopUtils.java @@ -64,8 +64,8 @@ public class HadoopUtils implements Closeable { } }); - private static Configuration configuration; - private static FileSystem fs; + private Configuration configuration; + private FileSystem fs; private static String hdfsUser = PropertyUtils.getString(Constants.HDFS_ROOT_USER); @@ -99,7 +99,7 @@ public class HadoopUtils implements Closeable { /** * init hadoop configuration */ - private static void init() { + private void init() { try { configuration = new Configuration(); From f28ba55014b30c5498bf823dff5259dcd6c25af4 Mon Sep 17 00:00:00 2001 From: tswstarplanet Date: Mon, 16 Mar 2020 18:56:36 +0800 Subject: [PATCH 105/221] fix ResultSet not close and reformat code (#2183) * fix ResultSet not close * fix ResultSet not close * extract code of closing resource to a method * remove redundant if condition * modify e2e timeout --- .../dolphinscheduler/dao/TaskRecordDao.java | 134 ++++++++++-------- .../test/resources/config/config.properties | 8 +- 2 files changed, 79 insertions(+), 63 deletions(-) diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/TaskRecordDao.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/TaskRecordDao.java index f7ba1054a5..f89002ebdc 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/TaskRecordDao.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/TaskRecordDao.java @@ -50,25 +50,28 @@ public class TaskRecordDao { static { try { conf = new PropertiesConfiguration(Constants.APPLICATION_PROPERTIES); - }catch (ConfigurationException e){ - logger.error("load configuration exception",e); + } catch (ConfigurationException e) { + logger.error("load configuration exception", e); System.exit(1); } } /** - * get task record flag + * get task record flag + * * @return whether startup taskrecord */ - public static boolean getTaskRecordFlag(){ - return conf.getBoolean(Constants.TASK_RECORD_FLAG); + public static boolean getTaskRecordFlag() { + return conf.getBoolean(Constants.TASK_RECORD_FLAG); } + /** * create connection + * * @return connection */ private static Connection getConn() { - if(!getTaskRecordFlag()){ + if (!getTaskRecordFlag()) { return null; } String driver = "com.mysql.jdbc.Driver"; @@ -90,101 +93,96 @@ public class TaskRecordDao { /** * generate where sql string + * * @param filterMap filterMap * @return sql string */ private static String getWhereString(Map filterMap) { - if(filterMap.size() ==0){ + if (filterMap.size() == 0) { return ""; } String result = " where 1=1 "; Object taskName = filterMap.get("taskName"); - if(taskName != null && StringUtils.isNotEmpty(taskName.toString())){ + if (taskName != null && StringUtils.isNotEmpty(taskName.toString())) { result += " and PROC_NAME like concat('%', '" + taskName.toString() + "', '%') "; } Object taskDate = filterMap.get("taskDate"); - if(taskDate != null && StringUtils.isNotEmpty(taskDate.toString())){ + if (taskDate != null && StringUtils.isNotEmpty(taskDate.toString())) { result += " and PROC_DATE='" + taskDate.toString() + "'"; } Object state = filterMap.get("state"); - if(state != null && StringUtils.isNotEmpty(state.toString())){ + if (state != null && StringUtils.isNotEmpty(state.toString())) { result += " and NOTE='" + state.toString() + "'"; } Object sourceTable = filterMap.get("sourceTable"); - if(sourceTable!= null && StringUtils.isNotEmpty(sourceTable.toString())){ - result += " and SOURCE_TAB like concat('%', '" + sourceTable.toString()+ "', '%')"; + if (sourceTable != null && StringUtils.isNotEmpty(sourceTable.toString())) { + result += " and SOURCE_TAB like concat('%', '" + sourceTable.toString() + "', '%')"; } Object targetTable = filterMap.get("targetTable"); - if(sourceTable!= null && StringUtils.isNotEmpty(targetTable.toString())){ - result += " and TARGET_TAB like concat('%', '"+ targetTable.toString()+"', '%') " ; + if (sourceTable != null && StringUtils.isNotEmpty(targetTable.toString())) { + result += " and TARGET_TAB like concat('%', '" + targetTable.toString() + "', '%') "; } Object start = filterMap.get("startTime"); - if(start != null && StringUtils.isNotEmpty(start.toString())){ + if (start != null && StringUtils.isNotEmpty(start.toString())) { result += " and STARTDATE>='" + start.toString() + "'"; } Object end = filterMap.get("endTime"); - if(end != null && StringUtils.isNotEmpty(end.toString())){ - result += " and ENDDATE>='" + end.toString()+ "'"; + if (end != null && StringUtils.isNotEmpty(end.toString())) { + result += " and ENDDATE>='" + end.toString() + "'"; } return result; } /** * count task record + * * @param filterMap filterMap - * @param table table + * @param table table * @return task record count */ - public static int countTaskRecord(Map filterMap, String table){ + public static int countTaskRecord(Map filterMap, String table) { int count = 0; Connection conn = null; PreparedStatement pstmt = null; + ResultSet rs = null; try { conn = getConn(); - if(conn == null){ + if (conn == null) { return count; } String sql = String.format("select count(1) as count from %s", table); sql += getWhereString(filterMap); pstmt = conn.prepareStatement(sql); - ResultSet rs = pstmt.executeQuery(); - while(rs.next()){ + rs = pstmt.executeQuery(); + while (rs.next()) { count = rs.getInt("count"); break; } } catch (SQLException e) { logger.error("Exception ", e); - }finally { - try { - if(pstmt != null) { - pstmt.close(); - } - if(conn != null){ - conn.close(); - } - } catch (SQLException e) { - logger.error("Exception ", e); - } + } finally { + closeResource(rs, pstmt, conn); } return count; } /** * query task record by filter map paging + * * @param filterMap filterMap - * @param table table + * @param table table * @return task record list */ - public static List queryAllTaskRecord(Map filterMap , String table) { + public static List queryAllTaskRecord(Map filterMap, String table) { String sql = String.format("select * from %s", table); sql += getWhereString(filterMap); @@ -194,9 +192,9 @@ public class TaskRecordDao { sql += String.format(" order by STARTDATE desc limit %d,%d", offset, pageSize); List recordList = new ArrayList<>(); - try{ + try { recordList = getQueryResult(sql); - }catch (Exception e){ + } catch (Exception e) { logger.error("Exception ", e); } return recordList; @@ -204,6 +202,7 @@ public class TaskRecordDao { /** * convert result set to task record + * * @param resultSet resultSet * @return task record * @throws SQLException if error throws SQLException @@ -232,6 +231,7 @@ public class TaskRecordDao { /** * query task list by select sql + * * @param selectSql select sql * @return task record list */ @@ -239,65 +239,81 @@ public class TaskRecordDao { List recordList = new ArrayList<>(); Connection conn = null; PreparedStatement pstmt = null; + ResultSet rs = null; try { conn = getConn(); - if(conn == null){ + if (conn == null) { return recordList; } pstmt = conn.prepareStatement(selectSql); - ResultSet rs = pstmt.executeQuery(); + rs = pstmt.executeQuery(); - while(rs.next()){ + while (rs.next()) { TaskRecord taskRecord = convertToTaskRecord(rs); recordList.add(taskRecord); } } catch (SQLException e) { logger.error("Exception ", e); - }finally { - try { - if(pstmt != null) { - pstmt.close(); - } - if(conn != null){ - conn.close(); - } - } catch (SQLException e) { - logger.error("Exception ", e); - } + } finally { + closeResource(rs, pstmt, conn); } return recordList; } /** * according to procname and procdate query task record + * * @param procName procName * @param procDate procDate * @return task record status */ - public static TaskRecordStatus getTaskRecordState(String procName,String procDate){ + public static TaskRecordStatus getTaskRecordState(String procName, String procDate) { String sql = String.format("SELECT * FROM eamp_hive_log_hd WHERE PROC_NAME='%s' and PROC_DATE like '%s'" - ,procName,procDate + "%"); + , procName, procDate + "%"); List taskRecordList = getQueryResult(sql); // contains no record and sql exception - if (CollectionUtils.isEmpty(taskRecordList)){ + if (CollectionUtils.isEmpty(taskRecordList)) { // exception return TaskRecordStatus.EXCEPTION; - }else if (taskRecordList.size() > 1){ + } else if (taskRecordList.size() > 1) { return TaskRecordStatus.EXCEPTION; - }else { + } else { TaskRecord taskRecord = taskRecordList.get(0); - if (taskRecord == null){ + if (taskRecord == null) { return TaskRecordStatus.EXCEPTION; } Long targetRowCount = taskRecord.getTargetRowCount(); - if (targetRowCount <= 0){ + if (targetRowCount <= 0) { return TaskRecordStatus.FAILURE; - }else { + } else { return TaskRecordStatus.SUCCESS; } } } + private static void closeResource(ResultSet rs, PreparedStatement pstmt, Connection conn) { + if (rs != null) { + try { + rs.close(); + } catch (SQLException e) { + logger.error("Exception ", e); + } + } + if (pstmt != null) { + try { + pstmt.close(); + } catch (SQLException e) { + logger.error("Exception ", e); + } + } + if (conn != null) { + try { + conn.close(); + } catch (SQLException e) { + logger.error("Exception ", e); + } + } + } } diff --git a/e2e/src/test/resources/config/config.properties b/e2e/src/test/resources/config/config.properties index 6a01234d01..352d9375be 100644 --- a/e2e/src/test/resources/config/config.properties +++ b/e2e/src/test/resources/config/config.properties @@ -27,13 +27,13 @@ PASSWORD=dolphinscheduler123 # driver path driver.chromeDriver=/usr/local/bin/chromedriver # implicitly wait(s) -driver.timeouts.implicitlyWait=10 +driver.timeouts.implicitlyWait=30 # show wait(s) -driver.timeouts.webDriverWait=10 +driver.timeouts.webDriverWait=30 # page load timeout(s) -driver.timeouts.pageLoadTimeout=10 +driver.timeouts.pageLoadTimeout=30 # JS wait timeouts(s) -driver.timeouts.setScriptTimeout=10 +driver.timeouts.setScriptTimeout=30 ############### redis ############## From 69633a29c60ac1c98704a419952f4e1cf56ea518 Mon Sep 17 00:00:00 2001 From: Rubik-W <39549317+Rubik-W@users.noreply.github.com> Date: Tue, 17 Mar 2020 11:10:55 +0800 Subject: [PATCH 106/221] fix: Api server startup fail (#2198) --- .../org/apache/dolphinscheduler/api/ApiApplicationServer.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/ApiApplicationServer.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/ApiApplicationServer.java index 8376c2876d..5998ec5a4d 100644 --- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/ApiApplicationServer.java +++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/ApiApplicationServer.java @@ -25,7 +25,9 @@ import springfox.documentation.swagger2.annotations.EnableSwagger2; @SpringBootApplication @ServletComponentScan -@ComponentScan("org.apache.dolphinscheduler") +@ComponentScan({"org.apache.dolphinscheduler.api", + "org.apache.dolphinscheduler.dao", + "org.apache.dolphinscheduler.service"}) public class ApiApplicationServer extends SpringBootServletInitializer { public static void main(String[] args) { From c5e1c55c0003e3d5da525314a7af906bda545ccb Mon Sep 17 00:00:00 2001 From: "gabry.wu" Date: Tue, 17 Mar 2020 14:58:16 +0800 Subject: [PATCH 107/221] Adapting partial code(file name start with M #2) to the sonar cloud rule (#2179) * Adapting partial code(file name start with M #2) to the sonar cloud rule * remove unused import --- .../dolphinscheduler/dao/MonitorDBDao.java | 2 +- .../dao/datasource/MySQLDataSource.java | 2 +- .../dao/upgrade/MysqlUpgradeDao.java | 23 +++---------------- .../dao/utils/MysqlPerformance.java | 3 +-- .../server/master/MasterServer.java | 3 +-- .../master/runner/MasterTaskExecThread.java | 7 +++--- .../server/monitor/MonitorServer.java | 2 +- .../sources/MysqlSourceGenerator.java | 19 ++++++++------- .../targets/MysqlTargetGenerator.java | 11 ++++----- 9 files changed, 25 insertions(+), 47 deletions(-) diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/MonitorDBDao.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/MonitorDBDao.java index 51f60666d1..5ea5966238 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/MonitorDBDao.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/MonitorDBDao.java @@ -61,7 +61,7 @@ public class MonitorDBDao { return new PostgrePerformance().getMonitorRecord(conn); } }catch (Exception e) { - logger.error("SQLException " + e); + logger.error("SQLException: {}", e.getMessage(), e); }finally { try { if (conn != null) { diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/MySQLDataSource.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/MySQLDataSource.java index 6e2fbe3dd8..969c17b54d 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/MySQLDataSource.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/MySQLDataSource.java @@ -39,7 +39,7 @@ public class MySQLDataSource extends BaseDataSource { @Override public String getJdbcUrl() { String address = getAddress(); - if (address.lastIndexOf("/") != (address.length() - 1)) { + if (address.lastIndexOf('/') != (address.length() - 1)) { address += "/"; } String jdbcUrl = address + getDatabase(); diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/upgrade/MysqlUpgradeDao.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/upgrade/MysqlUpgradeDao.java index 58bd673fc5..a20a3acb95 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/upgrade/MysqlUpgradeDao.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/upgrade/MysqlUpgradeDao.java @@ -29,15 +29,7 @@ import java.sql.SQLException; */ public class MysqlUpgradeDao extends UpgradeDao { - public static final Logger logger = LoggerFactory.getLogger(UpgradeDao.class); - - /** - * init - */ - @Override - protected void init() { - - } + public static final Logger logger = LoggerFactory.getLogger(MysqlUpgradeDao.class); /** * mysql upgrade dao holder @@ -69,12 +61,7 @@ public class MysqlUpgradeDao extends UpgradeDao { try { conn = dataSource.getConnection(); rs = conn.getMetaData().getTables(null, null, tableName, null); - if (rs.next()) { - return true; - } else { - return false; - } - + return rs.next(); } catch (SQLException e) { logger.error(e.getMessage(),e); throw new RuntimeException(e.getMessage(),e); @@ -96,11 +83,7 @@ public class MysqlUpgradeDao extends UpgradeDao { try { conn = dataSource.getConnection(); ResultSet rs = conn.getMetaData().getColumns(null,null,tableName,columnName); - if (rs.next()) { - return true; - } else { - return false; - } + return rs.next(); } catch (SQLException e) { logger.error(e.getMessage(),e); diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/MysqlPerformance.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/MysqlPerformance.java index 40d12ab36f..1e1ee78036 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/MysqlPerformance.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/utils/MysqlPerformance.java @@ -27,7 +27,6 @@ import java.util.Date; import org.apache.dolphinscheduler.common.enums.DbType; import org.apache.dolphinscheduler.common.enums.Flag; -import org.apache.dolphinscheduler.dao.MonitorDBDao; import org.apache.dolphinscheduler.dao.entity.MonitorRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,7 +36,7 @@ import org.slf4j.LoggerFactory; */ public class MysqlPerformance extends BaseDBPerformance{ - private static Logger logger = LoggerFactory.getLogger(MonitorDBDao.class); + private static Logger logger = LoggerFactory.getLogger(MysqlPerformance.class); /** diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java index 9512b1a1c2..bf58607262 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java @@ -243,7 +243,7 @@ public class MasterServer implements IStoppable { */ private Runnable heartBeatThread(){ logger.info("start master heart beat thread..."); - Runnable heartBeatThread = new Runnable() { + return new Runnable() { @Override public void run() { if(Stopper.isRunning()) { @@ -257,7 +257,6 @@ public class MasterServer implements IStoppable { } } }; - return heartBeatThread; } } diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecThread.java index 3cdb8a0271..38b5419a92 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecThread.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecThread.java @@ -64,7 +64,7 @@ public class MasterTaskExecThread extends MasterBaseTaskExecThread { /** * whether already Killed,default false */ - private Boolean alreadyKilled = false; + private boolean alreadyKilled = false; /** * submit task instance and wait complete @@ -98,7 +98,7 @@ public class MasterTaskExecThread extends MasterBaseTaskExecThread { logger.info("wait task: process id: {}, task id:{}, task name:{} complete", this.taskInstance.getProcessInstanceId(), this.taskInstance.getId(), this.taskInstance.getName()); // task time out - Boolean checkTimeout = false; + boolean checkTimeout = false; TaskTimeoutParameter taskTimeoutParameter = getTaskTimeoutParameter(); if(taskTimeoutParameter.getEnable()){ TaskTimeoutStrategy strategy = taskTimeoutParameter.getStrategy(); @@ -189,7 +189,6 @@ public class MasterTaskExecThread extends MasterBaseTaskExecThread { private long getRemaintime(long timeoutSeconds) { Date startTime = taskInstance.getStartTime(); long usedTime = (System.currentTimeMillis() - startTime.getTime()) / 1000; - long remainTime = timeoutSeconds - usedTime; - return remainTime; + return timeoutSeconds - usedTime; } } diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/monitor/MonitorServer.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/monitor/MonitorServer.java index ac549bc386..a1f43add6e 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/monitor/MonitorServer.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/monitor/MonitorServer.java @@ -30,7 +30,7 @@ import org.springframework.context.annotation.ComponentScan; @ComponentScan("org.apache.dolphinscheduler") public class MonitorServer implements CommandLineRunner { - private static Integer ARGS_LENGTH = 4; + private static final Integer ARGS_LENGTH = 4; private static final Logger logger = LoggerFactory.getLogger(MonitorServer.class); diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/sources/MysqlSourceGenerator.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/sources/MysqlSourceGenerator.java index 050fef7cc7..404f10bf39 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/sources/MysqlSourceGenerator.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/sources/MysqlSourceGenerator.java @@ -69,17 +69,16 @@ public class MysqlSourceGenerator implements ISourceGenerator { result.append(" --columns ").append(sourceMysqlParameter.getSrcColumns()); } - }else if(sourceMysqlParameter.getSrcQueryType() == QueryType.SQL.ordinal()){ - if(StringUtils.isNotEmpty(sourceMysqlParameter.getSrcQuerySql())){ - - String srcQuery = sourceMysqlParameter.getSrcQuerySql(); - if(srcQuery.toLowerCase().contains("where")){ - srcQuery += " AND "+"$CONDITIONS"; - }else{ - srcQuery += " WHERE $CONDITIONS"; - } - result.append(" --query \'"+srcQuery+"\'"); + }else if(sourceMysqlParameter.getSrcQueryType() == QueryType.SQL.ordinal() + && StringUtils.isNotEmpty(sourceMysqlParameter.getSrcQuerySql())){ + String srcQuery = sourceMysqlParameter.getSrcQuerySql(); + if(srcQuery.toLowerCase().contains("where")){ + srcQuery += " AND "+"$CONDITIONS"; + }else{ + srcQuery += " WHERE $CONDITIONS"; } + result.append(" --query \'"+srcQuery+"\'"); + } List mapColumnHive = sourceMysqlParameter.getMapColumnHive(); diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/targets/MysqlTargetGenerator.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/targets/MysqlTargetGenerator.java index 0733338812..0e33b176e5 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/targets/MysqlTargetGenerator.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sqoop/generator/targets/MysqlTargetGenerator.java @@ -75,12 +75,11 @@ public class MysqlTargetGenerator implements ITargetGenerator { result.append(" --lines-terminated-by '").append(targetMysqlParameter.getLinesTerminated()).append("'"); } - if(targetMysqlParameter.isUpdate()){ - if(StringUtils.isNotEmpty(targetMysqlParameter.getTargetUpdateKey())&& - StringUtils.isNotEmpty(targetMysqlParameter.getTargetUpdateMode())){ - result.append(" --update-key ").append(targetMysqlParameter.getTargetUpdateKey()) - .append(" --update-mode ").append(targetMysqlParameter.getTargetUpdateMode()); - } + if(targetMysqlParameter.isUpdate() + && StringUtils.isNotEmpty(targetMysqlParameter.getTargetUpdateKey()) + && StringUtils.isNotEmpty(targetMysqlParameter.getTargetUpdateMode())){ + result.append(" --update-key ").append(targetMysqlParameter.getTargetUpdateKey()) + .append(" --update-mode ").append(targetMysqlParameter.getTargetUpdateMode()); } } } From 7dd126da71acc9f2b64cb44f8a53e69f0ee50487 Mon Sep 17 00:00:00 2001 From: t1mon <178317391@qq.com> Date: Tue, 17 Mar 2020 15:01:42 +0800 Subject: [PATCH 108/221] Optimize PropertyUtils instantiation. (#2199) * Optimize PropertyUtils instantiation. * Fix info error. --- .../dolphinscheduler/common/utils/PropertyUtils.java | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/PropertyUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/PropertyUtils.java index 5c1011cd8b..ca928c94d5 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/PropertyUtils.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/PropertyUtils.java @@ -43,13 +43,11 @@ public class PropertyUtils { private static final Properties properties = new Properties(); - private static final PropertyUtils propertyUtils = new PropertyUtils(); - - private PropertyUtils(){ - init(); + private PropertyUtils() { + throw new IllegalStateException("PropertyUtils class"); } - private void init(){ + static { String[] propertyFiles = new String[]{COMMON_PROPERTIES_PATH}; for (String fileName : propertyFiles) { InputStream fis = null; From c82758c5054e0c8af8f2c9ef73d1af8653e2444f Mon Sep 17 00:00:00 2001 From: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> Date: Tue, 17 Mar 2020 17:57:12 +0800 Subject: [PATCH 109/221] =?UTF-8?q?e2e=20add=20the=20case=20of=20deleting?= =?UTF-8?q?=20user=20and=20tenant=20=C2=A0=20(#2210)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * add e2e --- .../dolphinscheduler/base/BaseDriver.java | 8 +- .../dolphinscheduler/base/BaseTest.java | 17 ++-- .../dolphinscheduler/data/LoginData.java | 2 +- .../data/project/CreatWorkflowData.java | 2 +- ...rojectData.java => CreateProjectData.java} | 4 +- .../data/project/CreateWorkflowData.java | 73 +++++++++++++++ .../data/security/TenantManageData.java | 10 +-- .../data/security/UserManageData.java | 5 +- .../project/CreateWorkflowLocator.java | 69 ++++++++++++-- .../locator/security/TenantManageLocator.java | 8 +- .../locator/security/UserManageLocator.java | 3 + .../dolphinscheduler/page/LoginPage.java | 14 ++- .../page/project/CreateProjectPage.java | 11 +-- .../page/project/CreateWorkflowPage.java | 89 +++++++++++++++---- .../page/security/TenantManagePage.java | 26 ++++-- .../page/security/UserManagePage.java | 25 ++++-- .../dolphinscheduler/testcase/LoginTest.java | 35 +++----- .../testcase/deleteData/DeleteTenantTest.java | 37 ++++++++ .../testcase/deleteData/DeleteUserTest.java | 36 ++++++++ .../testcase/project/CreateProjectTest.java | 7 +- .../testcase/project/CreateWorkflowTest.java | 10 ++- .../testcase/security/TenantManageTest.java | 11 ++- .../testcase/security/UserManageTest.java | 8 +- e2e/suite.xml | 18 ---- e2e/testng.xml | 28 +++--- 25 files changed, 415 insertions(+), 141 deletions(-) rename e2e/src/test/java/org/apache/dolphinscheduler/data/project/{CreatProjectData.java => CreateProjectData.java} (91%) create mode 100644 e2e/src/test/java/org/apache/dolphinscheduler/data/project/CreateWorkflowData.java create mode 100644 e2e/src/test/java/org/apache/dolphinscheduler/testcase/deleteData/DeleteTenantTest.java create mode 100644 e2e/src/test/java/org/apache/dolphinscheduler/testcase/deleteData/DeleteUserTest.java delete mode 100644 e2e/suite.xml diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/base/BaseDriver.java b/e2e/src/test/java/org/apache/dolphinscheduler/base/BaseDriver.java index 7d3ab9b837..044e28f082 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/base/BaseDriver.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/base/BaseDriver.java @@ -19,6 +19,7 @@ package org.apache.dolphinscheduler.base; import org.apache.dolphinscheduler.constant.TestConstant; import org.apache.dolphinscheduler.util.PropertiesReader; +import org.openqa.selenium.Cookie; import org.openqa.selenium.JavascriptExecutor; import org.openqa.selenium.WebDriver; import org.openqa.selenium.chrome.ChromeDriver; @@ -35,7 +36,7 @@ public class BaseDriver { /** * driver */ - private WebDriver driver; + private static WebDriver driver; /** * chrome driver path @@ -88,6 +89,7 @@ public class BaseDriver { chromeOptions.setPageLoadStrategy(PageLoadStrategy.NONE); chromeOptions.addArguments("--no-sandbox"); chromeOptions.addArguments("--disable-dev-shm-usage"); + //Browser client running requires annotation --headless chromeOptions.addArguments("--headless"); chromeOptions.addArguments("--disable-gpu"); chromeOptions.addArguments("--whitelisted-ips"); @@ -120,7 +122,7 @@ public class BaseDriver { * * @return driver */ - public WebDriver getDriver() { + public static WebDriver getDriver() { return driver; } @@ -141,7 +143,7 @@ public class BaseDriver { public void closeBrowser() throws InterruptedException { // JS Show a pop-up box to indicate the end of the test Thread.sleep(TestConstant.ONE_THOUSANG); - ((JavascriptExecutor) driver).executeScript("alert('Test completed, browser closes after 3s')"); +// ((JavascriptExecutor) driver).executeScript("alert('Test completed, browser closes after 3s')"); Thread.sleep(TestConstant.THREE_THOUSANG); if (driver != null) { driver.quit(); diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/base/BaseTest.java b/e2e/src/test/java/org/apache/dolphinscheduler/base/BaseTest.java index c12c19fc1d..88012e9f42 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/base/BaseTest.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/base/BaseTest.java @@ -17,7 +17,6 @@ package org.apache.dolphinscheduler.base; -import org.apache.dolphinscheduler.page.LoginPage; import org.apache.dolphinscheduler.util.PropertiesReader; import org.openqa.selenium.WebDriver; import org.testng.annotations.*; @@ -28,6 +27,7 @@ import java.util.Properties; /** * base test class */ +@Test(groups={"functionTests"}) public class BaseTest { /** * properties @@ -43,7 +43,7 @@ public class BaseTest { /** * driver */ - public WebDriver driver; + public static WebDriver driver; /** * Executed before executing a test suite  @@ -54,7 +54,7 @@ public class BaseTest { */ @BeforeSuite(alwaysRun = true) @Parameters({"propertiesPath"}) - public void beforeSuite(@Optional("src/test/resources/config/config.properties") String propertiesPath) throws IOException { + public void beforeSuite(@Optional("src/test/resources/config/config.properties") String propertiesPath) throws Exception { // read properties properties = PropertiesReader.readProperties(propertiesPath); } @@ -70,14 +70,13 @@ public class BaseTest { driver = baseDriver.getDriver(); } + + /** * Executed before executing a class method in a test case */ @BeforeClass(alwaysRun = true) - public void setUp() throws IOException, InterruptedException { - LoginPage loginPage = new LoginPage(driver); - loginPage.jumpPage(); - loginPage.login(); + public void setUp() throws Exception { } @@ -85,7 +84,7 @@ public class BaseTest { * Execute after executing a class method in a test case */ @AfterClass(alwaysRun = true) - public void afterClass() { + public void afterClass() throws InterruptedException { // logout } @@ -102,6 +101,6 @@ public class BaseTest { * Execute after executing a testsuite */ @AfterSuite(alwaysRun = true) - public void afterSuite() { + public void afterSuite() throws InterruptedException { } } \ No newline at end of file diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/data/LoginData.java b/e2e/src/test/java/org/apache/dolphinscheduler/data/LoginData.java index 532849565c..e56df5a448 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/data/LoginData.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/data/LoginData.java @@ -39,5 +39,5 @@ public class LoginData { */ public static final String PASSWORD = PropertiesReader.getKey("PASSWORD"); - public static final String TENANT = "Tenant Manage - DolphinScheduler"; + public static final String TENANT = "租户管理 - DolphinScheduler"; } diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/data/project/CreatWorkflowData.java b/e2e/src/test/java/org/apache/dolphinscheduler/data/project/CreatWorkflowData.java index 765a54f406..70f4238610 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/data/project/CreatWorkflowData.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/data/project/CreatWorkflowData.java @@ -18,7 +18,7 @@ package org.apache.dolphinscheduler.data.project; public class CreatWorkflowData { //input shell task name - public static final String SHELL_TASK_NAME = "shell task description test"; + public static final String SHELL_TASK_NAME = "shell task description test1"; //input shell task description public static final String SHELL_TASK_DESCRIPTION = "shell task description test"; diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/data/project/CreatProjectData.java b/e2e/src/test/java/org/apache/dolphinscheduler/data/project/CreateProjectData.java similarity index 91% rename from e2e/src/test/java/org/apache/dolphinscheduler/data/project/CreatProjectData.java rename to e2e/src/test/java/org/apache/dolphinscheduler/data/project/CreateProjectData.java index 8f6c9c8e97..57d79e87d7 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/data/project/CreatProjectData.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/data/project/CreateProjectData.java @@ -16,9 +16,9 @@ */ package org.apache.dolphinscheduler.data.project; -public class CreatProjectData { +public class CreateProjectData { // create project name - public static final String PROJECT_NAME = "selenium_project"; + public static final String PROJECT_NAME = "selenium_project_3"; // create project description public static final String DESCRIPTION = "test create project description"; // project page title diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/data/project/CreateWorkflowData.java b/e2e/src/test/java/org/apache/dolphinscheduler/data/project/CreateWorkflowData.java new file mode 100644 index 0000000000..31b5bdd1e0 --- /dev/null +++ b/e2e/src/test/java/org/apache/dolphinscheduler/data/project/CreateWorkflowData.java @@ -0,0 +1,73 @@ +/* + * 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.data.project; + +public class CreateWorkflowData { + /** + * create workflow data + */ + //input shell task name + public static final String SHELL_TASK_NAME = "shell_task_selenium_5"; + + //input shell task description + public static final String SHELL_TASK_DESCRIPTION = "shell task description test"; + + //input timeout + public static final String INPUT_TIMEOUT = "60"; + + //input shell script + public static final String SHELL_SCRIPT = "echo 1111111"; + + //input custom parameters + public static final String INPUT_CUSTOM_PARAMETERS = "selenium_parameter"; + + //input custom parameters value + public static final String INPUT_CUSTOM_PARAMETERS_VALUE = "selenium_parameter_123"; + + //input add custom parameters + public static final String INPUT_ADD_CUSTOM_PARAMETERS = "selenium_parameter_delete"; + + //input add custom parameters value + public static final String INPUT_ADD_CUSTOM_PARAMETERS_VALUE = "selenium_parameter_delete_456"; + + //create workflow title + public static final String WORKFLOW_TITLE = "创建流程定义 - DolphinScheduler"; + + /** + * save workflow data + */ + //input workflow name + public static final String INPUT_WORKFLOW_NAME = "selenium_shell_1"; + + //input workflow description + public static final String INPUT_WORKFLOW_DESCRIPTION = "test selenium_shell_1 description"; + + //input workflow timeout + public static final String INPUT_WORKFLOW_TIMEOUT = "30"; + + //input workflow global parameters + public static final String INPUT_WORKFLOW_GLOBAL_PARAMETERS = "selenium_global_parameters_1"; + + //input workflow global parameters value + public static final String INPUT_WORKFLOW_GLOBAL_PARAMETERS_VALUES = "selenium_global_parameters_value_1"; + + //input to add workflow global parameters + public static final String INPUT_ADD_WORKFLOW_GLOBAL_PARAMETERS = "selenium_global_parameters_2"; + + //input to add workflow global parameters value + public static final String INPUT_ADD_WORKFLOW_GLOBAL_PARAMETERS_VALUES = "selenium_global_parameters_value_2"; +} diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/data/security/TenantManageData.java b/e2e/src/test/java/org/apache/dolphinscheduler/data/security/TenantManageData.java index e6f6ee6b86..100351b600 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/data/security/TenantManageData.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/data/security/TenantManageData.java @@ -20,18 +20,12 @@ */ package org.apache.dolphinscheduler.data.security; -import org.apache.dolphinscheduler.data.LoginData; public class TenantManageData { - /** - * Tenant URL - */ - public static final String TENANAT_URL = LoginData.URL + "/ui/#/security/tenant"; - /** * Tenant Code */ - public static final String TENANAT_CODE = "dolphinscheduler_tenant_code2"; + public static final String TENANAT_CODE = "dolphinscheduler_tenant_code15"; /** * Tenant Name @@ -48,7 +42,7 @@ public class TenantManageData { */ public static final String DESCRIPTION = "creat tenant test"; - public static final String TENANAT_MANAGE = "Tenant Manage - DolphinScheduler"; + public static final String TENANAT_MANAGE = "租户管理 - DolphinScheduler"; diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/data/security/UserManageData.java b/e2e/src/test/java/org/apache/dolphinscheduler/data/security/UserManageData.java index 03c985fd81..6fcd923c53 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/data/security/UserManageData.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/data/security/UserManageData.java @@ -16,12 +16,9 @@ */ package org.apache.dolphinscheduler.data.security; -import org.apache.dolphinscheduler.data.LoginData; public class UserManageData { - public static final String USER_URL = LoginData.URL + "/ui/#/security/users"; - - public static final String USERNAME = "selenium111"; + public static final String USERNAME = "selenium_5"; public static final String PASSWORD = "123456qwe"; diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/locator/project/CreateWorkflowLocator.java b/e2e/src/test/java/org/apache/dolphinscheduler/locator/project/CreateWorkflowLocator.java index f063d6ef61..25defcf53f 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/locator/project/CreateWorkflowLocator.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/locator/project/CreateWorkflowLocator.java @@ -20,12 +20,20 @@ package org.apache.dolphinscheduler.locator.project; import org.openqa.selenium.By; public class CreateWorkflowLocator { + + /** + * jump workflow define page + */ + // click project name public static final By CLICK_PROJECT_NAME = By.xpath("//span/a"); // click workflow define public static final By CLICK_WORKFLOW_DEFINE = By.xpath("//li/span"); + /** + * create workflow + */ // click create workflow button public static final By CLICK_CREATE_WORKFLOW_BUTTON = By.xpath("//button/span"); @@ -96,15 +104,15 @@ public class CreateWorkflowLocator { public static final By INPUT_CUSTOM_PARAMETERS = By.xpath("//div[2]/div/div/div/div/div/input"); //input custom parameters value - public static final By INPUT_CUSTOM_PARAMETERS_VALUE = By.xpath("//div[2]/input"); + public static final By INPUT_CUSTOM_PARAMETERS_VALUE = By.xpath("//div[2]/div/div/div/div[1]/div[2]/input"); - //click add custom parameters - public static final By CLICK_ADD_CUSTOM_PARAMETERS = By.xpath("//span[2]/a/em"); + //click to add custom parameters + public static final By CLICK_ADD_CUSTOM_PARAMETERS = By.xpath("//div[2]/div/div/div/div/span[2]/a/em"); - //input add custom parameters - public static final By INPUT_ADD_CUSTOM_PARAMETERS = By.xpath("//div[2]/div/div/div/div[2]/div/input"); + //input to add custom parameters + public static final By INPUT_ADD_CUSTOM_PARAMETERS = By.xpath("//div[2]/div/div/div/div[2]/div[1]/input"); - //input add custom parameters value + //input to add custom parameters value public static final By INPUT_ADD_CUSTOM_PARAMETERS_VALUE = By.xpath("//div[2]/div[2]/input"); //delete custom parameters @@ -112,4 +120,53 @@ public class CreateWorkflowLocator { //click submit button public static final By CLICK_SUBMIT_BUTTON = By.xpath("//button[2]/span"); + + + /** + * save workflow + */ + //click save workflow button + public static final By CLICK_SAVE_WORKFLOW_BUTTON = By.xpath("//button/span"); + + //input workflow name + public static final By INPUT_WORKFLOW_NAME = By.xpath("//input"); + + //input workflow description + public static final By INPUT_WORKFLOW_DESCRIPTION = By.xpath("//textarea"); + + //select tenant + public static final By CLICK_TENANT = By.xpath("//div[4]/div/div/div/input"); + public static final By SELECT_TENANT = By.xpath("//div[2]/div/div/div/ul/li/span"); + + + //click workflow timeout alarm + public static final By CLICK_WORKFLOW_TIMEOUT_ALARM = By.xpath("//span[2]/span/span"); + + //input workflow timeout + public static final By INPUT_WORKFLOW_TIMEOUT = By.xpath("//span/div/input"); + + //click workflow global parameters + public static final By CLICK_WORKFLOW_GLOBAL_PARAMETERS = By.xpath("//span/a/em"); + + //input workflow global parameters + public static final By INPUT_WORKFLOW_GLOBAL_PARAMETERS = By.xpath("//div[8]/div/div/div/div/input"); + + //input workflow global parameters value + public static final By INPUT_WORKFLOW_GLOBAL_PARAMETERS_VALUES = By.xpath("//div[2]/input"); + + //click to add workflow global parameters + public static final By CLICK_ADD_WORKFLOW_GLOBAL_PARAMETERS = By.xpath("//span[2]/a/em"); + + //input to add workflow global parameters + public static final By INPUT_ADD_WORKFLOW_GLOBAL_PARAMETERS = By.xpath("//div[8]/div/div/div[2]/div/input"); + + //input to add workflow global parameters value + public static final By INPUT_ADD_WORKFLOW_GLOBAL_PARAMETERS_VALUES = By.xpath("//div[2]/div[2]/input"); + + //delete workflow global parameters value + public static final By CLICK_DELETE_WORKFLOW_GLOBAL_PARAMETERS = By.xpath("//div[2]/span/a/em"); + + //click add button + public static final By CLICK_ADD_BUTTON = By.xpath("//button[2]/span"); + } diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/locator/security/TenantManageLocator.java b/e2e/src/test/java/org/apache/dolphinscheduler/locator/security/TenantManageLocator.java index 7d9c8a57ce..d95265cf0b 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/locator/security/TenantManageLocator.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/locator/security/TenantManageLocator.java @@ -19,7 +19,9 @@ package org.apache.dolphinscheduler.locator.security; import org.openqa.selenium.By; public class TenantManageLocator{ - public static final By CREATE_TENANT_BUTTON = By.xpath("//button[@class='ans-btn ans-btn-ghost ans-btn-small']"); + public static final By TENANT_MANAGE = By.xpath("//div[2]/div/a/div/a/span"); + + public static final By CREATE_TENANT_BUTTON = By.xpath("//button/span"); public static final By TENANT_INPUT_CODE = By.xpath("//div[2]/div/div/div[2]/div/input"); @@ -30,4 +32,8 @@ public class TenantManageLocator{ public static final By DESCRIPTION = By.xpath("//textarea"); public static final By SUBMIT_BUTTON = By.xpath("//div[3]/button[2]/span"); + + public static final By DELETE_TENANT_BUTTON = By.xpath("//div[3]/div[1]/div/table/tr[2]/td[8]/span/button"); + + public static final By CONFIRM_DELETE_TENANT_BUTTON = By.xpath("//div[2]/div/button[2]/span"); } diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/locator/security/UserManageLocator.java b/e2e/src/test/java/org/apache/dolphinscheduler/locator/security/UserManageLocator.java index 0d84692cb2..b5ecccc67d 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/locator/security/UserManageLocator.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/locator/security/UserManageLocator.java @@ -42,4 +42,7 @@ public class UserManageLocator { public static final By SUBMIT = By.xpath("//div[3]/button[2]/span"); + public static final By DELETE_USER_BUTTON = By.xpath("//span[2]/button/i"); + + public static final By CONFIRM_DELETE_USER_BUTTON = By.xpath("//div[2]/div/button[2]/span"); } diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/page/LoginPage.java b/e2e/src/test/java/org/apache/dolphinscheduler/page/LoginPage.java index cd6b318651..14970bead5 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/page/LoginPage.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/page/LoginPage.java @@ -21,6 +21,7 @@ import org.apache.dolphinscheduler.constant.TestConstant; import org.apache.dolphinscheduler.data.LoginData; import org.apache.dolphinscheduler.locator.LoginLocator; import org.apache.dolphinscheduler.util.RedisUtil; +import org.openqa.selenium.Cookie; import org.openqa.selenium.WebDriver; @@ -38,18 +39,25 @@ public class LoginPage extends PageCommon { /** * jump page */ - public void jumpPage() { - System.out.println("jump login page"); + public void jumpPageEnlish() { + System.out.println("jump to English login page"); super.jumpPage(LoginData.URL); + Cookie cookie = new Cookie("language", "en_US", "/", null); + driver.manage().addCookie(cookie); } + public void jumpPageChinese() { + super.jumpPage(LoginData.URL); + Cookie cookie = new Cookie("language", "zh_CN", "/", null); + + driver.manage().addCookie(cookie); + } /** * login * * @return Whether to enter the specified page after searching */ public boolean login() throws InterruptedException { - System.out.println("LoginPage"); // login data sendInput(LoginLocator.LOGIN_INPUT_USER, LoginData.USER); sendInput(LoginLocator.LOGIN_INPUT_PASSWORD, LoginData.PASSWORD); diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/page/project/CreateProjectPage.java b/e2e/src/test/java/org/apache/dolphinscheduler/page/project/CreateProjectPage.java index 8dd1010a82..d8e1ef655d 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/page/project/CreateProjectPage.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/page/project/CreateProjectPage.java @@ -18,7 +18,7 @@ package org.apache.dolphinscheduler.page.project; import org.apache.dolphinscheduler.common.PageCommon; import org.apache.dolphinscheduler.constant.TestConstant; -import org.apache.dolphinscheduler.data.project.CreatProjectData; +import org.apache.dolphinscheduler.data.project.CreateProjectData; import org.apache.dolphinscheduler.locator.project.CreateProjectLocator; import org.openqa.selenium.WebDriver; @@ -26,6 +26,7 @@ public class CreateProjectPage extends PageCommon { public CreateProjectPage(WebDriver driver) { super(driver); } + /** * jump page */ @@ -36,7 +37,7 @@ public class CreateProjectPage extends PageCommon { } /** - * creatTenant + * createTenant * * @return Whether to enter the specified page after creat tenant */ @@ -46,13 +47,13 @@ public class CreateProjectPage extends PageCommon { Thread.sleep(TestConstant.ONE_THOUSANG); // input create project data - sendInput(CreateProjectLocator.PROJECT_NAME, CreatProjectData.PROJECT_NAME); - sendInput(CreateProjectLocator.PROJECT_DESCRIPTION, CreatProjectData.DESCRIPTION); + sendInput(CreateProjectLocator.PROJECT_NAME, CreateProjectData.PROJECT_NAME); + sendInput(CreateProjectLocator.PROJECT_DESCRIPTION, CreateProjectData.DESCRIPTION); // click submit button clickButton(CreateProjectLocator.SUBMIT_BUTTON); // Whether to enter the specified page after submit - return ifTitleContains(CreatProjectData.PROJECT_TITLE); + return ifTitleContains(CreateProjectData.PROJECT_TITLE); } } diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/page/project/CreateWorkflowPage.java b/e2e/src/test/java/org/apache/dolphinscheduler/page/project/CreateWorkflowPage.java index aeec83cd19..d115124d8f 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/page/project/CreateWorkflowPage.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/page/project/CreateWorkflowPage.java @@ -18,7 +18,7 @@ package org.apache.dolphinscheduler.page.project; import org.apache.dolphinscheduler.common.PageCommon; import org.apache.dolphinscheduler.constant.TestConstant; -import org.apache.dolphinscheduler.data.project.CreatWorkflowData; +import org.apache.dolphinscheduler.data.project.CreateWorkflowData; import org.apache.dolphinscheduler.locator.project.CreateWorkflowLocator; import org.openqa.selenium.WebDriver; @@ -26,29 +26,33 @@ public class CreateWorkflowPage extends PageCommon { public CreateWorkflowPage(WebDriver driver) { super(driver); } + /** - * jump page + * jump create workflow page */ - public boolean createWorkflow() throws InterruptedException { + System.out.println("Click on the project name to jump to the project homepage"); // click project name clickElement(CreateWorkflowLocator.CLICK_PROJECT_NAME); Thread.sleep(TestConstant.ONE_THOUSANG); + System.out.println("Click on workflow define"); // click workflow define clickElement(CreateWorkflowLocator.CLICK_WORKFLOW_DEFINE); Thread.sleep(TestConstant.ONE_THOUSANG); + System.out.println("Click create workflow button"); // click create workflow button clickElement(CreateWorkflowLocator.CLICK_CREATE_WORKFLOW_BUTTON); Thread.sleep(TestConstant.ONE_THOUSANG); + System.out.println("drag shell task"); //drag shell_task dragAndDrop(CreateWorkflowLocator.MOUSE_DOWN_AT_SHELL,CreateWorkflowLocator.MOUSE_MOVE_SHELL_AT_DAG); //input shell task _name - sendInput(CreateWorkflowLocator.INPUT_SHELL_TASK_NAME , CreatWorkflowData.SHELL_TASK_NAME); + sendInput(CreateWorkflowLocator.INPUT_SHELL_TASK_NAME , CreateWorkflowData.SHELL_TASK_NAME); //click stop run type clickElement(CreateWorkflowLocator.CLICK_STOP_RUN_TYPE); @@ -57,7 +61,7 @@ public class CreateWorkflowPage extends PageCommon { clickElement(CreateWorkflowLocator.CLICK_NORMAL_RUN_TYPE); //input shell task description - sendInput(CreateWorkflowLocator.INPUT_SHELL_TASK_DESCRIPTION , CreatWorkflowData.SHELL_TASK_DESCRIPTION); + sendInput(CreateWorkflowLocator.INPUT_SHELL_TASK_DESCRIPTION , CreateWorkflowData.SHELL_TASK_DESCRIPTION); //select task priority clickElement(CreateWorkflowLocator.CLICK_TASK_PRIORITY); @@ -73,19 +77,15 @@ public class CreateWorkflowPage extends PageCommon { //select failed retry interval clickElement(CreateWorkflowLocator.SELECT_FAIL_RETRIES_INTERVAL); - //click timeout alarm clickElement(CreateWorkflowLocator.CLICK_TIMEOUT_ALARM); - //select timeout fail clickElement(CreateWorkflowLocator.SELECT_TIMEOUT_FAIL); - //cancel timeout alarm clickElement(CreateWorkflowLocator.CANCEL_TIMEOUT_ALARM); - //select timeout alarm clickElement(CreateWorkflowLocator.SELECT_TIMEOUT_ALARM); @@ -94,10 +94,10 @@ public class CreateWorkflowPage extends PageCommon { clearInput(CreateWorkflowLocator.SELECT_TIMEOUT); //input timeout - sendInput(CreateWorkflowLocator.SELECT_TIMEOUT,CreatWorkflowData.INPUT_TIMEOUT); + sendInput(CreateWorkflowLocator.SELECT_TIMEOUT, CreateWorkflowData.INPUT_TIMEOUT); //click codeMirror and input script - inputCodeMirror(CreateWorkflowLocator.CLICK_CODE_MIRROR, CreateWorkflowLocator.INPUT_SCRIPT,CreatWorkflowData.SHELL_SCRIPT); + inputCodeMirror(CreateWorkflowLocator.CLICK_CODE_MIRROR, CreateWorkflowLocator.INPUT_SCRIPT, CreateWorkflowData.SHELL_SCRIPT); scrollToElementBottom(); Thread.sleep(TestConstant.ONE_THOUSANG); @@ -105,10 +105,10 @@ public class CreateWorkflowPage extends PageCommon { clickElement(CreateWorkflowLocator.CLICK_CUSTOM_PARAMETERS); //input custom parameters - sendInput(CreateWorkflowLocator.INPUT_CUSTOM_PARAMETERS, CreatWorkflowData.INPUT_CUSTOM_PARAMETERS); + sendInput(CreateWorkflowLocator.INPUT_CUSTOM_PARAMETERS, CreateWorkflowData.INPUT_CUSTOM_PARAMETERS); //input custom parameters value - sendInput(CreateWorkflowLocator.INPUT_CUSTOM_PARAMETERS_VALUE, CreatWorkflowData.INPUT_CUSTOM_PARAMETERS_VALUE); + sendInput(CreateWorkflowLocator.INPUT_CUSTOM_PARAMETERS_VALUE, CreateWorkflowData.INPUT_CUSTOM_PARAMETERS_VALUE); //click add custom parameters clickElement(CreateWorkflowLocator.CLICK_ADD_CUSTOM_PARAMETERS); @@ -117,10 +117,10 @@ public class CreateWorkflowPage extends PageCommon { Thread.sleep(TestConstant.ONE_THOUSANG); //input add custom parameters - sendInput(CreateWorkflowLocator.INPUT_ADD_CUSTOM_PARAMETERS,CreatWorkflowData.INPUT_ADD_CUSTOM_PARAMETERS); + sendInput(CreateWorkflowLocator.INPUT_ADD_CUSTOM_PARAMETERS, CreateWorkflowData.INPUT_ADD_CUSTOM_PARAMETERS); //input add custom parameters value - sendInput(CreateWorkflowLocator.INPUT_ADD_CUSTOM_PARAMETERS_VALUE,CreatWorkflowData.INPUT_ADD_CUSTOM_PARAMETERS_VALUE); + sendInput(CreateWorkflowLocator.INPUT_ADD_CUSTOM_PARAMETERS_VALUE, CreateWorkflowData.INPUT_ADD_CUSTOM_PARAMETERS_VALUE); //click delete custom parameters clickElement(CreateWorkflowLocator.CLICK_DELETE_CUSTOM_PARAMETERS); @@ -129,9 +129,64 @@ public class CreateWorkflowPage extends PageCommon { //click submit button clickElement(CreateWorkflowLocator.CLICK_SUBMIT_BUTTON); Thread.sleep(TestConstant.ONE_THOUSANG); - + System.out.println("Task node set up successfully"); + System.out.println("move to Dag Element "); moveToDragElement(CreateWorkflowLocator.MOUSE_MOVE_SHELL_AT_DAG,-300,-100); - return ifTitleContains(CreatWorkflowData.WORKFLOW_TITLE); + return ifTitleContains(CreateWorkflowData.WORKFLOW_TITLE); + } + + /** + * save workflow + */ + public boolean saveWorkflow() throws InterruptedException { + System.out.println("start to save workflow "); + + //click save workflow button + clickElement(CreateWorkflowLocator.CLICK_SAVE_WORKFLOW_BUTTON); + + //input workflow name + sendInput(CreateWorkflowLocator.INPUT_WORKFLOW_NAME, CreateWorkflowData.INPUT_WORKFLOW_NAME); + + //input workflow description + sendInput(CreateWorkflowLocator.INPUT_WORKFLOW_DESCRIPTION, CreateWorkflowData.INPUT_WORKFLOW_DESCRIPTION); + + //select tenant + clickElement(CreateWorkflowLocator.CLICK_TENANT); + clickElement(CreateWorkflowLocator.SELECT_TENANT); + + //click workflow timeout alarm + clickElement(CreateWorkflowLocator.CLICK_WORKFLOW_TIMEOUT_ALARM); + clearInput(CreateWorkflowLocator.INPUT_WORKFLOW_TIMEOUT); + + //input workflow timeout + sendInput(CreateWorkflowLocator.INPUT_WORKFLOW_TIMEOUT, CreateWorkflowData.INPUT_WORKFLOW_TIMEOUT); + + //click workflow global parameters + clickElement(CreateWorkflowLocator.CLICK_WORKFLOW_GLOBAL_PARAMETERS); + + //input workflow global parameters + sendInput(CreateWorkflowLocator.INPUT_WORKFLOW_GLOBAL_PARAMETERS, CreateWorkflowData.INPUT_WORKFLOW_GLOBAL_PARAMETERS); + + //input workflow global parameters value + sendInput(CreateWorkflowLocator.INPUT_WORKFLOW_GLOBAL_PARAMETERS_VALUES, CreateWorkflowData.INPUT_WORKFLOW_GLOBAL_PARAMETERS_VALUES); + + //click to add workflow global parameters + clickElement(CreateWorkflowLocator.CLICK_ADD_WORKFLOW_GLOBAL_PARAMETERS); + + //input to add workflow global parameters + sendInput(CreateWorkflowLocator.INPUT_ADD_WORKFLOW_GLOBAL_PARAMETERS, CreateWorkflowData.INPUT_ADD_WORKFLOW_GLOBAL_PARAMETERS); + + //input to add workflow global parameters value + sendInput(CreateWorkflowLocator.INPUT_ADD_WORKFLOW_GLOBAL_PARAMETERS_VALUES, CreateWorkflowData.INPUT_ADD_WORKFLOW_GLOBAL_PARAMETERS_VALUES); + + //delete workflow global parameters value + clickElement(CreateWorkflowLocator.CLICK_DELETE_WORKFLOW_GLOBAL_PARAMETERS); + Thread.sleep(TestConstant.ONE_THOUSANG); + + //click add button + clickElement(CreateWorkflowLocator.CLICK_ADD_BUTTON); + System.out.println("submit workflow"); + return ifTitleContains(CreateWorkflowData.WORKFLOW_TITLE); } } diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/page/security/TenantManagePage.java b/e2e/src/test/java/org/apache/dolphinscheduler/page/security/TenantManagePage.java index 4c88f6575e..618e1bc213 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/page/security/TenantManagePage.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/page/security/TenantManagePage.java @@ -34,15 +34,6 @@ public class TenantManagePage extends PageCommon { super(driver); } - - /** - * jump page - */ - public void jumpPage() { - System.out.println("jump tenant page"); - super.jumpPage(TenantManageData.TENANAT_URL); - } - /** * createTenant * @@ -50,6 +41,7 @@ public class TenantManagePage extends PageCommon { */ public boolean createTenant() throws InterruptedException { Thread.sleep(TestConstant.ONE_THOUSANG); + clickButton(TenantManageLocator.TENANT_MANAGE); //create tenant clickButton(TenantManageLocator.CREATE_TENANT_BUTTON); @@ -66,4 +58,20 @@ public class TenantManagePage extends PageCommon { // Whether to enter the specified page after submit return ifTitleContains(TenantManageData.TENANAT_MANAGE); } + + public boolean deleteTenant() throws InterruptedException { + Thread.sleep(TestConstant.ONE_THOUSANG); + clickButton(TenantManageLocator.TENANT_MANAGE); + Thread.sleep(TestConstant.ONE_THOUSANG); + + // click delete button + clickButton(TenantManageLocator.DELETE_TENANT_BUTTON); + Thread.sleep(TestConstant.ONE_THOUSANG); + + //click confirm delete button + clickButton(TenantManageLocator.CONFIRM_DELETE_TENANT_BUTTON); + + // Whether to enter the specified page after submit + return ifTitleContains(TenantManageData.TENANAT_MANAGE); + } } diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/page/security/UserManagePage.java b/e2e/src/test/java/org/apache/dolphinscheduler/page/security/UserManagePage.java index 3e2be2d126..70311f129e 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/page/security/UserManagePage.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/page/security/UserManagePage.java @@ -26,16 +26,9 @@ public class UserManagePage extends PageCommon { public UserManagePage(WebDriver driver) { super(driver); } - /** - * jump page - */ - public void jumpPage() { - System.out.println("jump tenant page"); - super.jumpPage(UserManageData.USER_URL); - } /** - * creatTenant + * createTenant * * @return Whether to enter the specified page after creat tenant */ @@ -64,4 +57,20 @@ public class UserManagePage extends PageCommon { // Whether to enter the specified page after submit return ifTitleContains(UserManageData.USER_MANAGE); } + + public boolean deleteUser() throws InterruptedException { + Thread.sleep(TestConstant.ONE_THOUSANG); + // click user manage + clickElement(UserManageLocator.CLICK_USER_MANAGE); + Thread.sleep(TestConstant.ONE_THOUSANG); + + // click delete user button + clickButton(UserManageLocator.DELETE_USER_BUTTON ); + + // click confirm delete button + clickButton(UserManageLocator.CONFIRM_DELETE_USER_BUTTON); + + // Whether to enter the specified page after submit + return ifTitleContains(UserManageData.USER_MANAGE); + } } diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/testcase/LoginTest.java b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/LoginTest.java index bd3c31d38b..47ce11287d 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/testcase/LoginTest.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/LoginTest.java @@ -1,37 +1,26 @@ -/* - * 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.testcase; -import org.apache.dolphinscheduler.base.BaseTest; import org.apache.dolphinscheduler.page.LoginPage; +import org.testng.Assert; import org.testng.annotations.Test; +import static org.apache.dolphinscheduler.base.BaseTest.driver; -public class LoginTest extends BaseTest { +@Test(groups={"functionTests","login"}) +public class LoginTest { private LoginPage loginPage; + @Test(description = "LoginTest", priority = 1) public void testLogin() throws InterruptedException { - // init login page loginPage = new LoginPage(driver); + System.out.println("==================================="); + System.out.println("jump to Chinese login page"); + loginPage.jumpPageChinese(); - // enter login page - loginPage.jumpPage(); + System.out.println("start login"); + assert loginPage.login(); + System.out.println("end login"); + System.out.println("==================================="); - //assert login page - assert loginPage.login(); } } diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/testcase/deleteData/DeleteTenantTest.java b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/deleteData/DeleteTenantTest.java new file mode 100644 index 0000000000..ddbc9d603b --- /dev/null +++ b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/deleteData/DeleteTenantTest.java @@ -0,0 +1,37 @@ +/* + * 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.testcase.deleteData; + +import org.apache.dolphinscheduler.base.BaseTest; +import org.apache.dolphinscheduler.page.security.TenantManagePage; +import org.testng.annotations.Test; + +public class DeleteTenantTest extends BaseTest { + private TenantManagePage tenantManagePage; + + @Test(groups={"functionTests"},dependsOnGroups = { "login","createTenant"},description = "DeleteTenantTest") + public void testDeleteTenant() throws InterruptedException { + tenantManagePage = new TenantManagePage(driver); + //assert tenant manage page + System.out.println("start delete tenant"); + assert tenantManagePage.deleteTenant(); + System.out.println("end delete tenant"); + System.out.println("==================================="); + } + + +} diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/testcase/deleteData/DeleteUserTest.java b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/deleteData/DeleteUserTest.java new file mode 100644 index 0000000000..770fae077d --- /dev/null +++ b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/deleteData/DeleteUserTest.java @@ -0,0 +1,36 @@ +/* + * 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.testcase.deleteData; + +import org.apache.dolphinscheduler.base.BaseTest; +import org.apache.dolphinscheduler.page.security.UserManagePage; +import org.testng.annotations.Test; + +public class DeleteUserTest extends BaseTest { + private UserManagePage userManagePage; + + @Test(groups={"functionTests"},dependsOnGroups = { "login","user" },description = "DeleteUserTest") + public void testDeleteUser() throws InterruptedException { + userManagePage = new UserManagePage(driver); + //assert user manage page + System.out.println("start delete user"); + assert userManagePage.deleteUser(); + System.out.println("end delete user"); + System.out.println("==================================="); + } +} + diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/testcase/project/CreateProjectTest.java b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/project/CreateProjectTest.java index 8abd09d37c..546792f0d8 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/testcase/project/CreateProjectTest.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/project/CreateProjectTest.java @@ -23,13 +23,16 @@ import org.testng.annotations.Test; public class CreateProjectTest extends BaseTest { private CreateProjectPage createProjectPage; - @Test(description = "TenantTest", priority = 1) - public void testUserManage() throws InterruptedException { + @Test(groups={"functionTests"},dependsOnGroups = { "login" },description = "CreateProjectTest",priority=4) + public void testCreateProject() throws InterruptedException { createProjectPage = new CreateProjectPage(driver); // enter user manage page createProjectPage.jumpProjectManagePage(); //assert user manage page + System.out.println("start create project"); assert createProjectPage.createProject(); + System.out.println("end create project"); + System.out.println("==================================="); } } diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/testcase/project/CreateWorkflowTest.java b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/project/CreateWorkflowTest.java index 6ac13f8124..ea87f48529 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/testcase/project/CreateWorkflowTest.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/project/CreateWorkflowTest.java @@ -26,12 +26,16 @@ public class CreateWorkflowTest extends BaseTest { private CreateProjectPage createProjectPage; - @Test(description = "CreateWorkflowTest", priority = 1) - public void CreateWorkflowTest() throws InterruptedException { + @Test(groups={"functionTests"},dependsOnGroups = { "login" },description = "CreateWorkflowTest",priority=5) + public void testCreateWorkflow() throws InterruptedException { createProjectPage = new CreateProjectPage(driver); createProjectPage.jumpProjectManagePage(); + createWorkflowPage = new CreateWorkflowPage(driver); - //assert create workflow + System.out.println("start create workflow"); assert createWorkflowPage.createWorkflow(); + assert createWorkflowPage.saveWorkflow(); + System.out.println("end create workflow"); + System.out.println("==================================="); } } diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/testcase/security/TenantManageTest.java b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/security/TenantManageTest.java index 7124b4e094..8b21d3bb61 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/testcase/security/TenantManageTest.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/security/TenantManageTest.java @@ -20,15 +20,20 @@ import org.apache.dolphinscheduler.base.BaseTest; import org.apache.dolphinscheduler.page.security.TenantManagePage; import org.testng.annotations.Test; + public class TenantManageTest extends BaseTest { private TenantManagePage tenantManagePage; - @Test(description = "TenantTest", priority = 1) + @Test(groups={"functionTests","createTenant"},dependsOnGroups = { "login" },description = "TenantManageTest") public void testTenantManage() throws InterruptedException { tenantManagePage = new TenantManagePage(driver); - // enter tenant manage page - tenantManagePage.jumpPage(); //assert tenant manage page + System.out.println("start create tenant"); assert tenantManagePage.createTenant(); + System.out.println("end create tenant"); + System.out.println("==================================="); + } + + } diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/testcase/security/UserManageTest.java b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/security/UserManageTest.java index 834ebdbbf0..9381b68c75 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/testcase/security/UserManageTest.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/security/UserManageTest.java @@ -23,12 +23,14 @@ import org.testng.annotations.Test; public class UserManageTest extends BaseTest { private UserManagePage userManagePage; - @Test(description = "TenantTest", priority = 1) + @Test(groups={"functionTests","user"},dependsOnGroups = { "login" },description = "UserManageTest") public void testUserManage() throws InterruptedException { userManagePage = new UserManagePage(driver); - // enter user manage page - userManagePage.jumpPage(); //assert user manage page + System.out.println("start create user"); assert userManagePage.createUser(); + System.out.println("end create user"); + System.out.println("==================================="); + } } diff --git a/e2e/suite.xml b/e2e/suite.xml deleted file mode 100644 index d9d7ae6845..0000000000 --- a/e2e/suite.xml +++ /dev/null @@ -1,18 +0,0 @@ - - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/e2e/testng.xml b/e2e/testng.xml index 757ffab248..56c9649135 100644 --- a/e2e/testng.xml +++ b/e2e/testng.xml @@ -1,19 +1,23 @@ - - - - - - + + + + + + + + - + + + + + + + - - - - - + From 7789e05c20ed16422eb58971efbbc3dddf89f746 Mon Sep 17 00:00:00 2001 From: xingchun-chen <55787491+xingchun-chen@users.noreply.github.com> Date: Wed, 18 Mar 2020 10:03:03 +0800 Subject: [PATCH 110/221] add LoginTest license (#2217) Co-authored-by: chenxingchun <438044805@qq.com> --- .../dolphinscheduler/testcase/LoginTest.java | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/e2e/src/test/java/org/apache/dolphinscheduler/testcase/LoginTest.java b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/LoginTest.java index 47ce11287d..024ffcdc8a 100644 --- a/e2e/src/test/java/org/apache/dolphinscheduler/testcase/LoginTest.java +++ b/e2e/src/test/java/org/apache/dolphinscheduler/testcase/LoginTest.java @@ -1,3 +1,19 @@ +/* + * 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.testcase; import org.apache.dolphinscheduler.page.LoginPage; From e388e343db58ff11f9a47f0d869e865d0f993f12 Mon Sep 17 00:00:00 2001 From: break60 <790061044@qq.com> Date: Wed, 18 Mar 2020 11:26:11 +0800 Subject: [PATCH 111/221] dd branch flow node verification --- .../js/conf/home/pages/dag/_source/dag.scss | 2 +- .../js/conf/home/pages/dag/_source/dag.vue | 91 +++++++++++-------- .../pages/dag/_source/formModel/formModel.vue | 2 +- .../formModel/tasks/_source/nodeStatus.vue | 4 - .../src/js/module/i18n/locale/en_US.js | 3 +- .../src/js/module/i18n/locale/zh_CN.js | 3 +- 6 files changed, 60 insertions(+), 45 deletions(-) diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.scss b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.scss index 6d97856960..9973750d98 100755 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.scss +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.scss @@ -135,7 +135,7 @@ width: 36px; height: 36px; float: left; - margin-bottom: 11px; + margin-bottom: 8px; border-radius: 3px; .disabled { .icos { diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.vue index a1ccd39260..8628fdb8ef 100755 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/dag.vue @@ -326,45 +326,62 @@ * Storage interface */ _save (sourceType) { - return new Promise((resolve, reject) => { - this.spinnerLoading = true - // Storage store - Dag.saveStore().then(res => { - if (this.urlParam.id) { - /** - * Edit - * @param saveInstanceEditDAGChart => Process instance editing - * @param saveEditDAGChart => Process definition editing - */ - this[this.type === 'instance' ? 'updateInstance' : 'updateDefinition'](this.urlParam.id).then(res => { - this.$message.success(res.msg) - this.spinnerLoading = false - resolve() - }).catch(e => { - this.$message.error(e.msg || '') - this.spinnerLoading = false - reject(e) - }) - } else { - // New - this.saveDAGchart().then(res => { - this.$message.success(res.msg) - this.spinnerLoading = false - // source @/conf/home/pages/dag/_source/editAffirmModel/index.js - if (sourceType !== 'affirm') { - // Jump process definition - this.$router.push({ name: 'projects-definition-list' }) - } - resolve() - }).catch(e => { - this.$message.error(e.msg || '') - this.setName('') - this.spinnerLoading = false - reject(e) - }) - } + if(this._verifConditions()) { + return new Promise((resolve, reject) => { + this.spinnerLoading = true + // Storage store + Dag.saveStore().then(res => { + if (this.urlParam.id) { + /** + * Edit + * @param saveInstanceEditDAGChart => Process instance editing + * @param saveEditDAGChart => Process definition editing + */ + this[this.type === 'instance' ? 'updateInstance' : 'updateDefinition'](this.urlParam.id).then(res => { + this.$message.success(res.msg) + this.spinnerLoading = false + resolve() + }).catch(e => { + this.$message.error(e.msg || '') + this.spinnerLoading = false + reject(e) + }) + } else { + // New + this.saveDAGchart().then(res => { + this.$message.success(res.msg) + this.spinnerLoading = false + // source @/conf/home/pages/dag/_source/editAffirmModel/index.js + if (sourceType !== 'affirm') { + // Jump process definition + this.$router.push({ name: 'projects-definition-list' }) + } + resolve() + }).catch(e => { + this.$message.error(e.msg || '') + this.setName('') + this.spinnerLoading = false + reject(e) + }) + } + }) }) + } + }, + _verifConditions () { + let tasks = this.$store.state.dag.tasks + let bool = true + tasks.map(v=>{ + if(v.type == 'CONDITIONS' && (v.conditionResult.successNode[0] =='' || v.conditionResult.successNode[0] == null || v.conditionResult.failedNode[0] =='' || v.conditionResult.failedNode[0] == null)) { + bool = false + return false + } }) + if(!bool) { + this.$message.warning(`${i18n.$t('Successful branch flow and failed branch flow are required')}`) + return false + } + return true }, /** * Global parameter diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue index cc1c8b6d6c..a49e700e00 100755 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue @@ -470,7 +470,7 @@ this.$message.warning(`${i18n.$t('Please enter name (required)')}`) return false } - if (this.successBranch !='' && this.successBranch == this.failedBranch) { + if (this.successBranch !='' && this.successBranch !=null && this.successBranch == this.failedBranch) { this.$message.warning(`${i18n.$t('Cannot select the same node for successful branch flow and failed branch flow')}`) return false } diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/nodeStatus.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/nodeStatus.vue index fa7ee89e98..0c3f7433a3 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/nodeStatus.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/_source/nodeStatus.vue @@ -128,10 +128,6 @@ this.store.dispatch('dag/getProcessTasksList', { processDefinitionId: ids }).then(res => { resolve(['ALL'].concat(_.map(res, v => v.name))) }) - } else { - this.store.dispatch('dag/getTaskListDefIdAll', { processDefinitionIdList: ids }).then(res => { - resolve(res) - }) } }) }, diff --git a/dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js b/dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js index 64ab298437..d2c096e503 100755 --- a/dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js +++ b/dolphinscheduler-ui/src/js/module/i18n/locale/en_US.js @@ -568,5 +568,6 @@ export default { 'All Columns': 'All Columns', 'Some Columns': 'Some Columns', 'Branch flow': 'Branch flow', - 'Cannot select the same node for successful branch flow and failed branch flow': 'Cannot select the same node for successful branch flow and failed branch flow' + 'Cannot select the same node for successful branch flow and failed branch flow': 'Cannot select the same node for successful branch flow and failed branch flow', + 'Successful branch flow and failed branch flow are required': 'Successful branch flow and failed branch flow are required' } diff --git a/dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js b/dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js index 3363eea198..1003d6eeba 100755 --- a/dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js +++ b/dolphinscheduler-ui/src/js/module/i18n/locale/zh_CN.js @@ -568,5 +568,6 @@ export default { 'All Columns': '全表导入', 'Some Columns': '选择列', 'Branch flow': '分支流转', - 'Cannot select the same node for successful branch flow and failed branch flow': '成功分支流转和失败分支流转不能选择同一个节点' + 'Cannot select the same node for successful branch flow and failed branch flow': '成功分支流转和失败分支流转不能选择同一个节点', + 'Successful branch flow and failed branch flow are required': '成功分支流转和失败分支流转必填', } From 0e5fb86bef1f62c64159aedd9ae1c5e361669fb0 Mon Sep 17 00:00:00 2001 From: "gabry.wu" Date: Wed, 18 Mar 2020 17:02:57 +0800 Subject: [PATCH 112/221] Adapting partial code(file name start with N) to the sonar cloud rule (#2215) --- .../dolphinscheduler/remote/handler/NettyClientHandler.java | 4 ++-- .../dolphinscheduler/remote/handler/NettyServerHandler.java | 4 ++-- .../dolphinscheduler/remote/utils/NamedThreadFactory.java | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java index 80e561d05c..76ca4649bd 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyClientHandler.java @@ -126,14 +126,14 @@ public class NettyClientHandler extends ChannelInboundHandlerAdapter { if (!ch.isWritable()) { if (logger.isWarnEnabled()) { logger.warn("{} is not writable, over high water level : {}", - new Object[]{ch, config.getWriteBufferHighWaterMark()}); + ch, config.getWriteBufferHighWaterMark()); } config.setAutoRead(false); } else { if (logger.isWarnEnabled()) { logger.warn("{} is writable, to low water : {}", - new Object[]{ch, config.getWriteBufferLowWaterMark()}); + ch, config.getWriteBufferLowWaterMark()); } config.setAutoRead(true); } diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyServerHandler.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyServerHandler.java index 22251c35c1..88cd2d5518 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyServerHandler.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/handler/NettyServerHandler.java @@ -158,14 +158,14 @@ public class NettyServerHandler extends ChannelInboundHandlerAdapter { if (!ch.isWritable()) { if (logger.isWarnEnabled()) { logger.warn("{} is not writable, over high water level : {}", - new Object[]{ch, config.getWriteBufferHighWaterMark()}); + ch, config.getWriteBufferHighWaterMark()); } config.setAutoRead(false); } else { if (logger.isWarnEnabled()) { logger.warn("{} is writable, to low water : {}", - new Object[]{ch, config.getWriteBufferLowWaterMark()}); + ch, config.getWriteBufferLowWaterMark()); } config.setAutoRead(true); } diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/NamedThreadFactory.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/NamedThreadFactory.java index 2f0d05ebd4..be84f0f221 100644 --- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/NamedThreadFactory.java +++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/NamedThreadFactory.java @@ -52,8 +52,8 @@ public class NamedThreadFactory implements ThreadFactory { */ @Override public Thread newThread(Runnable r) { - final String threadName = count > 0 ? String.format(name + "_%d_%d", count, increment.getAndIncrement()) - : String.format(name + "_%d", increment.getAndIncrement()); + final String threadName = count > 0 ? String.format("%s_%d_%d", name, count, increment.getAndIncrement()) + : String.format("%s_%d", name, increment.getAndIncrement()); Thread t = new Thread(r, threadName); t.setDaemon(true); return t; From 880e995654e148511ff94dcf439ef0d591fa5796 Mon Sep 17 00:00:00 2001 From: ruson <648380139@qq.com> Date: Wed, 18 Mar 2020 23:26:07 +0800 Subject: [PATCH 113/221] add repace time when run history job and batch complement job ,not current time (#2196) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * add release notes in 1.2.1 (#1654) * fix Monitor bug (#1656) * fix Monitor bug * fix zk monitor bug * fix api url (#1673) * get root path from zookeeper config * Fixed DAG zoom in and zoom out nodes separated from arrows #1679 (#1685) * fix api url * Fixed DAG zoom in and zoom out nodes separated from arrows #1679 * check license and update (#1725) * Fixed tasks_queue and tasks_kill did not exist in zookeeper #1696 (#1734) Co-authored-by: elonlo * upgrade jackson from 2.9.8 to 2.10.1 (#1767) * add out put log when master/worker server start (#1769) * merge hadoop.properties into common.properties * merge hadoop,zookeeper.properties into common.properties remove combined.properties/master.properties/worker.properties * change db user/pwd to test/test * rename .dolphinscheduler_en.sh to dolphinscheduler_env.sh remove some unused in install.sh * add out put log when master/worker server start... * add start log when servers start * add check download resource permission in order to fix issues 1770 (#1788) * merge dev branch front-end code (#1786) * fix #1775 bug,delete process definition when process instance is running (#1790) * fix #1775 bug,delete process definition when process instance is running * revert CONTRIBUTING.md * fix udfs assignment and task instance page jump #1789 (#1791) * merge dev branch front-end code * fix udfs assignment * Fix task instance page jump * fix udfs assignment and task instance page jump #1789 * update method checkDownloadPermission in order to fix issues 1770 (#1794) * add check download resource permission in order to fix issues 1770 * update method checkDownloadPermission in order to fix issues 1770 * update method listUnauthorizedResource in order to fix issues 1770 * update method listUnauthorizedResource in order to fix issues 1770 (#1797) * if login user is admin,it will has permission to query all udf function (#1799) * Fixed space and icon display issues before and after the input box (#1798) * merge dev branch front-end code * fix udfs assignment * Fix task instance page jump * fix udfs assignment and task instance page jump #1789 * Fixed space and icon display issues before and after the input box * add license * add license * donot submit task to queue if sub process (#1793) * dockerfile modify (#1800) * fix #1775 bug,delete process definition when process instance is running * revert CONTRIBUTING.md * dockerfile modify * dockerfile modify * dockerfile modify * Fix data echo, style and popup cannot be closed (#1801) * merge dev branch front-end code * fix udfs assignment * Fix task instance page jump * fix udfs assignment and task instance page jump #1789 * Fixed space and icon display issues before and after the input box * add license * add license * Fix data echo, style and popup cannot be closed * Fix page number loading issue and dag not getting value(#1810) (#1815) * merge dev branch front-end code * fix udfs assignment * Fix task instance page jump * fix udfs assignment and task instance page jump #1789 * Fixed space and icon display issues before and after the input box * add license * add license * Fix data echo, style and popup cannot be closed * Fix page number loading issue and dag not getting value * [Fix issue #1770]check udf and data source in order to fix issue 1770 (#1817) * if login user is admin,it will has permission to query all udf function * check udf and data source in order to fix issue 1770 * check udf and data source in order to fix issue 1770 * check udf and data source in order to fix issue 1770 * check udf and data source in order to fix issue 1770 * check udf and data source in order to fix issue 1770 * revert common.properties * update the test method name * remove useless code and import in unit test * refactor has permission and check permission * #1813 remove "_001" after the master/server register path in zookeepe (#1824) * donot submit task to queue if sub process * [feature] #1813 remove "_001" after the master/server register path in zookeeper (#1820) * change master/worker register path. * remove "_" from register path. * remove install.sh server.servlet.context-path replace (#1823) * fix #1775 bug,delete process definition when process instance is running * revert CONTRIBUTING.md * dockerfile modify * dockerfile modify * dockerfile modify * remove install.sh server.servlet.context-path replace * Fix DAG add dependent project value exception # 1816 and keep requesting task interface (#1827) * merge dev branch front-end code * fix udfs assignment * Fix task instance page jump * fix udfs assignment and task instance page jump #1789 * Fixed space and icon display issues before and after the input box * add license * add license * Fix data echo, style and popup cannot be closed * Fix page number loading issue and dag not getting value * Fix DAG add dependent project value exception # 1816 and keep requesting task interface * rpm package add UI (#1846) * [fix #1828] when the executor of process instance is not the owner of udf resouce, the path of the read resource file is incorrect (#1847) * fix issue 1828:get the udf resource path error when create udf function * update grantResources * first verify whether udf resource is bound by udf function * update grantResources * update testListAuthorizedUdfFunc * update getUserInfo in order to run success * add LoggerServer RPC PORT modified. #1848 (#1849) * fix #1775 bug,delete process definition when process instance is running * revert CONTRIBUTING.md * dockerfile modify * dockerfile modify * dockerfile modify * remove install.sh server.servlet.context-path replace * add LoggerServer RPC PORT modified. #1848 * LoggerService Logger RPC PORT get Error modify . #1848 (#1857) * fix #1775 bug,delete process definition when process instance is running * revert CONTRIBUTING.md * dockerfile modify * dockerfile modify * dockerfile modify * remove install.sh server.servlet.context-path replace * add LoggerServer RPC PORT modified. #1848 * LoggerService Logger RPC PORT get Error modify . #1848 * Fix UDF function list delete data without refresh and page data display widened #1851 (#1852) * merge dev branch front-end code * fix udfs assignment * Fix task instance page jump * fix udfs assignment and task instance page jump #1789 * Fixed space and icon display issues before and after the input box * add license * add license * Fix data echo, style and popup cannot be closed * Fix page number loading issue and dag not getting value * Fix DAG add dependent project value exception # 1816 and keep requesting task interface * Fix UDF function list delete data without refresh and page data display widened * [Fix #1828]check whether has permission to download udf file or delete udf function (#1858) * fix issue 1828:get the udf resource path error when create udf function * update grantResources * first verify whether udf resource is bound by udf function * update grantResources * update testListAuthorizedUdfFunc * update getUserInfo in order to run success * check whether has permission to download udf file or delete udf file * update listAuthorizedResourceById in ResourceMapper.xml * add log, don't swallow exception info (#1877) * Added hints in Resource Center (#1891) * update jackson version from 2.9.8 to 2.10.1 (#1892) * update release notes and modify the plugin of rat (#1899) * modify how to build (#1902) * update release notes and modify the plugin of rat * modify how to build * [maven-release-plugin] prepare release 1.2.1 * [maven-release-plugin] prepare for next development iteration * get authorized udf resources need exclude all created by self (#1905) * get authorized udf resources need exclude all created by self * reset pom version to 1.2.1-SNAPSHOT * Update ReleaseNotes.md add detail info * Update ReleaseNotes.md * update notice year form 2019 to 2019-2020 (#1907) * [maven-release-plugin] prepare release 1.2.1 * [maven-release-plugin] prepare for next development iteration * 当手动重跑历史活着补数的时候对shell和sql脚本中的[YYYYmmddd...]变量赋值与传递的日期,而不是当前日期 Co-authored-by: lgcareer <18610854716@163.com> Co-authored-by: bao liang <29528966+lenboo@users.noreply.github.com> Co-authored-by: break60 <790061044@qq.com> Co-authored-by: Tboy Co-authored-by: elonlo Co-authored-by: qiaozhanwei Co-authored-by: zhangchunyang <18910529250@163.com> Co-authored-by: dailidong Co-authored-by: 谷雨 --- .../dolphinscheduler/common/Constants.java | 7 ++ .../common/utils/ParameterUtils.java | 79 +++++++++++++++++++ .../server/worker/task/shell/ShellTask.java | 17 +++- .../server/worker/task/sql/SqlTask.java | 12 +-- 4 files changed, 107 insertions(+), 8 deletions(-) diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java index b0a7b74d39..73655e7a9d 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java @@ -999,4 +999,11 @@ public final class Constants { * dataSource sensitive param */ public static final String DATASOURCE_PASSWORD_REGEX = "(?<=(\"password\":\")).*?(?=(\"))"; + + + /** + * new + * schedule time + */ + public static final String PARAMETER_SHECDULE_TIME = "schedule.time"; } diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/ParameterUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/ParameterUtils.java index 7a4553aaf5..270e0c4696 100644 --- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/ParameterUtils.java +++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/ParameterUtils.java @@ -78,6 +78,45 @@ public class ParameterUtils { return parameterString; } + /** + * new + * convert parameters place holders + * + * @param parameterString parameter + * @param parameterMap parameter map + * @return convert parameters place holders + */ + public static String convertParameterPlaceholders2(String parameterString, Map parameterMap) { + if (StringUtils.isEmpty(parameterString)) { + return parameterString; + } + //Get current time, schedule execute time + String cronTimeStr = parameterMap.get(Constants.PARAMETER_SHECDULE_TIME); + Date cronTime = null; + + if (StringUtils.isNotEmpty(cronTimeStr)) { + try { + cronTime = DateUtils.parseDate(cronTimeStr, new String[]{Constants.PARAMETER_FORMAT_TIME}); + + } catch (ParseException e) { + logger.error(String.format("parse %s exception", cronTimeStr), e); + } + } else { + cronTime = new Date(); + } + + // replace variable ${} form,refers to the replacement of system variables and custom variables + parameterString = PlaceholderUtils.replacePlaceholders(parameterString, parameterMap, true); + + // replace time $[...] form, eg. $[yyyyMMdd] + if (cronTime != null) { + parameterString = TimePlaceholderUtils.replacePlaceholders(parameterString, cronTime, true); + + } + return parameterString; + } + + /** * set in parameter * @param index index @@ -173,4 +212,44 @@ public class ParameterUtils { } return inputString; } + + /** + * new + * $[yyyyMMdd] replace scheduler time + * @param text + * @param paramsMap + * @return + */ + public static String replaceScheduleTime(String text, Date scheduleTime, Map paramsMap) { + if (paramsMap != null) { + //if getScheduleTime null ,is current date + if (null == scheduleTime) { + scheduleTime = new Date(); + } + String dateTime = org.apache.dolphinscheduler.common.utils.DateUtils.format(scheduleTime, Constants.PARAMETER_FORMAT_TIME); + Property p = new Property(); + p.setValue(dateTime); + p.setProp(Constants.PARAMETER_SHECDULE_TIME); + paramsMap.put(Constants.PARAMETER_SHECDULE_TIME, p); + text = ParameterUtils.convertParameterPlaceholders2(text, convert(paramsMap)); + } + return text; + } + + + /** + * format convert + * @param paramsMap params map + * @return Map of converted + * see org.apache.dolphinscheduler.server.utils.ParamUtils.convert + */ + public static Map convert(Map paramsMap){ + Map map = new HashMap<>(); + Iterator> iter = paramsMap.entrySet().iterator(); + while (iter.hasNext()){ + Map.Entry en = iter.next(); + map.put(en.getKey(),en.getValue().getValue()); + } + return map; + } } diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTask.java index 90661a690a..2c8433aa01 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/shell/ShellTask.java @@ -21,6 +21,7 @@ import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.process.Property; import org.apache.dolphinscheduler.common.task.AbstractParameters; import org.apache.dolphinscheduler.common.task.shell.ShellParameters; +import org.apache.dolphinscheduler.common.utils.DateUtils; import org.apache.dolphinscheduler.common.utils.JSONUtils; import org.apache.dolphinscheduler.common.utils.OSUtils; import org.apache.dolphinscheduler.common.utils.ParameterUtils; @@ -142,11 +143,21 @@ public class ShellTask extends AbstractTask { shellParameters.getLocalParametersMap(), taskProps.getCmdTypeIfComplement(), taskProps.getScheduleTime()); - if (paramsMap != null){ - script = ParameterUtils.convertParameterPlaceholders(script, ParamUtils.convert(paramsMap)); +// if (paramsMap != null){ +// script = ParameterUtils.convertParameterPlaceholders(script, ParamUtils.convert(paramsMap)); +// } + + //new +// replace variable TIME with $[YYYYmmddd...] in shell file when history run job and batch complement job + if (paramsMap != null) { + String dateTime = DateUtils.format(taskProps.getScheduleTime(), Constants.PARAMETER_FORMAT_TIME); + Property p = new Property(); + p.setValue(dateTime); + p.setProp(Constants.PARAMETER_SHECDULE_TIME); + paramsMap.put(Constants.PARAMETER_SHECDULE_TIME, p); + script = ParameterUtils.convertParameterPlaceholders2(script, ParamUtils.convert(paramsMap)); } - shellParameters.setRawScript(script); logger.info("raw script : {}", shellParameters.getRawScript()); diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java index 82034740fc..71ab56b528 100644 --- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java +++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/task/sql/SqlTask.java @@ -222,7 +222,9 @@ public class SqlTask extends AbstractTask { logger.info("SQL title : {}",title); sqlParameters.setTitle(title); } - + //new + //replace variable TIME with $[YYYYmmddd...] in sql when history run job and batch complement job + sql = ParameterUtils.replaceScheduleTime(sql, taskProps.getScheduleTime(), paramsMap); // special characters need to be escaped, ${} needs to be escaped String rgex = "['\"]*\\$\\{(.*?)\\}['\"]*"; setSqlParamsMap(sql, rgex, sqlParamsMap, paramsMap); @@ -341,10 +343,10 @@ public class SqlTask extends AbstractTask { logger.error(e.getMessage(),e); throw new RuntimeException(e.getMessage()); } finally { - try { - connection.close(); - } catch (Exception e) { - logger.error(e.getMessage(), e); + try { + connection.close(); + } catch (Exception e) { + logger.error(e.getMessage(), e); } } return connection; From 3c5227ac0fdcd899e27ca0e8937a4ffa2528b4ba Mon Sep 17 00:00:00 2001 From: tswstarplanet Date: Wed, 18 Mar 2020 23:48:19 +0800 Subject: [PATCH 114/221] Refactor ConnectionFactory to static inner holder Singleton (#2204) * refactor ConnectionFactory to static inner holder Singleton * remove redundant import * make getSqlSessionFactory() method private * fix sonar issue * remove @Ignore of MailUtilsTest * add MailUtilsTest to pom.xml to test * fix MailUtilsTest path error in pom.xml * modify test method name * add assert and logger to unit test * add log * add log * add AlertDaoTest * move AlertDaoTest to new module * modify test in pom.xml * remove unnecessary log --- .../alert/utils/MailUtilsTest.java | 8 +- .../apache/dolphinscheduler/dao/AlertDao.java | 4 +- .../dao/datasource/ConnectionFactory.java | 97 ++++++++++--------- .../dao/upgrade/UpgradeDao.java | 2 +- .../dolphinscheduler/dao/AlertDaoTest.java | 34 +++++++ .../dao/mapper/ConnectionFactoryTest.java | 2 +- pom.xml | 2 + 7 files changed, 97 insertions(+), 52 deletions(-) create mode 100644 dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/AlertDaoTest.java diff --git a/dolphinscheduler-alert/src/test/java/org/apache/dolphinscheduler/alert/utils/MailUtilsTest.java b/dolphinscheduler-alert/src/test/java/org/apache/dolphinscheduler/alert/utils/MailUtilsTest.java index 612de3e31d..1820a1ef89 100644 --- a/dolphinscheduler-alert/src/test/java/org/apache/dolphinscheduler/alert/utils/MailUtilsTest.java +++ b/dolphinscheduler-alert/src/test/java/org/apache/dolphinscheduler/alert/utils/MailUtilsTest.java @@ -23,7 +23,7 @@ import org.apache.dolphinscheduler.dao.AlertDao; import org.apache.dolphinscheduler.dao.DaoFactory; import org.apache.dolphinscheduler.dao.entity.Alert; import org.apache.dolphinscheduler.dao.entity.User; -import org.junit.Ignore; +import org.junit.Assert; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,7 +33,6 @@ import java.util.*; /** */ -@Ignore public class MailUtilsTest { private static final Logger logger = LoggerFactory.getLogger(MailUtilsTest.class); @Test @@ -138,8 +137,10 @@ public class MailUtilsTest { * Table */ @Test - public void addAlertTable(){ + public void testAddAlertTable(){ + logger.info("testAddAlertTable"); AlertDao alertDao = DaoFactory.getDaoInstance(AlertDao.class); + Assert.assertNotNull(alertDao); Alert alert = new Alert(); alert.setTitle("Mysql Exception"); alert.setShowType(ShowType.TABLE); @@ -149,6 +150,7 @@ public class MailUtilsTest { alert.setAlertType(AlertType.EMAIL); alert.setAlertGroupId(1); alertDao.addAlert(alert); + logger.info("" +alert); } @Test diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/AlertDao.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/AlertDao.java index 1a8c09e611..2ba93d7607 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/AlertDao.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/AlertDao.java @@ -50,8 +50,8 @@ public class AlertDao extends AbstractBaseDao { @Override protected void init() { - alertMapper = ConnectionFactory.getMapper(AlertMapper.class); - userAlertGroupMapper = ConnectionFactory.getMapper(UserAlertGroupMapper.class); + alertMapper = ConnectionFactory.getInstance().getMapper(AlertMapper.class); + userAlertGroupMapper = ConnectionFactory.getInstance().getMapper(UserAlertGroupMapper.class); } /** diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/ConnectionFactory.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/ConnectionFactory.java index a3bc6a0150..199fed0ce6 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/ConnectionFactory.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/ConnectionFactory.java @@ -34,29 +34,47 @@ import javax.sql.DataSource; /** - * not spring manager connection, only use for init db, and alert module for non-spring application + * not spring manager connection, only use for init db, and alert module for non-spring application * data source connection factory */ -public class ConnectionFactory extends SpringConnectionFactory{ +public class ConnectionFactory extends SpringConnectionFactory { private static final Logger logger = LoggerFactory.getLogger(ConnectionFactory.class); + private static class ConnectionFactoryHolder { + private static final ConnectionFactory connectionFactory = new ConnectionFactory(); + } + + public static ConnectionFactory getInstance() { + return ConnectionFactoryHolder.connectionFactory; + } + + private ConnectionFactory() { + try { + sqlSessionFactory = getSqlSessionFactory(); + sqlSessionTemplate = getSqlSessionTemplate(); + } catch (Exception e) { + logger.error("Initializing ConnectionFactory error", e); + throw new RuntimeException(e); + } + } /** * sql session factory */ - private static SqlSessionFactory sqlSessionFactory; + private SqlSessionFactory sqlSessionFactory; /** * sql session template */ - private static SqlSessionTemplate sqlSessionTemplate; + private SqlSessionTemplate sqlSessionTemplate; /** * get the data source + * * @return druid dataSource */ - public static DruidDataSource getDataSource() { + public DruidDataSource getDataSource() { DruidDataSource druidDataSource = new DruidDataSource(); @@ -89,65 +107,54 @@ public class ConnectionFactory extends SpringConnectionFactory{ /** * * get sql session factory + * * @return sqlSessionFactory * @throws Exception sqlSessionFactory exception */ - public static SqlSessionFactory getSqlSessionFactory() throws Exception { - if (sqlSessionFactory == null) { - synchronized (ConnectionFactory.class) { - if (sqlSessionFactory == null) { - DataSource dataSource = getDataSource(); - TransactionFactory transactionFactory = new JdbcTransactionFactory(); - - Environment environment = new Environment("development", transactionFactory, dataSource); - - MybatisConfiguration configuration = new MybatisConfiguration(); - configuration.setEnvironment(environment); - configuration.setLazyLoadingEnabled(true); - configuration.addMappers("org.apache.dolphinscheduler.dao.mapper"); - configuration.addInterceptor(new PaginationInterceptor()); - - MybatisSqlSessionFactoryBean sqlSessionFactoryBean = new MybatisSqlSessionFactoryBean(); - sqlSessionFactoryBean.setConfiguration(configuration); - sqlSessionFactoryBean.setDataSource(dataSource); - - sqlSessionFactoryBean.setTypeEnumsPackage("org.apache.dolphinscheduler.*.enums"); - sqlSessionFactory = sqlSessionFactoryBean.getObject(); - } - } - } + private SqlSessionFactory getSqlSessionFactory() throws Exception { + DataSource dataSource = getDataSource(); + TransactionFactory transactionFactory = new JdbcTransactionFactory(); + + Environment environment = new Environment("development", transactionFactory, dataSource); + + MybatisConfiguration configuration = new MybatisConfiguration(); + configuration.setEnvironment(environment); + configuration.setLazyLoadingEnabled(true); + configuration.addMappers("org.apache.dolphinscheduler.dao.mapper"); + configuration.addInterceptor(new PaginationInterceptor()); + + MybatisSqlSessionFactoryBean sqlSessionFactoryBean = new MybatisSqlSessionFactoryBean(); + sqlSessionFactoryBean.setConfiguration(configuration); + sqlSessionFactoryBean.setDataSource(dataSource); + + sqlSessionFactoryBean.setTypeEnumsPackage("org.apache.dolphinscheduler.*.enums"); + sqlSessionFactory = sqlSessionFactoryBean.getObject(); return sqlSessionFactory; +} + + private SqlSessionTemplate getSqlSessionTemplate() { + sqlSessionTemplate = new SqlSessionTemplate(sqlSessionFactory); + return sqlSessionTemplate; } /** * get sql session + * * @return sqlSession */ - public static SqlSession getSqlSession() { - if (sqlSessionTemplate == null) { - synchronized (ConnectionFactory.class) { - if (sqlSessionTemplate == null) { - try { - sqlSessionTemplate = new SqlSessionTemplate(getSqlSessionFactory()); - return sqlSessionTemplate; - } catch (Exception e) { - logger.error("getSqlSession error", e); - throw new RuntimeException(e); - } - } - } - } + public SqlSession getSqlSession() { return sqlSessionTemplate; } /** * get mapper + * * @param type target class - * @param generic + * @param generic * @return target object */ - public static T getMapper(Class type) { + public T getMapper(Class type) { try { return getSqlSession().getMapper(type); } catch (Exception e) { diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/upgrade/UpgradeDao.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/upgrade/UpgradeDao.java index aed93038f5..384c3604c4 100644 --- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/upgrade/UpgradeDao.java +++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/upgrade/UpgradeDao.java @@ -53,7 +53,7 @@ public abstract class UpgradeDao extends AbstractBaseDao { * @return DruidDataSource */ public static DruidDataSource getDataSource(){ - DruidDataSource dataSource = ConnectionFactory.getDataSource(); + DruidDataSource dataSource = ConnectionFactory.getInstance().getDataSource(); dataSource.setInitialSize(2); dataSource.setMinIdle(2); dataSource.setMaxActive(2); diff --git a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/AlertDaoTest.java b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/AlertDaoTest.java new file mode 100644 index 0000000000..b4f197adc9 --- /dev/null +++ b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/AlertDaoTest.java @@ -0,0 +1,34 @@ +/* + * 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.dao; + +import org.junit.Assert; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class AlertDaoTest { + private static final Logger logger = LoggerFactory.getLogger(AlertDaoTest.class); + + @Test + public void testGetAlertDao() { + logger.info("testGetAlertDao start"); + AlertDao alertDao = DaoFactory.getDaoInstance(AlertDao.class); + Assert.assertNotNull(alertDao); + logger.info("testGetAlertDao end"); + } +} diff --git a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/ConnectionFactoryTest.java b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/ConnectionFactoryTest.java index 5ba2936aaf..f4139444f5 100644 --- a/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/ConnectionFactoryTest.java +++ b/dolphinscheduler-dao/src/test/java/org/apache/dolphinscheduler/dao/mapper/ConnectionFactoryTest.java @@ -31,7 +31,7 @@ public class ConnectionFactoryTest { */ @Test public void testConnection()throws Exception{ - Connection connection = ConnectionFactory.getDataSource().getPooledConnection().getConnection(); + Connection connection = ConnectionFactory.getInstance().getDataSource().getPooledConnection().getConnection(); Assert.assertTrue(connection != null); } } \ No newline at end of file diff --git a/pom.xml b/pom.xml index 8910723117..412a28eda7 100644 --- a/pom.xml +++ b/pom.xml @@ -740,6 +740,8 @@ **/server/worker/task/sqoop/SqoopTaskTest.java **/server/utils/DataxUtilsTest.java **/service/zk/DefaultEnsembleProviderTest.java + **/alert/utils/MailUtilsTest.java + **/dao/AlertDaoTest.java From bfcdc17d5d18a60a26e8c68daa4dee2690e7e1e5 Mon Sep 17 00:00:00 2001 From: break60 <790061044@qq.com> Date: Thu, 19 Mar 2020 14:00:18 +0800 Subject: [PATCH 115/221] datax --- .../pages/dag/_source/formModel/formModel.vue | 3 +- .../dag/_source/formModel/tasks/datax.vue | 345 ++++++++++++------ .../module/components/transfer/resource.vue | 4 +- .../src/js/module/i18n/locale/en_US.js | 1 + .../src/js/module/i18n/locale/zh_CN.js | 1 + 5 files changed, 233 insertions(+), 121 deletions(-) diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue index a49e700e00..56cefd5b31 100755 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/formModel.vue @@ -430,7 +430,8 @@ * return params */ _onParams (o) { - this.params = Object.assign(this.params, {}, o) + console.log(o) + this.params = Object.assign({}, o) }, _onCacheParams (o) { diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/datax.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/datax.vue index 959610f95a..0f82903844 100755 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/datax.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/datax.vue @@ -17,90 +17,127 @@ diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/mr.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/mr.vue index 706a35f4fe..3691985d7d 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/mr.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/mr.vue @@ -44,19 +44,9 @@
    {{$t('Main jar package')}}
    - - - - + +
    {{ node.raw.fullName }}
    +
    @@ -88,12 +78,9 @@
    {{$t('Resources')}}
    - - + +
    {{ node.raw.fullName }}
    +
    @@ -115,6 +102,8 @@ import mListBox from './_source/listBox' import mResources from './_source/resources' import mLocalParams from './_source/localParams' + import Treeselect from '@riophae/vue-treeselect' + import '@riophae/vue-treeselect/dist/vue-treeselect.css' import disabledState from '@/module/mixin/disabledState' export default { name: 'mr', @@ -125,6 +114,7 @@ // Master jar package mainJar: null, // Main jar package (List) + mainJarLists: [], mainJarList: [], // Resource(list) resourceList: [], @@ -139,7 +129,12 @@ // Program type programType: 'JAVA', // Program type(List) - programTypeList: [{ code: 'JAVA' }, { code: 'PYTHON' }] + programTypeList: [{ code: 'JAVA' }, { code: 'PYTHON' }], + normalizer(node) { + return { + label: node.name + } + } } }, props: { @@ -165,6 +160,12 @@ _onCacheResourcesData (a) { this.cacheResourceList = a }, + diGuiTree(item) { // Recursive convenience tree structure + item.forEach(item => { + item.children === '' || item.children === undefined || item.children === null || item.children.length === 0?         + delete item.children : this.diGuiTree(item.children); + }) + }, /** * verification */ @@ -179,22 +180,19 @@ return false } - if (!this.$refs.refResources._verifResources()) { - return false - } - // localParams Subcomponent verification if (!this.$refs.refLocalParams._verifProp()) { return false } - // storage this.$emit('on-params', { mainClass: this.mainClass, mainJar: { - res: this.mainJar + id: this.mainJar }, - resourceList: this.resourceList, + resourceList: _.map(this.resourceList, v => { + return {id: v} + }), localParams: this.localParams, mainArgs: this.mainArgs, others: this.others, @@ -202,24 +200,7 @@ }) return true }, - /** - * Get resource data - */ - _getResourcesList () { - return new Promise((resolve, reject) => { - let isJar = (alias) => { - return alias.substring(alias.lastIndexOf('.') + 1, alias.length) !== 'jar' - } - this.mainJarList = _.map(_.cloneDeep(this.store.state.dag.resourcesListS), v => { - return { - id: v.id, - code: v.alias, - disabled: isJar(v.alias) - } - }) - resolve() - }) - } + }, watch: { /** @@ -240,9 +221,11 @@ return { mainClass: this.mainClass, mainJar: { - res: this.mainJar + id: this.mainJar }, - resourceList: this.cacheResourceList, + resourceList: _.map(this.resourceList, v => { + return {id: v} + }), localParams: this.localParams, mainArgs: this.mainArgs, others: this.others, @@ -251,13 +234,18 @@ } }, created () { - this._getResourcesList().then(() => { + let item = this.store.state.dag.resourcesListS + let items = this.store.state.dag.resourcesListJar + this.diGuiTree(item) + this.diGuiTree(items) + this.mainJarList = item + this.mainJarLists = items let o = this.backfillItem // Non-null objects represent backfill if (!_.isEmpty(o)) { this.mainClass = o.params.mainClass || '' - this.mainJar = o.params.mainJar.res || '' + this.mainJar = o.params.mainJar.id || '' this.mainArgs = o.params.mainArgs || '' this.others = o.params.others this.programType = o.params.programType || 'JAVA' @@ -265,7 +253,9 @@ // backfill resourceList let resourceList = o.params.resourceList || [] if (resourceList.length) { - this.resourceList = resourceList + this.resourceList = _.map(resourceList, v => { + return v.id + }) this.cacheResourceList = resourceList } @@ -275,12 +265,11 @@ this.localParams = localParams } } - }) }, mounted () { }, - components: { mLocalParams, mListBox, mResources } + components: { mLocalParams, mListBox, mResources, Treeselect } } diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/python.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/python.vue index 6f495d22a0..e8e7e58771 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/python.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/python.vue @@ -108,10 +108,6 @@ return false } - if (!this.$refs.refResources._verifResources()) { - return false - } - // localParams Subcomponent verification if (!this.$refs.refLocalParams._verifProp()) { return false @@ -119,7 +115,9 @@ // storage this.$emit('on-params', { - resourceList: this.resourceList, + resourceList: _.map(this.resourceList, v => { + return {id: v} + }), localParams: this.localParams, rawScript: editor.getValue() }) @@ -129,8 +127,6 @@ * Processing code highlighting */ _handlerEditor () { - this._destroyEditor() - // editor editor = codemirror('code-python-mirror', { mode: 'python', @@ -145,45 +141,28 @@ } } - this.changes = () => { - this._cacheParams() - } - // Monitor keyboard editor.on('keypress', this.keypress) - editor.on('changes', this.changes) - editor.setValue(this.rawScript) return editor - }, - _cacheParams () { - this.$emit('on-cache-params', { - resourceList: this.cacheResourceList, - localParams: this.localParams, - rawScript: editor ? editor.getValue() : '' - }); - }, - _destroyEditor () { - if (editor) { - editor.toTextArea() // Uninstall - editor.off($('.code-python-mirror'), 'keypress', this.keypress) - editor.off($('.code-python-mirror'), 'changes', this.changes) - } } }, watch: { //Watch the cacheParams cacheParams (val) { - this._cacheParams() + this.$emit('on-cache-params', val); } }, computed: { cacheParams () { return { - resourceList: this.cacheResourceList, - localParams: this.localParams + resourceList: _.map(this.resourceList, v => { + return {id: v} + }), + localParams: this.localParams, + rawScript: editor ? editor.getValue() : '' } } }, @@ -197,7 +176,9 @@ // backfill resourceList let resourceList = o.params.resourceList || [] if (resourceList.length) { - this.resourceList = resourceList + this.resourceList = _.map(resourceList, v => { + return v.id + }) this.cacheResourceList = resourceList } @@ -214,11 +195,8 @@ }, 200) }, destroyed () { - if (editor) { - editor.toTextArea() // Uninstall - editor.off($('.code-python-mirror'), 'keypress', this.keypress) - editor.off($('.code-python-mirror'), 'changes', this.changes) - } + editor.toTextArea() // Uninstall + editor.off($('.code-python-mirror'), 'keypress', this.keypress) }, components: { mLocalParams, mListBox, mResources } } diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/shell.vue b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/shell.vue index a4b20f3310..952fd3a38d 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/shell.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/dag/_source/formModel/tasks/shell.vue @@ -32,6 +32,14 @@
    +
    {{$t('Resources')}}
    +
    + +
    {{ node.raw.fullName }}
    +
    +
    +
    +
    {{$t('Custom Parameters')}}
    @@ -63,6 +71,8 @@ import mResources from './_source/resources' import mLocalParams from './_source/localParams' import disabledState from '@/module/mixin/disabledState' + import Treeselect from '@riophae/vue-treeselect' + import '@riophae/vue-treeselect/dist/vue-treeselect.css' import codemirror from '@/conf/home/pages/resource/pages/file/pages/_source/codemirror' let editor @@ -78,7 +88,14 @@ // resource(list) resourceList: [], // Cache ResourceList - cacheResourceList: [] + cacheResourceList: [], + // define options + options: [], + normalizer(node) { + return { + label: node.name + } + }, } }, mixins: [disabledState], @@ -143,17 +160,19 @@ return false } - if (!this.$refs.refResources._verifResources()) { - return false - } - // localParams Subcomponent verification if (!this.$refs.refLocalParams._verifProp()) { return false } + // Process resourcelist + let dataProcessing= _.map(this.resourceList, v => { + return { + id: v + } + }) // storage this.$emit('on-params', { - resourceList: this.resourceList, + resourceList: dataProcessing, localParams: this.localParams, rawScript: editor.getValue() }) @@ -163,8 +182,6 @@ * Processing code highlighting */ _handlerEditor () { - this._destroyEditor() - // editor editor = codemirror('code-shell-mirror', { mode: 'shell', @@ -179,51 +196,41 @@ } } - this.changes = () => { - this._cacheParams() - } - // Monitor keyboard editor.on('keypress', this.keypress) - - editor.on('changes', this.changes) - editor.setValue(this.rawScript) return editor }, - _cacheParams () { - this.$emit('on-cache-params', { - resourceList: this.cacheResourceList, - localParams: this.localParams, - rawScript: editor ? editor.getValue() : '' - }); - }, - _destroyEditor () { - if (editor) { - editor.toTextArea() // Uninstall - editor.off($('.code-sql-mirror'), 'keypress', this.keypress) - editor.off($('.code-sql-mirror'), 'changes', this.changes) - } + diGuiTree(item) { // Recursive convenience tree structure + item.forEach(item => { + item.children === '' || item.children === undefined || item.children === null || item.children.length === 0?         + delete item.children : this.diGuiTree(item.children); + }) } }, watch: { //Watch the cacheParams cacheParams (val) { - this._cacheParams() + this.$emit('on-cache-params', val); } }, computed: { cacheParams () { return { - resourceList: this.cacheResourceList, - localParams: this.localParams + resourceList: _.map(this.resourceList, v => { + return {id: v} + }), + localParams: this.localParams, + rawScript: editor ? editor.getValue() : '' } } }, created () { + let item = this.store.state.dag.resourcesListS + this.diGuiTree(item) + this.options = item let o = this.backfillItem - // Non-null objects represent backfill if (!_.isEmpty(o)) { this.rawScript = o.params.rawScript || '' @@ -231,7 +238,9 @@ // backfill resourceList let resourceList = o.params.resourceList || [] if (resourceList.length) { - this.resourceList = resourceList + this.resourceList = _.map(resourceList, v => { + return v.id + }) this.cacheResourceList = resourceList } @@ -251,10 +260,9 @@ if (editor) { editor.toTextArea() // Uninstall editor.off($('.code-shell-mirror'), 'keypress', this.keypress) - editor.off($('.code-shell-mirror'), 'changes', this.changes) } }, - components: { mLocalParams, mListBox, mResources, mScriptBox } + components: { mLocalParams, mListBox, mResources, mScriptBox, Treeselect } } diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/createUdfFolder/index.vue b/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/createUdfFolder/index.vue new file mode 100755 index 0000000000..2511452269 --- /dev/null +++ b/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/createUdfFolder/index.vue @@ -0,0 +1,131 @@ +/* + * 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. + */ + + + + diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/details/index.vue b/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/details/index.vue index e961d8b1ee..6875cd4b2e 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/details/index.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/details/index.vue @@ -21,7 +21,7 @@

    {{name}}
    - + {{size}}

    diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/edit/index.vue b/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/edit/index.vue index a0d1d7d187..0290af0988 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/edit/index.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/edit/index.vue @@ -44,8 +44,8 @@ + + diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/subFileFolder/index.vue b/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/subFileFolder/index.vue new file mode 100755 index 0000000000..9f903a127b --- /dev/null +++ b/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/subFileFolder/index.vue @@ -0,0 +1,144 @@ +/* + * 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. + */ + + + + diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/subdirectory/_source/list.vue b/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/subdirectory/_source/list.vue new file mode 100755 index 0000000000..f5e801a205 --- /dev/null +++ b/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/subdirectory/_source/list.vue @@ -0,0 +1,251 @@ +/* + * 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. + */ + + diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/subdirectory/_source/rename.vue b/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/subdirectory/_source/rename.vue new file mode 100755 index 0000000000..6f7dacae89 --- /dev/null +++ b/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/subdirectory/_source/rename.vue @@ -0,0 +1,120 @@ +/* + * 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. + */ + + \ No newline at end of file diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/subdirectory/index.vue b/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/subdirectory/index.vue new file mode 100755 index 0000000000..12be6b0bc8 --- /dev/null +++ b/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/file/pages/subdirectory/index.vue @@ -0,0 +1,173 @@ +/* + * 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. + */ + + + diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/udf/pages/createUdfFolder/index.vue b/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/udf/pages/createUdfFolder/index.vue new file mode 100755 index 0000000000..c707ce8c90 --- /dev/null +++ b/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/udf/pages/createUdfFolder/index.vue @@ -0,0 +1,128 @@ +/* + * 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. + */ + + + + diff --git a/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/udf/pages/function/_source/createUdf.vue b/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/udf/pages/function/_source/createUdf.vue index 01d8d22650..1408c552db 100644 --- a/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/udf/pages/function/_source/createUdf.vue +++ b/dolphinscheduler-ui/src/js/conf/home/pages/resource/pages/udf/pages/function/_source/createUdf.vue @@ -15,7 +15,7 @@ * limitations under the License. */