lvyanquan commented on code in PR #3643:
URL: https://github.com/apache/flink-cdc/pull/3643#discussion_r2013464438


##########
flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/deployment/YarnApplicationDeploymentExecutor.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.flink.cdc.composer.flink.deployment;
+
+import org.apache.flink.cdc.common.utils.Preconditions;
+import org.apache.flink.cdc.composer.PipelineDeploymentExecutor;
+import org.apache.flink.cdc.composer.PipelineExecution;
+import org.apache.flink.client.deployment.ClusterSpecification;
+import org.apache.flink.client.deployment.application.ApplicationConfiguration;
+import org.apache.flink.client.program.ClusterClient;
+import org.apache.flink.client.program.ClusterClientProvider;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.PipelineOptions;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.fs.FileStatus;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.yarn.YarnClusterClientFactory;
+import org.apache.flink.yarn.YarnClusterDescriptor;
+import org.apache.flink.yarn.configuration.YarnConfigOptions;
+import org.apache.flink.yarn.configuration.YarnLogConfigUtil;
+
+import org.apache.flink.shaded.guava31.com.google.common.base.Joiner;
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+/** Deploy flink cdc job by yarn application mode. */
+public class YarnApplicationDeploymentExecutor implements 
PipelineDeploymentExecutor {
+    private static final Logger LOG =
+            LoggerFactory.getLogger(YarnApplicationDeploymentExecutor.class);
+    private static final String FLINK_CDC_HOME_ENV_VAR = "FLINK_CDC_HOME";
+
+    @Override
+    public PipelineExecution.ExecutionInfo deploy(
+            CommandLine commandLine,
+            Configuration flinkConfig,
+            List<Path> additionalJars,
+            Path flinkHome)
+            throws Exception {
+        LOG.info("Submitting application in 'Flink Yarn Application Mode'.");
+        if (flinkConfig.get(PipelineOptions.JARS) == null) {
+            flinkConfig.set(
+                    PipelineOptions.JARS, 
Collections.singletonList(getFlinkCDCDistJarFromEnv()));
+        }
+        flinkConfig.set(
+                YarnConfigOptions.SHIP_FILES,
+                
additionalJars.stream().map(Path::toString).collect(Collectors.toList()));
+
+        ObjectMapper mapper = new ObjectMapper(new YAMLFactory());
+
+        Path pipelinePath = new Path(commandLine.getArgList().get(0));
+        FileSystem fileSystem = FileSystem.get(pipelinePath.toUri());
+        FSDataInputStream pipelineInStream = fileSystem.open(pipelinePath);
+
+        flinkConfig.set(
+                ApplicationConfiguration.APPLICATION_ARGS,
+                
Collections.singletonList(mapper.readTree(pipelineInStream).toString()));
+        YarnLogConfigUtil.setLogConfigFileInConfig(
+                flinkConfig, Joiner.on(File.separator).join(flinkHome, 
"conf"));
+
+        flinkConfig.set(
+                ApplicationConfiguration.APPLICATION_MAIN_CLASS,
+                "org.apache.flink.cdc.cli.CliExecutor");
+        final YarnClusterClientFactory yarnClusterClientFactory = new 
YarnClusterClientFactory();
+        final YarnClusterDescriptor descriptor =
+                yarnClusterClientFactory.createClusterDescriptor(flinkConfig);
+        ClusterSpecification specification =
+                yarnClusterClientFactory.getClusterSpecification(flinkConfig);
+        ApplicationConfiguration applicationConfiguration =
+                ApplicationConfiguration.fromConfiguration(flinkConfig);
+
+        ClusterClient<ApplicationId> client = null;
+        try {
+            ClusterClientProvider<ApplicationId> clusterClientProvider =
+                    descriptor.deployApplicationCluster(specification, 
applicationConfiguration);
+            client = clusterClientProvider.getClusterClient();
+            ApplicationId clusterId = client.getClusterId();
+            LOG.info("Deployment Flink CDC From Cluster ID {}", clusterId);
+            return new PipelineExecution.ExecutionInfo(
+                    clusterId.toString(), "submit job successful");
+        } catch (Exception e) {
+            if (client != null) {
+                client.shutDownCluster();
+            }
+            throw new RuntimeException("Failed to deploy Flink CDC job", e);
+        } finally {
+            descriptor.close();
+            if (client != null) {
+                client.close();
+            }
+        }
+    }
+
+    private String getFlinkCDCDistJarFromEnv() throws IOException {
+        String flinkCDCHomeFromEnvVar = System.getenv(FLINK_CDC_HOME_ENV_VAR);
+        Preconditions.checkNotNull(
+                flinkCDCHomeFromEnvVar,
+                "FLINK_CDC_HOME is not correctly set in environment variable, 
current FLINK_CDC_HOME is: "
+                        + FLINK_CDC_HOME_ENV_VAR);
+        Path flinkCDCLibPath = new Path(flinkCDCHomeFromEnvVar, "lib");
+        if (!flinkCDCLibPath.getFileSystem().exists(flinkCDCLibPath)
+                || 
!flinkCDCLibPath.getFileSystem().getFileStatus(flinkCDCLibPath).isDir()) {
+            throw new RuntimeException(
+                    "Flink cdc home lib is not file or not directory: "
+                            + 
flinkCDCLibPath.makeQualified(flinkCDCLibPath.getFileSystem()));
+        }
+
+        FileStatus[] fileStatuses = 
flinkCDCLibPath.getFileSystem().listStatus(flinkCDCLibPath);
+        Optional<Path> distJars =
+                Arrays.stream(fileStatuses)
+                        .filter(status -> !status.isDir())
+                        .filter(
+                                file ->
+                                        file.getPath()
+                                                .getName()
+                                                .matches(
+                                                        
"^flink-cdc-dist-(\\d+(\\.\\d+)*)(-SNAPSHOT)?\\.jar$"))
+                        .map(FileStatus::getPath)

Review Comment:
   `file.getPath()` has been called twice, can be simplified by:
   ```
   Optional<Path> distJars =
           Arrays.stream(fileStatuses)
                   .filter(status -> !status.isDir())
                   .map(FileStatus::getPath)
                   .filter(
                           path ->
                                   path
                                           .getName()
                                           .matches(
                                                   
"^flink-cdc-dist-(\\d+(\\.\\d+)*)(-SNAPSHOT)?\\.jar$"))
                   .findFirst();
   ```



##########
flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestOnYarnEnvironment.java:
##########
@@ -0,0 +1,327 @@
+/*
+ * 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.flink.cdc.pipeline.tests.utils;
+
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.cdc.common.test.utils.TestUtils;
+import org.apache.flink.cdc.common.utils.Preconditions;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.core.testutils.CommonTestUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.MiniYARNCluster;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import static java.lang.Thread.sleep;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Fail.fail;
+
+/** Test environment running pipeline job on YARN mini-cluster. */
+public class PipelineTestOnYarnEnvironment extends TestLogger {
+    private static final Logger LOG = 
LoggerFactory.getLogger(PipelineTestOnYarnEnvironment.class);
+
+    protected static final YarnConfiguration YARN_CONFIGURATION;
+    private YarnClient yarnClient = null;
+    protected static MiniYARNCluster yarnCluster = null;
+
+    protected static final String TEST_CLUSTER_NAME_KEY = 
"flink-yarn-minicluster-name";
+    protected static final int NUM_NODEMANAGERS = 2;
+
+    protected static File yarnSiteXML = null;
+
+    @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+    private static final Duration yarnAppTerminateTimeout = 
Duration.ofSeconds(120);
+    private static final int sleepIntervalInMS = 100;
+
+    // copy from org.apache.flink.yarn.YarnTestBase
+    static {
+        YARN_CONFIGURATION = new YarnConfiguration();
+        
YARN_CONFIGURATION.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 
32);
+        YARN_CONFIGURATION.setInt(
+                YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
+                4096); // 4096 is the available memory anyways
+        YARN_CONFIGURATION.setBoolean(
+                YarnConfiguration.RM_SCHEDULER_INCLUDE_PORT_IN_NODE_NAME, 
true);
+        YARN_CONFIGURATION.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
+        
YARN_CONFIGURATION.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, 2);
+        
YARN_CONFIGURATION.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES,
 4);
+        YARN_CONFIGURATION.setInt(YarnConfiguration.DEBUG_NM_DELETE_DELAY_SEC, 
3600);
+        
YARN_CONFIGURATION.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, false);
+        YARN_CONFIGURATION.setInt(
+                YarnConfiguration.NM_VCORES, 666); // memory is overwritten in 
the MiniYARNCluster.
+        // so we have to change the number of cores for testing.
+        YARN_CONFIGURATION.setFloat(
+                YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, 
99.0F);
+        YARN_CONFIGURATION.set(YarnConfiguration.YARN_APPLICATION_CLASSPATH, 
getYarnClasspath());
+        YARN_CONFIGURATION.setInt(
+                YarnConfiguration.RESOURCEMANAGER_CONNECT_RETRY_INTERVAL_MS, 
1000);
+        
YARN_CONFIGURATION.setInt(YarnConfiguration.RESOURCEMANAGER_CONNECT_MAX_WAIT_MS,
 5000);
+        YARN_CONFIGURATION.set(TEST_CLUSTER_NAME_KEY, 
"flink-yarn-tests-application");
+    }
+
+    @Before
+    public void setupYarnClient() throws Exception {
+        if (yarnClient == null) {
+            yarnClient = YarnClient.createYarnClient();
+            yarnClient.init(getYarnConfiguration());
+            yarnClient.start();
+        }
+    }
+
+    @After
+    public void shutdownYarnClient() {
+        yarnClient.stop();
+    }
+
+    @AfterClass
+    public static void teardown() {
+
+        if (yarnCluster != null) {
+            LOG.info("Stopping MiniYarn Cluster");
+            yarnCluster.stop();
+            yarnCluster = null;
+        }
+
+        // Unset FLINK_CONF_DIR, as it might change the behavior of other tests
+        Map<String, String> map = new HashMap<>(System.getenv());
+        map.remove(ConfigConstants.ENV_FLINK_CONF_DIR);
+        map.remove("YARN_CONF_DIR");
+        map.remove("IN_TESTS");
+        CommonTestUtils.setEnv(map);
+
+        if (yarnSiteXML != null) {
+            yarnSiteXML.delete();
+        }
+    }
+
+    protected static YarnConfiguration getYarnConfiguration() {
+        return YARN_CONFIGURATION;
+    }
+
+    public static void startMiniYARNCluster() {
+        try {
+            LOG.info("Starting up MiniYARNCluster");
+            if (yarnCluster == null) {
+                final String testName =
+                        
YARN_CONFIGURATION.get(PipelineTestOnYarnEnvironment.TEST_CLUSTER_NAME_KEY);
+                yarnCluster =
+                        new MiniYARNCluster(
+                                testName == null ? "YarnTest_" + 
UUID.randomUUID() : testName,
+                                NUM_NODEMANAGERS,
+                                1,
+                                1);
+
+                yarnCluster.init(YARN_CONFIGURATION);
+                yarnCluster.start();
+            }
+
+            File targetTestClassesFolder = new File("target/test-classes");
+            writeYarnSiteConfigXML(YARN_CONFIGURATION, 
targetTestClassesFolder);
+
+            Map<String, String> map = new HashMap<String, 
String>(System.getenv());
+            map.put(
+                    "IN_TESTS",
+                    "yes we are in tests"); // see YarnClusterDescriptor() for 
more infos
+            map.put("YARN_CONF_DIR", 
targetTestClassesFolder.getAbsolutePath());
+            map.put("MAX_LOG_FILE_NUMBER", "10");
+            CommonTestUtils.setEnv(map);
+
+            
assertThat(yarnCluster.getServiceState()).isEqualTo(Service.STATE.STARTED);
+            // wait for the nodeManagers to connect
+            while (!yarnCluster.waitForNodeManagersToConnect(500)) {
+                LOG.info("Waiting for Nodemanagers to connect");

Review Comment:
   NodeManagers



##########
flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/deployment/YarnApplicationDeploymentExecutor.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.flink.cdc.composer.flink.deployment;
+
+import org.apache.flink.cdc.common.utils.Preconditions;
+import org.apache.flink.cdc.composer.PipelineDeploymentExecutor;
+import org.apache.flink.cdc.composer.PipelineExecution;
+import org.apache.flink.client.deployment.ClusterSpecification;
+import org.apache.flink.client.deployment.application.ApplicationConfiguration;
+import org.apache.flink.client.program.ClusterClient;
+import org.apache.flink.client.program.ClusterClientProvider;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.PipelineOptions;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.fs.FileStatus;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.yarn.YarnClusterClientFactory;
+import org.apache.flink.yarn.YarnClusterDescriptor;
+import org.apache.flink.yarn.configuration.YarnConfigOptions;
+import org.apache.flink.yarn.configuration.YarnLogConfigUtil;
+
+import org.apache.flink.shaded.guava31.com.google.common.base.Joiner;
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+/** Deploy flink cdc job by yarn application mode. */
+public class YarnApplicationDeploymentExecutor implements 
PipelineDeploymentExecutor {
+    private static final Logger LOG =
+            LoggerFactory.getLogger(YarnApplicationDeploymentExecutor.class);
+    private static final String FLINK_CDC_HOME_ENV_VAR = "FLINK_CDC_HOME";
+
+    @Override
+    public PipelineExecution.ExecutionInfo deploy(
+            CommandLine commandLine,
+            Configuration flinkConfig,
+            List<Path> additionalJars,
+            Path flinkHome)
+            throws Exception {
+        LOG.info("Submitting application in 'Flink Yarn Application Mode'.");
+        if (flinkConfig.get(PipelineOptions.JARS) == null) {
+            flinkConfig.set(
+                    PipelineOptions.JARS, 
Collections.singletonList(getFlinkCDCDistJarFromEnv()));
+        }
+        flinkConfig.set(
+                YarnConfigOptions.SHIP_FILES,
+                
additionalJars.stream().map(Path::toString).collect(Collectors.toList()));
+
+        ObjectMapper mapper = new ObjectMapper(new YAMLFactory());
+
+        Path pipelinePath = new Path(commandLine.getArgList().get(0));
+        FileSystem fileSystem = FileSystem.get(pipelinePath.toUri());
+        FSDataInputStream pipelineInStream = fileSystem.open(pipelinePath);
+
+        flinkConfig.set(
+                ApplicationConfiguration.APPLICATION_ARGS,
+                
Collections.singletonList(mapper.readTree(pipelineInStream).toString()));
+        YarnLogConfigUtil.setLogConfigFileInConfig(
+                flinkConfig, Joiner.on(File.separator).join(flinkHome, 
"conf"));
+
+        flinkConfig.set(
+                ApplicationConfiguration.APPLICATION_MAIN_CLASS,
+                "org.apache.flink.cdc.cli.CliExecutor");
+        final YarnClusterClientFactory yarnClusterClientFactory = new 
YarnClusterClientFactory();
+        final YarnClusterDescriptor descriptor =
+                yarnClusterClientFactory.createClusterDescriptor(flinkConfig);
+        ClusterSpecification specification =
+                yarnClusterClientFactory.getClusterSpecification(flinkConfig);
+        ApplicationConfiguration applicationConfiguration =
+                ApplicationConfiguration.fromConfiguration(flinkConfig);
+
+        ClusterClient<ApplicationId> client = null;
+        try {
+            ClusterClientProvider<ApplicationId> clusterClientProvider =
+                    descriptor.deployApplicationCluster(specification, 
applicationConfiguration);
+            client = clusterClientProvider.getClusterClient();
+            ApplicationId clusterId = client.getClusterId();
+            LOG.info("Deployment Flink CDC From Cluster ID {}", clusterId);
+            return new PipelineExecution.ExecutionInfo(
+                    clusterId.toString(), "submit job successful");
+        } catch (Exception e) {
+            if (client != null) {
+                client.shutDownCluster();
+            }
+            throw new RuntimeException("Failed to deploy Flink CDC job", e);
+        } finally {
+            descriptor.close();
+            if (client != null) {
+                client.close();
+            }
+        }
+    }
+
+    private String getFlinkCDCDistJarFromEnv() throws IOException {
+        String flinkCDCHomeFromEnvVar = System.getenv(FLINK_CDC_HOME_ENV_VAR);
+        Preconditions.checkNotNull(
+                flinkCDCHomeFromEnvVar,
+                "FLINK_CDC_HOME is not correctly set in environment variable, 
current FLINK_CDC_HOME is: "
+                        + FLINK_CDC_HOME_ENV_VAR);
+        Path flinkCDCLibPath = new Path(flinkCDCHomeFromEnvVar, "lib");
+        if (!flinkCDCLibPath.getFileSystem().exists(flinkCDCLibPath)
+                || 
!flinkCDCLibPath.getFileSystem().getFileStatus(flinkCDCLibPath).isDir()) {
+            throw new RuntimeException(
+                    "Flink cdc home lib is not file or not directory: "
+                            + 
flinkCDCLibPath.makeQualified(flinkCDCLibPath.getFileSystem()));
+        }
+
+        FileStatus[] fileStatuses = 
flinkCDCLibPath.getFileSystem().listStatus(flinkCDCLibPath);
+        Optional<Path> distJars =
+                Arrays.stream(fileStatuses)
+                        .filter(status -> !status.isDir())
+                        .filter(
+                                file ->
+                                        file.getPath()
+                                                .getName()
+                                                .matches(
+                                                        
"^flink-cdc-dist-(\\d+(\\.\\d+)*)(-SNAPSHOT)?\\.jar$"))

Review Comment:
   Move this to a static variable.



##########
flink-cdc-cli/src/main/java/org/apache/flink/cdc/cli/CliExecutor.java:
##########
@@ -19,85 +19,103 @@
 
 import org.apache.flink.cdc.cli.parser.PipelineDefinitionParser;
 import org.apache.flink.cdc.cli.parser.YamlPipelineDefinitionParser;
-import org.apache.flink.cdc.cli.utils.ConfigurationUtils;
-import org.apache.flink.cdc.cli.utils.FlinkEnvironmentUtils;
 import org.apache.flink.cdc.common.annotation.VisibleForTesting;
 import org.apache.flink.cdc.common.configuration.Configuration;
 import org.apache.flink.cdc.composer.PipelineComposer;
 import org.apache.flink.cdc.composer.PipelineDeploymentExecutor;
 import org.apache.flink.cdc.composer.PipelineExecution;
 import org.apache.flink.cdc.composer.definition.PipelineDef;
-import org.apache.flink.cdc.composer.flink.deployment.ComposeDeploymentFactory;
-import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings;
+import org.apache.flink.cdc.composer.flink.FlinkPipelineComposer;
+import org.apache.flink.cdc.composer.flink.deployment.ComposeDeployment;
+import 
org.apache.flink.cdc.composer.flink.deployment.K8SApplicationDeploymentExecutor;
+import 
org.apache.flink.cdc.composer.flink.deployment.YarnApplicationDeploymentExecutor;
+import org.apache.flink.configuration.DeploymentOptions;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 
 import org.apache.commons.cli.CommandLine;
 
-import java.nio.file.Path;
 import java.util.List;
 
+import static 
org.apache.flink.cdc.composer.flink.deployment.ComposeDeployment.REMOTE;
+
 /** Executor for doing the composing and submitting logic for {@link 
CliFrontend}. */
 public class CliExecutor {
 
     private final Path pipelineDefPath;
-    private final Configuration flinkConfig;
+    private final org.apache.flink.configuration.Configuration flinkConfig;
     private final Configuration globalPipelineConfig;
-    private final boolean useMiniCluster;
     private final List<Path> additionalJars;
-
+    private final Path flinkHome;
     private final CommandLine commandLine;
-
     private PipelineComposer composer = null;
 
-    private final SavepointRestoreSettings savepointSettings;
-
     public CliExecutor(
             CommandLine commandLine,
             Path pipelineDefPath,
-            Configuration flinkConfig,
+            org.apache.flink.configuration.Configuration flinkConfig,
             Configuration globalPipelineConfig,
-            boolean useMiniCluster,
             List<Path> additionalJars,
-            SavepointRestoreSettings savepointSettings) {
+            Path flinkHome) {
         this.commandLine = commandLine;
         this.pipelineDefPath = pipelineDefPath;
         this.flinkConfig = flinkConfig;
         this.globalPipelineConfig = globalPipelineConfig;
-        this.useMiniCluster = useMiniCluster;
         this.additionalJars = additionalJars;
-        this.savepointSettings = savepointSettings;
+        this.flinkHome = flinkHome;
     }
 
     public PipelineExecution.ExecutionInfo run() throws Exception {
         // Create Submit Executor to deployment flink cdc job Or Run Flink CDC 
Job
-        boolean isDeploymentMode = 
ConfigurationUtils.isDeploymentMode(commandLine);
-        if (isDeploymentMode) {
-            ComposeDeploymentFactory composeDeploymentFactory = new 
ComposeDeploymentFactory();
-            PipelineDeploymentExecutor composeExecutor =
-                    
composeDeploymentFactory.getFlinkComposeExecutor(commandLine);
-            org.apache.flink.configuration.Configuration configuration =
-                    
org.apache.flink.configuration.Configuration.fromMap(flinkConfig.toMap());
-            SavepointRestoreSettings.toConfiguration(savepointSettings, 
configuration);
-            return composeExecutor.deploy(commandLine, configuration, 
additionalJars);
-        } else {
-            // Run CDC Job And Parse pipeline definition file
-            PipelineDefinitionParser pipelineDefinitionParser = new 
YamlPipelineDefinitionParser();
-            PipelineDef pipelineDef =
-                    pipelineDefinitionParser.parse(pipelineDefPath, 
globalPipelineConfig);
-            // Create composer
-            PipelineComposer composer = getComposer();
-            // Compose pipeline
-            PipelineExecution execution = composer.compose(pipelineDef);
-            // Execute or submit the pipeline
-            return execution.execute();
+        String deploymentTargetStr = getDeploymentTarget();
+        ComposeDeployment deploymentTarget =
+                ComposeDeployment.getDeploymentFromName(getDeploymentTarget());

Review Comment:
   Nil:
   ```
   ComposeDeployment deploymentTarget =
           ComposeDeployment.getDeploymentFromName(deploymentTargetStr);
   ```



##########
flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestOnYarnEnvironment.java:
##########
@@ -0,0 +1,327 @@
+/*
+ * 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.flink.cdc.pipeline.tests.utils;
+
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.cdc.common.test.utils.TestUtils;
+import org.apache.flink.cdc.common.utils.Preconditions;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.core.testutils.CommonTestUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.MiniYARNCluster;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import static java.lang.Thread.sleep;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Fail.fail;
+
+/** Test environment running pipeline job on YARN mini-cluster. */
+public class PipelineTestOnYarnEnvironment extends TestLogger {
+    private static final Logger LOG = 
LoggerFactory.getLogger(PipelineTestOnYarnEnvironment.class);
+
+    protected static final YarnConfiguration YARN_CONFIGURATION;
+    private YarnClient yarnClient = null;
+    protected static MiniYARNCluster yarnCluster = null;
+
+    protected static final String TEST_CLUSTER_NAME_KEY = 
"flink-yarn-minicluster-name";
+    protected static final int NUM_NODEMANAGERS = 2;
+
+    protected static File yarnSiteXML = null;
+
+    @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+    private static final Duration yarnAppTerminateTimeout = 
Duration.ofSeconds(120);
+    private static final int sleepIntervalInMS = 100;
+
+    // copy from org.apache.flink.yarn.YarnTestBase
+    static {
+        YARN_CONFIGURATION = new YarnConfiguration();
+        
YARN_CONFIGURATION.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 
32);
+        YARN_CONFIGURATION.setInt(
+                YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
+                4096); // 4096 is the available memory anyways
+        YARN_CONFIGURATION.setBoolean(
+                YarnConfiguration.RM_SCHEDULER_INCLUDE_PORT_IN_NODE_NAME, 
true);
+        YARN_CONFIGURATION.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
+        
YARN_CONFIGURATION.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, 2);
+        
YARN_CONFIGURATION.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES,
 4);
+        YARN_CONFIGURATION.setInt(YarnConfiguration.DEBUG_NM_DELETE_DELAY_SEC, 
3600);
+        
YARN_CONFIGURATION.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, false);
+        YARN_CONFIGURATION.setInt(
+                YarnConfiguration.NM_VCORES, 666); // memory is overwritten in 
the MiniYARNCluster.
+        // so we have to change the number of cores for testing.
+        YARN_CONFIGURATION.setFloat(
+                YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, 
99.0F);
+        YARN_CONFIGURATION.set(YarnConfiguration.YARN_APPLICATION_CLASSPATH, 
getYarnClasspath());
+        YARN_CONFIGURATION.setInt(
+                YarnConfiguration.RESOURCEMANAGER_CONNECT_RETRY_INTERVAL_MS, 
1000);
+        
YARN_CONFIGURATION.setInt(YarnConfiguration.RESOURCEMANAGER_CONNECT_MAX_WAIT_MS,
 5000);
+        YARN_CONFIGURATION.set(TEST_CLUSTER_NAME_KEY, 
"flink-yarn-tests-application");
+    }
+
+    @Before
+    public void setupYarnClient() throws Exception {
+        if (yarnClient == null) {
+            yarnClient = YarnClient.createYarnClient();
+            yarnClient.init(getYarnConfiguration());
+            yarnClient.start();
+        }
+    }
+
+    @After
+    public void shutdownYarnClient() {
+        yarnClient.stop();
+    }
+
+    @AfterClass
+    public static void teardown() {
+
+        if (yarnCluster != null) {
+            LOG.info("Stopping MiniYarn Cluster");
+            yarnCluster.stop();
+            yarnCluster = null;
+        }
+
+        // Unset FLINK_CONF_DIR, as it might change the behavior of other tests
+        Map<String, String> map = new HashMap<>(System.getenv());
+        map.remove(ConfigConstants.ENV_FLINK_CONF_DIR);
+        map.remove("YARN_CONF_DIR");
+        map.remove("IN_TESTS");
+        CommonTestUtils.setEnv(map);
+
+        if (yarnSiteXML != null) {
+            yarnSiteXML.delete();
+        }
+    }
+
+    protected static YarnConfiguration getYarnConfiguration() {
+        return YARN_CONFIGURATION;
+    }
+
+    public static void startMiniYARNCluster() {
+        try {
+            LOG.info("Starting up MiniYARNCluster");
+            if (yarnCluster == null) {
+                final String testName =
+                        
YARN_CONFIGURATION.get(PipelineTestOnYarnEnvironment.TEST_CLUSTER_NAME_KEY);
+                yarnCluster =
+                        new MiniYARNCluster(
+                                testName == null ? "YarnTest_" + 
UUID.randomUUID() : testName,
+                                NUM_NODEMANAGERS,
+                                1,
+                                1);
+
+                yarnCluster.init(YARN_CONFIGURATION);
+                yarnCluster.start();
+            }
+
+            File targetTestClassesFolder = new File("target/test-classes");
+            writeYarnSiteConfigXML(YARN_CONFIGURATION, 
targetTestClassesFolder);
+
+            Map<String, String> map = new HashMap<String, 
String>(System.getenv());
+            map.put(
+                    "IN_TESTS",
+                    "yes we are in tests"); // see YarnClusterDescriptor() for 
more infos
+            map.put("YARN_CONF_DIR", 
targetTestClassesFolder.getAbsolutePath());
+            map.put("MAX_LOG_FILE_NUMBER", "10");
+            CommonTestUtils.setEnv(map);
+
+            
assertThat(yarnCluster.getServiceState()).isEqualTo(Service.STATE.STARTED);
+            // wait for the nodeManagers to connect
+            while (!yarnCluster.waitForNodeManagersToConnect(500)) {
+                LOG.info("Waiting for Nodemanagers to connect");
+            }
+        } catch (Exception ex) {
+            ex.printStackTrace();
+            LOG.error("setup failure", ex);
+            fail("");
+        }
+    }
+
+    // write yarn-site.xml to target/test-classes so that flink pick can pick 
up this when
+    // initializing YarnClient properly from classpath
+    public static void writeYarnSiteConfigXML(Configuration yarnConf, File 
targetFolder)
+            throws IOException {
+        yarnSiteXML = new File(targetFolder, "/yarn-site.xml");
+        try (FileWriter writer = new FileWriter(yarnSiteXML)) {
+            yarnConf.writeXml(writer);
+            writer.flush();
+        }
+    }
+
+    public String submitPipelineJob(String pipelineJob, Path... jars) throws 
Exception {
+        ProcessBuilder processBuilder = new ProcessBuilder();
+        Map<String, String> env = getEnv();
+        processBuilder.environment().putAll(env);
+        Path yamlScript = 
temporaryFolder.newFile("mysql-to-values.yml").toPath();
+        Files.write(yamlScript, pipelineJob.getBytes());
+
+        List<String> commandList = new ArrayList<>();
+        commandList.add(env.get("FLINK_CDC_HOME") + "/bin/flink-cdc.sh");
+        commandList.add("-t");
+        commandList.add("yarn-application");
+        commandList.add(yamlScript.toAbsolutePath().toString());
+        for (Path jar : jars) {
+            commandList.add("--jar");
+            commandList.add(jar.toString());
+        }
+
+        processBuilder.command(commandList);
+        LOG.info("starting flink-cdc task with flink on yarn-application");
+        Process process = processBuilder.start();
+        process.waitFor();
+        String applicationIdStr = getApplicationId(process);
+        Preconditions.checkNotNull(
+                applicationIdStr, "applicationId should not be null, please 
check logs");
+        ApplicationId applicationId = 
ApplicationId.fromString(applicationIdStr);
+        waitApplicationFinished(applicationId, yarnAppTerminateTimeout, 
sleepIntervalInMS);
+        LOG.info("started flink-cdc task with flink on yarn-application");
+        return applicationIdStr;
+    }
+
+    public Map<String, String> getEnv() {
+        Path flinkHome =
+                TestUtils.getResource(
+                        "flink-\\d+(\\.\\d+)*$",
+                        
"flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/target");
+        Map<String, String> env = new HashMap<>();
+        env.put("FLINK_HOME", flinkHome.toString());
+        env.put("FLINK_CONF_DIR", flinkHome.resolve("conf").toString());
+        addFlinkConf(flinkHome.resolve("conf").resolve("config.yaml"));
+        Path flinkcdcHome =
+                TestUtils.getResource("flink-cdc-\\d+(\\.\\d+)*(-SNAPSHOT)?$", 
"flink-cdc-dist");
+        env.put("FLINK_CDC_HOME", flinkcdcHome.toString());
+        env.put("HADOOP_CLASSPATH", getYarnClasspath());
+        return env;
+    }
+
+    // TODO Maybe pipeline.yml should support adding flink conf
+    public void addFlinkConf(Path flinkConf) {

Review Comment:
   We do this in another pull-request and remove TODO here.



##########
flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/deployment/K8SApplicationDeploymentExecutor.java:
##########
@@ -61,7 +61,7 @@ public PipelineExecution.ExecutionInfo deploy(
         flinkConfig.set(ApplicationConfiguration.APPLICATION_ARGS, 
commandLine.getArgList());
         flinkConfig.set(
                 ApplicationConfiguration.APPLICATION_MAIN_CLASS,
-                "org.apache.flink.cdc.cli.CliFrontend");
+                "org.apache.flink.cdc.cli.CliExecutor");

Review Comment:
   Ditto, we should use a unified variable to maintain this.



##########
flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/deployment/YarnApplicationDeploymentExecutor.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.flink.cdc.composer.flink.deployment;
+
+import org.apache.flink.cdc.common.utils.Preconditions;
+import org.apache.flink.cdc.composer.PipelineDeploymentExecutor;
+import org.apache.flink.cdc.composer.PipelineExecution;
+import org.apache.flink.client.deployment.ClusterSpecification;
+import org.apache.flink.client.deployment.application.ApplicationConfiguration;
+import org.apache.flink.client.program.ClusterClient;
+import org.apache.flink.client.program.ClusterClientProvider;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.PipelineOptions;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.fs.FileStatus;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.yarn.YarnClusterClientFactory;
+import org.apache.flink.yarn.YarnClusterDescriptor;
+import org.apache.flink.yarn.configuration.YarnConfigOptions;
+import org.apache.flink.yarn.configuration.YarnLogConfigUtil;
+
+import org.apache.flink.shaded.guava31.com.google.common.base.Joiner;
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+/** Deploy flink cdc job by yarn application mode. */
+public class YarnApplicationDeploymentExecutor implements 
PipelineDeploymentExecutor {
+    private static final Logger LOG =
+            LoggerFactory.getLogger(YarnApplicationDeploymentExecutor.class);
+    private static final String FLINK_CDC_HOME_ENV_VAR = "FLINK_CDC_HOME";
+
+    @Override
+    public PipelineExecution.ExecutionInfo deploy(
+            CommandLine commandLine,
+            Configuration flinkConfig,
+            List<Path> additionalJars,
+            Path flinkHome)
+            throws Exception {
+        LOG.info("Submitting application in 'Flink Yarn Application Mode'.");
+        if (flinkConfig.get(PipelineOptions.JARS) == null) {
+            flinkConfig.set(
+                    PipelineOptions.JARS, 
Collections.singletonList(getFlinkCDCDistJarFromEnv()));
+        }
+        flinkConfig.set(
+                YarnConfigOptions.SHIP_FILES,
+                
additionalJars.stream().map(Path::toString).collect(Collectors.toList()));
+
+        ObjectMapper mapper = new ObjectMapper(new YAMLFactory());
+
+        Path pipelinePath = new Path(commandLine.getArgList().get(0));
+        FileSystem fileSystem = FileSystem.get(pipelinePath.toUri());
+        FSDataInputStream pipelineInStream = fileSystem.open(pipelinePath);
+
+        flinkConfig.set(
+                ApplicationConfiguration.APPLICATION_ARGS,
+                
Collections.singletonList(mapper.readTree(pipelineInStream).toString()));
+        YarnLogConfigUtil.setLogConfigFileInConfig(
+                flinkConfig, Joiner.on(File.separator).join(flinkHome, 
"conf"));
+
+        flinkConfig.set(
+                ApplicationConfiguration.APPLICATION_MAIN_CLASS,
+                "org.apache.flink.cdc.cli.CliExecutor");

Review Comment:
   Please move this hard-coding variable to a static variable.



##########
flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/deployment/YarnApplicationDeploymentExecutor.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.flink.cdc.composer.flink.deployment;
+
+import org.apache.flink.cdc.common.utils.Preconditions;
+import org.apache.flink.cdc.composer.PipelineDeploymentExecutor;
+import org.apache.flink.cdc.composer.PipelineExecution;
+import org.apache.flink.client.deployment.ClusterSpecification;
+import org.apache.flink.client.deployment.application.ApplicationConfiguration;
+import org.apache.flink.client.program.ClusterClient;
+import org.apache.flink.client.program.ClusterClientProvider;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.PipelineOptions;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.fs.FileStatus;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.yarn.YarnClusterClientFactory;
+import org.apache.flink.yarn.YarnClusterDescriptor;
+import org.apache.flink.yarn.configuration.YarnConfigOptions;
+import org.apache.flink.yarn.configuration.YarnLogConfigUtil;
+
+import org.apache.flink.shaded.guava31.com.google.common.base.Joiner;
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+/** Deploy flink cdc job by yarn application mode. */
+public class YarnApplicationDeploymentExecutor implements 
PipelineDeploymentExecutor {
+    private static final Logger LOG =
+            LoggerFactory.getLogger(YarnApplicationDeploymentExecutor.class);
+    private static final String FLINK_CDC_HOME_ENV_VAR = "FLINK_CDC_HOME";
+
+    @Override
+    public PipelineExecution.ExecutionInfo deploy(
+            CommandLine commandLine,
+            Configuration flinkConfig,
+            List<Path> additionalJars,
+            Path flinkHome)
+            throws Exception {
+        LOG.info("Submitting application in 'Flink Yarn Application Mode'.");
+        if (flinkConfig.get(PipelineOptions.JARS) == null) {
+            flinkConfig.set(
+                    PipelineOptions.JARS, 
Collections.singletonList(getFlinkCDCDistJarFromEnv()));
+        }
+        flinkConfig.set(
+                YarnConfigOptions.SHIP_FILES,
+                
additionalJars.stream().map(Path::toString).collect(Collectors.toList()));
+
+        ObjectMapper mapper = new ObjectMapper(new YAMLFactory());
+
+        Path pipelinePath = new Path(commandLine.getArgList().get(0));
+        FileSystem fileSystem = FileSystem.get(pipelinePath.toUri());
+        FSDataInputStream pipelineInStream = fileSystem.open(pipelinePath);
+
+        flinkConfig.set(
+                ApplicationConfiguration.APPLICATION_ARGS,
+                
Collections.singletonList(mapper.readTree(pipelineInStream).toString()));
+        YarnLogConfigUtil.setLogConfigFileInConfig(
+                flinkConfig, Joiner.on(File.separator).join(flinkHome, 
"conf"));
+
+        flinkConfig.set(
+                ApplicationConfiguration.APPLICATION_MAIN_CLASS,
+                "org.apache.flink.cdc.cli.CliExecutor");
+        final YarnClusterClientFactory yarnClusterClientFactory = new 
YarnClusterClientFactory();
+        final YarnClusterDescriptor descriptor =
+                yarnClusterClientFactory.createClusterDescriptor(flinkConfig);
+        ClusterSpecification specification =
+                yarnClusterClientFactory.getClusterSpecification(flinkConfig);
+        ApplicationConfiguration applicationConfiguration =
+                ApplicationConfiguration.fromConfiguration(flinkConfig);
+
+        ClusterClient<ApplicationId> client = null;
+        try {
+            ClusterClientProvider<ApplicationId> clusterClientProvider =
+                    descriptor.deployApplicationCluster(specification, 
applicationConfiguration);
+            client = clusterClientProvider.getClusterClient();
+            ApplicationId clusterId = client.getClusterId();
+            LOG.info("Deployment Flink CDC From Cluster ID {}", clusterId);
+            return new PipelineExecution.ExecutionInfo(
+                    clusterId.toString(), "submit job successful");
+        } catch (Exception e) {
+            if (client != null) {
+                client.shutDownCluster();
+            }
+            throw new RuntimeException("Failed to deploy Flink CDC job", e);
+        } finally {
+            descriptor.close();
+            if (client != null) {
+                client.close();
+            }
+        }
+    }
+
+    private String getFlinkCDCDistJarFromEnv() throws IOException {
+        String flinkCDCHomeFromEnvVar = System.getenv(FLINK_CDC_HOME_ENV_VAR);
+        Preconditions.checkNotNull(
+                flinkCDCHomeFromEnvVar,
+                "FLINK_CDC_HOME is not correctly set in environment variable, 
current FLINK_CDC_HOME is: "
+                        + FLINK_CDC_HOME_ENV_VAR);
+        Path flinkCDCLibPath = new Path(flinkCDCHomeFromEnvVar, "lib");
+        if (!flinkCDCLibPath.getFileSystem().exists(flinkCDCLibPath)
+                || 
!flinkCDCLibPath.getFileSystem().getFileStatus(flinkCDCLibPath).isDir()) {
+            throw new RuntimeException(
+                    "Flink cdc home lib is not file or not directory: "
+                            + 
flinkCDCLibPath.makeQualified(flinkCDCLibPath.getFileSystem()));
+        }
+
+        FileStatus[] fileStatuses = 
flinkCDCLibPath.getFileSystem().listStatus(flinkCDCLibPath);
+        Optional<Path> distJars =
+                Arrays.stream(fileStatuses)
+                        .filter(status -> !status.isDir())
+                        .filter(
+                                file ->
+                                        file.getPath()
+                                                .getName()
+                                                .matches(
+                                                        
"^flink-cdc-dist-(\\d+(\\.\\d+)*)(-SNAPSHOT)?\\.jar$"))
+                        .map(FileStatus::getPath)
+                        .findFirst();
+
+        if (distJars.isPresent()) {
+            Path path = 
distJars.get().makeQualified(distJars.get().getFileSystem());
+            return path.toString();
+        } else {
+            throw new FileNotFoundException(
+                    "Failed to fetch Flink CDC dist jar from path + "
+                            + distJars.get()

Review Comment:
   Will cause NullPointException here?



##########
flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestOnYarnEnvironment.java:
##########
@@ -0,0 +1,327 @@
+/*
+ * 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.flink.cdc.pipeline.tests.utils;
+
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.cdc.common.test.utils.TestUtils;
+import org.apache.flink.cdc.common.utils.Preconditions;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.core.testutils.CommonTestUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.Service;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.MiniYARNCluster;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import static java.lang.Thread.sleep;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Fail.fail;
+
+/** Test environment running pipeline job on YARN mini-cluster. */
+public class PipelineTestOnYarnEnvironment extends TestLogger {
+    private static final Logger LOG = 
LoggerFactory.getLogger(PipelineTestOnYarnEnvironment.class);
+
+    protected static final YarnConfiguration YARN_CONFIGURATION;
+    private YarnClient yarnClient = null;
+    protected static MiniYARNCluster yarnCluster = null;
+
+    protected static final String TEST_CLUSTER_NAME_KEY = 
"flink-yarn-minicluster-name";
+    protected static final int NUM_NODEMANAGERS = 2;
+
+    protected static File yarnSiteXML = null;
+
+    @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+    private static final Duration yarnAppTerminateTimeout = 
Duration.ofSeconds(120);
+    private static final int sleepIntervalInMS = 100;
+
+    // copy from org.apache.flink.yarn.YarnTestBase
+    static {
+        YARN_CONFIGURATION = new YarnConfiguration();
+        
YARN_CONFIGURATION.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 
32);
+        YARN_CONFIGURATION.setInt(
+                YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB,
+                4096); // 4096 is the available memory anyways
+        YARN_CONFIGURATION.setBoolean(
+                YarnConfiguration.RM_SCHEDULER_INCLUDE_PORT_IN_NODE_NAME, 
true);
+        YARN_CONFIGURATION.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);
+        
YARN_CONFIGURATION.setInt(YarnConfiguration.RM_MAX_COMPLETED_APPLICATIONS, 2);
+        
YARN_CONFIGURATION.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES,
 4);
+        YARN_CONFIGURATION.setInt(YarnConfiguration.DEBUG_NM_DELETE_DELAY_SEC, 
3600);
+        
YARN_CONFIGURATION.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, false);
+        YARN_CONFIGURATION.setInt(
+                YarnConfiguration.NM_VCORES, 666); // memory is overwritten in 
the MiniYARNCluster.
+        // so we have to change the number of cores for testing.
+        YARN_CONFIGURATION.setFloat(
+                YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, 
99.0F);
+        YARN_CONFIGURATION.set(YarnConfiguration.YARN_APPLICATION_CLASSPATH, 
getYarnClasspath());
+        YARN_CONFIGURATION.setInt(
+                YarnConfiguration.RESOURCEMANAGER_CONNECT_RETRY_INTERVAL_MS, 
1000);
+        
YARN_CONFIGURATION.setInt(YarnConfiguration.RESOURCEMANAGER_CONNECT_MAX_WAIT_MS,
 5000);
+        YARN_CONFIGURATION.set(TEST_CLUSTER_NAME_KEY, 
"flink-yarn-tests-application");
+    }
+
+    @Before
+    public void setupYarnClient() throws Exception {
+        if (yarnClient == null) {
+            yarnClient = YarnClient.createYarnClient();
+            yarnClient.init(getYarnConfiguration());
+            yarnClient.start();
+        }
+    }
+
+    @After
+    public void shutdownYarnClient() {
+        yarnClient.stop();
+    }
+
+    @AfterClass
+    public static void teardown() {
+
+        if (yarnCluster != null) {
+            LOG.info("Stopping MiniYarn Cluster");
+            yarnCluster.stop();
+            yarnCluster = null;
+        }
+
+        // Unset FLINK_CONF_DIR, as it might change the behavior of other tests
+        Map<String, String> map = new HashMap<>(System.getenv());
+        map.remove(ConfigConstants.ENV_FLINK_CONF_DIR);
+        map.remove("YARN_CONF_DIR");
+        map.remove("IN_TESTS");
+        CommonTestUtils.setEnv(map);
+
+        if (yarnSiteXML != null) {
+            yarnSiteXML.delete();
+        }
+    }
+
+    protected static YarnConfiguration getYarnConfiguration() {
+        return YARN_CONFIGURATION;
+    }
+
+    public static void startMiniYARNCluster() {
+        try {
+            LOG.info("Starting up MiniYARNCluster");
+            if (yarnCluster == null) {
+                final String testName =
+                        
YARN_CONFIGURATION.get(PipelineTestOnYarnEnvironment.TEST_CLUSTER_NAME_KEY);
+                yarnCluster =
+                        new MiniYARNCluster(
+                                testName == null ? "YarnTest_" + 
UUID.randomUUID() : testName,
+                                NUM_NODEMANAGERS,
+                                1,
+                                1);
+
+                yarnCluster.init(YARN_CONFIGURATION);
+                yarnCluster.start();
+            }
+
+            File targetTestClassesFolder = new File("target/test-classes");
+            writeYarnSiteConfigXML(YARN_CONFIGURATION, 
targetTestClassesFolder);
+
+            Map<String, String> map = new HashMap<String, 
String>(System.getenv());
+            map.put(
+                    "IN_TESTS",
+                    "yes we are in tests"); // see YarnClusterDescriptor() for 
more infos
+            map.put("YARN_CONF_DIR", 
targetTestClassesFolder.getAbsolutePath());
+            map.put("MAX_LOG_FILE_NUMBER", "10");
+            CommonTestUtils.setEnv(map);
+
+            
assertThat(yarnCluster.getServiceState()).isEqualTo(Service.STATE.STARTED);
+            // wait for the nodeManagers to connect
+            while (!yarnCluster.waitForNodeManagersToConnect(500)) {
+                LOG.info("Waiting for Nodemanagers to connect");
+            }
+        } catch (Exception ex) {
+            ex.printStackTrace();
+            LOG.error("setup failure", ex);
+            fail("");

Review Comment:
   Could be simplified by  fail("setup failure", ex); here?



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to