[WIP][Feature]  Support submit spark jar and sql tasks to yarn (#3537)

* [Improve] Console core module code optimization

* [Feature] Support submit spark jar tasks to yarn

* Optimized code

* pom add the spark module

* Supports spark sql to submit yarn

* Resolving code conflicts

* Modify mysql-schema.sql

* Optimized code

* Modify comment

* Modify the SqlCommandParser
diff --git a/pom.xml b/pom.xml
index d66486a..329e771 100644
--- a/pom.xml
+++ b/pom.xml
@@ -79,6 +79,7 @@
     <modules>
         <module>streampark-common</module>
         <module>streampark-flink</module>
+        <module>streampark-spark</module>
         <module>streampark-console</module>
         <module>streampark-tests</module>
     </modules>
@@ -98,7 +99,7 @@
         <flink.shaded.version>1.8.1</flink.shaded.version>
         <streampark.shaded.version>1.0.0</streampark.shaded.version>
         <streampark.flink.shims.version>1.14</streampark.flink.shims.version>
-        <spark.version>3.2.0</spark.version>
+        <spark.version>3.1.2</spark.version>
         <scalikejdbc.version>4.0.0</scalikejdbc.version>
         <hive.version>2.3.4</hive.version>
         <hadoop.version>3.3.4</hadoop.version>
diff --git a/streampark-common/src/main/java/org/apache/streampark/common/Constant.java b/streampark-common/src/main/java/org/apache/streampark/common/Constant.java
index 553d68b..ddadcf7 100644
--- a/streampark-common/src/main/java/org/apache/streampark/common/Constant.java
+++ b/streampark-common/src/main/java/org/apache/streampark/common/Constant.java
@@ -48,5 +48,8 @@
   public static final String STREAMPARK_FLINKSQL_CLIENT_CLASS =
       "org.apache.streampark.flink.cli.SqlClient";
 
+  public static final String STREAMPARK_SPARKSQL_CLIENT_CLASS =
+      "org.apache.streampark.spark.cli.SqlClient";
+
   public static final String PYTHON_EXECUTABLE = "venv.zip/venv/bin/python3";
 }
diff --git a/streampark-common/src/main/java/org/apache/streampark/common/enums/SparkExecutionMode.java b/streampark-common/src/main/java/org/apache/streampark/common/enums/SparkExecutionMode.java
new file mode 100644
index 0000000..9452f92
--- /dev/null
+++ b/streampark-common/src/main/java/org/apache/streampark/common/enums/SparkExecutionMode.java
@@ -0,0 +1,119 @@
+/*
+ * 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.streampark.common.enums;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+/** Spark execution mode enum. */
+public enum SparkExecutionMode {
+
+  /** Unknown Mode */
+  UNKNOWN(-1, "Unknown"),
+
+  /** Local mode */
+  LOCAL(0, "local"),
+
+  /** remote */
+  REMOTE(1, "remote"),
+
+  /** yarn-cluster mode */
+  YARN_CLUSTER(2, "yarn-cluster"),
+
+  /** yarn client */
+  YARN_CLIENT(3, "yarn-client");
+  private final Integer mode;
+
+  private final String name;
+
+  SparkExecutionMode(@Nonnull Integer mode, @Nonnull String name) {
+    this.mode = mode;
+    this.name = name;
+  }
+
+  /**
+   * Try to resolve the mode value into {@link SparkExecutionMode}.
+   *
+   * @param value The mode value of potential spark execution mode.
+   * @return The parsed spark execution mode enum.
+   */
+  @Nonnull
+  public static SparkExecutionMode of(@Nullable Integer value) {
+    for (SparkExecutionMode mode : values()) {
+      if (mode.mode.equals(value)) {
+        return mode;
+      }
+    }
+    return SparkExecutionMode.UNKNOWN;
+  }
+
+  /**
+   * Try to resolve the mode name into {@link SparkExecutionMode}.
+   *
+   * @param name The mode name of potential spark execution mode.
+   * @return The parsed spark execution mode enum.
+   */
+  @Nonnull
+  public static SparkExecutionMode of(@Nullable String name) {
+    for (SparkExecutionMode mode : values()) {
+      if (mode.name.equals(name)) {
+        return mode;
+      }
+    }
+    return SparkExecutionMode.UNKNOWN;
+  }
+
+  public int getMode() {
+    return mode;
+  }
+
+  @Nonnull
+  public String getName() {
+    return name;
+  }
+
+  /**
+   * Judge the given mode whether is yarn mode.
+   *
+   * @param mode The given mode.
+   * @return The judged result.
+   */
+  public static boolean isYarnMode(@Nullable SparkExecutionMode mode) {
+    return YARN_CLUSTER == mode || YARN_CLIENT == mode;
+  }
+
+  /**
+   * Judge the mode value whether is yarn execution mode.
+   *
+   * @param value The mode value of potential spark execution mode.
+   * @return The judged result.
+   */
+  public static boolean isYarnMode(@Nullable Integer value) {
+    return isYarnMode(of(value));
+  }
+
+  /** Judge the given spark execution mode value whether is remote execution mode. */
+  public static boolean isRemoteMode(@Nullable Integer value) {
+    return isRemoteMode(of(value));
+  }
+
+  /** Judge the given spark execution mode whether is remote execution mode. */
+  public static boolean isRemoteMode(@Nullable SparkExecutionMode mode) {
+    return REMOTE == mode;
+  }
+}
diff --git a/streampark-common/src/main/scala/org/apache/streampark/common/conf/ConfigKeys.scala b/streampark-common/src/main/scala/org/apache/streampark/common/conf/ConfigKeys.scala
index f2fb98f..fb189db 100644
--- a/streampark-common/src/main/scala/org/apache/streampark/common/conf/ConfigKeys.scala
+++ b/streampark-common/src/main/scala/org/apache/streampark/common/conf/ConfigKeys.scala
@@ -80,6 +80,8 @@
 
   val KEY_FLINK_TABLE_PREFIX = "flink.table."
 
+  val KEY_SPARK_PROPERTY_PREFIX = "spark.property."
+
   val KEY_APP_PREFIX = "app."
 
   val KEY_SQL_PREFIX = "sql."
diff --git a/streampark-common/src/main/scala/org/apache/streampark/common/conf/SparkVersion.scala b/streampark-common/src/main/scala/org/apache/streampark/common/conf/SparkVersion.scala
new file mode 100644
index 0000000..b22e44d
--- /dev/null
+++ b/streampark-common/src/main/scala/org/apache/streampark/common/conf/SparkVersion.scala
@@ -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.streampark.common.conf
+
+import org.apache.streampark.common.util.{CommandUtils, Logger}
+
+import org.apache.commons.lang3.StringUtils
+
+import java.io.File
+import java.net.{URL => NetURL}
+import java.util.concurrent.atomic.{AtomicBoolean, AtomicReference}
+import java.util.function.Consumer
+import java.util.regex.Pattern
+
+import scala.collection.convert.ImplicitConversions._
+import scala.collection.mutable
+
+/** @param sparkHome actual spark home that must be a readable local path */
+class SparkVersion(val sparkHome: String) extends java.io.Serializable with Logger {
+
+  private[this] lazy val SPARK_VER_PATTERN = Pattern.compile("^(\\d+\\.\\d+)(\\.)?.*$")
+
+  private[this] lazy val SPARK_VERSION_PATTERN = Pattern.compile("(version) (\\d+\\.\\d+\\.\\d+)")
+
+  private[this] lazy val SPARK_SCALA_VERSION_PATTERN =
+    Pattern.compile("^spark-core_(.*)-[0-9].*.jar$")
+
+  lazy val scalaVersion: String = {
+    val matcher = SPARK_SCALA_VERSION_PATTERN.matcher(sparkCoreJar.getName)
+    if (matcher.matches()) {
+      matcher.group(1);
+    } else {
+      "2.12"
+    }
+  }
+
+  def checkVersion(throwException: Boolean = true): Boolean = {
+    version.split("\\.").map(_.trim.toInt) match {
+      case Array(3, v, _) if v >= 1 && v <= 3 => true
+      case _ =>
+        if (throwException) {
+          throw new UnsupportedOperationException(s"Unsupported flink version: $version")
+        } else {
+          false
+        }
+    }
+  }
+
+  lazy val fullVersion: String = s"${version}_$scalaVersion"
+
+  lazy val sparkLib: File = {
+    require(sparkHome != null, "[StreamPark] sparkHome must not be null.")
+    require(new File(sparkHome).exists(), "[StreamPark] sparkHome must be exists.")
+    val lib = new File(s"$sparkHome/jars")
+    require(
+      lib.exists() && lib.isDirectory,
+      s"[StreamPark] $sparkHome/lib must be exists and must be directory.")
+    lib
+  }
+
+  lazy val sparkLibs: List[NetURL] = sparkLib.listFiles().map(_.toURI.toURL).toList
+
+  lazy val majorVersion: String = {
+    if (version == null) {
+      null
+    } else {
+      val matcher = SPARK_VER_PATTERN.matcher(version)
+      matcher.matches()
+      matcher.group(1)
+    }
+  }
+
+  lazy val version: String = {
+    val sparkVersion = new AtomicReference[String]
+    val cmd = List(s"$sparkHome/bin/spark-submit --version")
+    val success = new AtomicBoolean(false)
+    val buffer = new mutable.StringBuilder
+    CommandUtils.execute(
+      sparkHome,
+      cmd,
+      new Consumer[String]() {
+        override def accept(out: String): Unit = {
+          buffer.append(out).append("\n")
+          val matcher = SPARK_VERSION_PATTERN.matcher(out)
+          if (matcher.find && StringUtils.isBlank(sparkVersion.get())) {
+            success.set(true)
+            sparkVersion.set(matcher.group(2))
+          }
+        }
+      }
+    )
+    logInfo(buffer.toString())
+    if (!success.get()) {
+      throw new IllegalStateException(s"[StreamPark] parse spark version failed. $buffer")
+    }
+    buffer.clear()
+    sparkVersion.get
+  }
+
+  lazy val sparkCoreJar: File = {
+    val distJar = sparkLib.listFiles().filter(_.getName.matches("spark-core.*\\.jar"))
+    distJar match {
+      case x if x.isEmpty =>
+        throw new IllegalArgumentException(s"[StreamPark] can no found spark-core jar in $sparkLib")
+      case x if x.length > 1 =>
+        throw new IllegalArgumentException(
+          s"[StreamPark] found multiple spark-core jar in $sparkLib")
+      case _ =>
+    }
+    distJar.head
+  }
+
+  override def toString: String =
+    s"""
+       |----------------------------------------- spark version -----------------------------------
+       |     sparkHome    : $sparkHome
+       |     sparkVersion : $version
+       |     scalaVersion : $scalaVersion
+       |-------------------------------------------------------------------------------------------
+       |""".stripMargin
+
+}
diff --git a/streampark-common/src/main/scala/org/apache/streampark/common/conf/Workspace.scala b/streampark-common/src/main/scala/org/apache/streampark/common/conf/Workspace.scala
index c447afc..f383bbb 100644
--- a/streampark-common/src/main/scala/org/apache/streampark/common/conf/Workspace.scala
+++ b/streampark-common/src/main/scala/org/apache/streampark/common/conf/Workspace.scala
@@ -110,6 +110,8 @@
 
   lazy val APP_FLINK = s"$WORKSPACE/flink"
 
+  lazy val APP_SPARK = s"$WORKSPACE/spark"
+
   lazy val APP_BACKUPS = s"$WORKSPACE/backups"
 
   lazy val APP_SAVEPOINTS = s"$WORKSPACE/savepoints"
diff --git a/streampark-console/streampark-console-service/pom.xml b/streampark-console/streampark-console-service/pom.xml
index c483c16..f6b48a6 100644
--- a/streampark-console/streampark-console-service/pom.xml
+++ b/streampark-console/streampark-console-service/pom.xml
@@ -369,6 +369,12 @@
 
         <dependency>
             <groupId>org.apache.streampark</groupId>
+            <artifactId>streampark-spark-client-api_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.streampark</groupId>
             <artifactId>streampark-flink-kubernetes_${scala.binary.version}</artifactId>
             <version>${project.version}</version>
         </dependency>
@@ -387,6 +393,12 @@
 
         <dependency>
             <groupId>org.apache.streampark</groupId>
+            <artifactId>streampark-spark-sqlclient_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.streampark</groupId>
             <artifactId>streampark-flink-sql-gateway-base</artifactId>
             <version>${project.version}</version>
         </dependency>
@@ -591,6 +603,13 @@
                             <version>${project.version}</version>
                             <outputDirectory>${project.build.directory}/lib</outputDirectory>
                         </dependency>
+                        <!-- spark-submit-core -->
+                        <dependency>
+                            <groupId>org.apache.streampark</groupId>
+                            <artifactId>streampark-spark-client-core_${scala.binary.version}</artifactId>
+                            <version>${project.version}</version>
+                            <outputDirectory>${project.build.directory}/lib</outputDirectory>
+                        </dependency>
                     </artifactItems>
                 </configuration>
                 <executions>
diff --git a/streampark-console/streampark-console-service/src/main/assembly/assembly.xml b/streampark-console/streampark-console-service/src/main/assembly/assembly.xml
index 408547f..b8f82cc 100644
--- a/streampark-console/streampark-console-service/src/main/assembly/assembly.xml
+++ b/streampark-console/streampark-console-service/src/main/assembly/assembly.xml
@@ -31,6 +31,7 @@
             <outputDirectory>lib</outputDirectory>
             <excludes>
                 <exclude>org.apache.streampark:streampark-flink-sqlclient_${scala.binary.version}</exclude>
+                <exclude>org.apache.streampark:streampark-spark-sqlclient_${scala.binary.version}</exclude>
                 <exclude>javax.servlet:servlet-api</exclude>
             </excludes>
         </dependencySet>
@@ -38,6 +39,7 @@
             <outputDirectory>client</outputDirectory>
             <includes>
                 <include>org.apache.streampark:streampark-flink-sqlclient_${scala.binary.version}</include>
+                <include>org.apache.streampark:streampark-spark-sqlclient_${scala.binary.version}</include>
             </includes>
         </dependencySet>
     </dependencySets>
diff --git a/streampark-console/streampark-console-service/src/main/assembly/script/schema/mysql-schema.sql b/streampark-console/streampark-console-service/src/main/assembly/script/schema/mysql-schema.sql
index 08fce99..e994a1e 100644
--- a/streampark-console/streampark-console-service/src/main/assembly/script/schema/mysql-schema.sql
+++ b/streampark-console/streampark-console-service/src/main/assembly/script/schema/mysql-schema.sql
@@ -557,4 +557,97 @@
 `modify_time` datetime not null default current_timestamp comment 'modify time'
 );
 
+
+-- ----------------------------
+-- table structure for t_spark_env
+-- ----------------------------
+drop table if exists `t_spark_env`;
+create table `t_spark_env` (
+                               `id` bigint not null auto_increment comment 'id',
+                               `spark_name` varchar(128) collate utf8mb4_general_ci not null comment 'spark instance name',
+                               `spark_home` varchar(255) collate utf8mb4_general_ci not null comment 'spark home path',
+                               `version` varchar(64) collate utf8mb4_general_ci not null comment 'spark version',
+                               `scala_version` varchar(64) collate utf8mb4_general_ci not null comment 'scala version of spark',
+                               `spark_conf` text collate utf8mb4_general_ci not null comment 'spark-conf',
+                               `is_default` tinyint not null default 0 comment 'whether default version or not',
+                               `description` varchar(255) collate utf8mb4_general_ci default null comment 'description',
+                               `create_time` datetime default null comment 'create time',
+                               primary key (`id`) using btree,
+                               unique key `un_env_name` (`spark_name`) using btree
+) engine=innodb auto_increment=100000 default charset=utf8mb4 collate=utf8mb4_general_ci;
+
+
+-- ----------------------------
+-- Table structure for t_spark_app
+-- ----------------------------
+drop table if exists `t_spark_app`;
+create table `t_spark_app` (
+                               `id` bigint not null auto_increment,
+                               `team_id` bigint not null,
+                               `job_type` tinyint default null,
+                               `execution_mode` tinyint default null,
+                               `resource_from` tinyint default null,
+                               `project_id` bigint default null,
+                               `job_name` varchar(255) collate utf8mb4_general_ci default null,
+                               `module` varchar(255) collate utf8mb4_general_ci default null,
+                               `jar` varchar(255) collate utf8mb4_general_ci default null,
+                               `jar_check_sum` bigint default null,
+                               `main_class` varchar(255) collate utf8mb4_general_ci default null,
+                               `args` text collate utf8mb4_general_ci,
+                               `options` text collate utf8mb4_general_ci,
+                               `hot_params` text collate utf8mb4_general_ci,
+                               `user_id` bigint default null,
+                               `app_id` varchar(64) collate utf8mb4_general_ci default null,
+                               `app_type` tinyint default null,
+                               `duration` bigint default null,
+                               `job_id` varchar(64) collate utf8mb4_general_ci default null,
+                               `job_manager_url` varchar(255) collate utf8mb4_general_ci default null,
+                               `version_id` bigint default null,
+                               `cluster_id` varchar(45) collate utf8mb4_general_ci default null,
+                               `k8s_name` varchar(63) collate utf8mb4_general_ci default null,
+                               `k8s_namespace` varchar(63) collate utf8mb4_general_ci default null,
+                               `spark_image` varchar(128) collate utf8mb4_general_ci default null,
+                               `state` int default null,
+                               `restart_size` int default null,
+                               `restart_count` int default null,
+                               `cp_threshold` int default null,
+                               `cp_max_failure_interval` int default null,
+                               `cp_failure_rate_interval` int default null,
+                               `cp_failure_action` tinyint default null,
+                               `dynamic_properties` text collate utf8mb4_general_ci,
+                               `description` varchar(255) collate utf8mb4_general_ci default null,
+                               `resolve_order` tinyint default null,
+                               `k8s_rest_exposed_type` tinyint default null,
+                               `jm_memory` int default null,
+                               `tm_memory` int default null,
+                               `total_task` int default null,
+                               `total_tm` int default null,
+                               `total_slot` int default null,
+                               `available_slot` int default null,
+                               `option_state` tinyint default null,
+                               `tracking` tinyint default null,
+                               `create_time` datetime default null comment 'create time',
+                               `modify_time` datetime default null comment 'modify time',
+                               `option_time` datetime default null,
+                               `release` tinyint default 1,
+                               `build` tinyint default 1,
+                               `start_time` datetime default null,
+                               `end_time` datetime default null,
+                               `alert_id` bigint default null,
+                               `k8s_pod_template` text collate utf8mb4_general_ci,
+                               `k8s_jm_pod_template` text collate utf8mb4_general_ci,
+                               `k8s_tm_pod_template` text collate utf8mb4_general_ci,
+                               `k8s_hadoop_integration` tinyint default 0,
+                               `spark_cluster_id` bigint default null,
+                               `ingress_template` text collate utf8mb4_general_ci,
+                               `default_mode_ingress` text collate utf8mb4_general_ci,
+                               `tags` varchar(500) default null,
+                               `probing` tinyint default 0,
+                               `hadoop_user` varchar(64) collate utf8mb4_general_ci default null,
+                               primary key (`id`) using btree,
+                               key `inx_job_type` (`job_type`) using btree,
+                               key `inx_track` (`tracking`) using btree,
+                               index `inx_team` (`team_id`) using btree
+) engine=innodb auto_increment=100000 default charset=utf8mb4 collate=utf8mb4_general_ci;
+
 set foreign_key_checks = 1;
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/controller/SparkApplicationBuildPipelineController.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/controller/SparkApplicationBuildPipelineController.java
new file mode 100644
index 0000000..459d30e
--- /dev/null
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/controller/SparkApplicationBuildPipelineController.java
@@ -0,0 +1,84 @@
+/*
+ * 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.streampark.console.core.controller;
+
+import org.apache.streampark.console.base.domain.RestResponse;
+import org.apache.streampark.console.core.annotation.PermissionAction;
+import org.apache.streampark.console.core.entity.AppBuildPipeline;
+import org.apache.streampark.console.core.enums.PermissionTypeEnum;
+import org.apache.streampark.console.core.service.SparkAppBuildPipeService;
+
+import org.apache.shiro.authz.annotation.RequiresPermissions;
+
+import io.swagger.v3.oas.annotations.tags.Tag;
+import lombok.extern.slf4j.Slf4j;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.validation.annotation.Validated;
+import org.springframework.web.bind.annotation.PostMapping;
+import org.springframework.web.bind.annotation.RequestMapping;
+import org.springframework.web.bind.annotation.RestController;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+
+@Tag(name = "SPARK_APPLICATION_BUILD_PIPELINE_TAG")
+@Slf4j
+@Validated
+@RestController
+@RequestMapping("spark/pipe")
+public class SparkApplicationBuildPipelineController {
+
+  @Autowired private SparkAppBuildPipeService appBuildPipeService;
+
+  /**
+   * Release application building pipeline.
+   *
+   * @param appId application id
+   * @param forceBuild forced start pipeline or not
+   * @return Whether the pipeline was successfully started
+   */
+  @PermissionAction(id = "#appId", type = PermissionTypeEnum.APP)
+  @PostMapping(value = "build")
+  @RequiresPermissions("app:create")
+  public RestResponse buildApplication(Long appId, boolean forceBuild) {
+    try {
+      boolean actionResult = appBuildPipeService.buildApplication(appId, forceBuild);
+      return RestResponse.success(actionResult);
+    } catch (Exception e) {
+      return RestResponse.success(false).message(e.getMessage());
+    }
+  }
+
+  /**
+   * Get application building pipeline progress detail.
+   *
+   * @param appId application id
+   * @return "pipeline" -> pipeline details, "docker" -> docker resolved snapshot
+   */
+  //  @Operation(summary = "Get application release pipeline")
+  //  @ApiAccess
+  @PostMapping("/detail")
+  @RequiresPermissions("app:view")
+  public RestResponse getBuildProgressDetail(Long appId) {
+    Map<String, Object> details = new HashMap<>(0);
+    Optional<AppBuildPipeline> pipeline = appBuildPipeService.getCurrentBuildPipeline(appId);
+    details.put("pipeline", pipeline.map(AppBuildPipeline::toView).orElse(null));
+    return RestResponse.success(details);
+  }
+}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/controller/SparkApplicationController.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/controller/SparkApplicationController.java
new file mode 100644
index 0000000..0cc19ae
--- /dev/null
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/controller/SparkApplicationController.java
@@ -0,0 +1,319 @@
+/*
+ * 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.streampark.console.core.controller;
+
+import org.apache.streampark.common.util.Utils;
+import org.apache.streampark.common.util.YarnUtils;
+import org.apache.streampark.console.base.domain.RestRequest;
+import org.apache.streampark.console.base.domain.RestResponse;
+import org.apache.streampark.console.base.exception.InternalException;
+import org.apache.streampark.console.core.annotation.ApiAccess;
+import org.apache.streampark.console.core.annotation.AppUpdated;
+import org.apache.streampark.console.core.annotation.PermissionAction;
+import org.apache.streampark.console.core.entity.ApplicationBackUp;
+import org.apache.streampark.console.core.entity.ApplicationLog;
+import org.apache.streampark.console.core.entity.SparkApplication;
+import org.apache.streampark.console.core.enums.AppExistsStateEnum;
+import org.apache.streampark.console.core.enums.PermissionTypeEnum;
+import org.apache.streampark.console.core.service.ApplicationBackUpService;
+import org.apache.streampark.console.core.service.ApplicationLogService;
+import org.apache.streampark.console.core.service.ResourceService;
+import org.apache.streampark.console.core.service.application.SparkApplicationActionService;
+import org.apache.streampark.console.core.service.application.SparkApplicationInfoService;
+import org.apache.streampark.console.core.service.application.SparkApplicationManageService;
+
+import org.apache.shiro.authz.annotation.RequiresPermissions;
+
+import com.baomidou.mybatisplus.core.metadata.IPage;
+import io.swagger.v3.oas.annotations.Hidden;
+import io.swagger.v3.oas.annotations.Operation;
+import io.swagger.v3.oas.annotations.Parameter;
+import io.swagger.v3.oas.annotations.tags.Tag;
+import lombok.extern.slf4j.Slf4j;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.validation.annotation.Validated;
+import org.springframework.web.bind.annotation.PostMapping;
+import org.springframework.web.bind.annotation.RequestMapping;
+import org.springframework.web.bind.annotation.RestController;
+import org.springframework.web.multipart.MultipartFile;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.Serializable;
+import java.net.URI;
+import java.util.Map;
+
+@Tag(name = "SPARK_APPLICATION_TAG")
+@Slf4j
+@Validated
+@RestController
+@RequestMapping("spark/app")
+public class SparkApplicationController {
+
+  @Autowired private SparkApplicationManageService applicationManageService;
+
+  @Autowired private SparkApplicationActionService applicationActionService;
+
+  @Autowired private SparkApplicationInfoService applicationInfoService;
+
+  @Autowired private ApplicationBackUpService backUpService;
+
+  @Autowired private ApplicationLogService applicationLogService;
+
+  @Autowired private ResourceService resourceService;
+
+  @Operation(summary = "Get application")
+  @ApiAccess
+  @PostMapping("get")
+  @RequiresPermissions("app:detail")
+  public RestResponse get(SparkApplication app) {
+    SparkApplication application = applicationManageService.getApp(app.getId());
+    return RestResponse.success(application);
+  }
+
+  @Operation(summary = "Create application")
+  @ApiAccess
+  @PermissionAction(id = "#app.teamId", type = PermissionTypeEnum.TEAM)
+  @PostMapping("create")
+  @RequiresPermissions("app:create")
+  public RestResponse create(SparkApplication app) throws IOException {
+    boolean saved = applicationManageService.create(app);
+    return RestResponse.success(saved);
+  }
+
+  @ApiAccess
+  @PermissionAction(id = "#app.id", type = PermissionTypeEnum.APP)
+  @PostMapping(value = "copy")
+  @RequiresPermissions("app:copy")
+  public RestResponse copy(@Parameter(hidden = true) SparkApplication app) throws IOException {
+    applicationManageService.copy(app);
+    return RestResponse.success();
+  }
+
+  @Operation(summary = "Update application")
+  @AppUpdated
+  @PermissionAction(id = "#app.id", type = PermissionTypeEnum.APP)
+  @PostMapping("update")
+  @RequiresPermissions("app:update")
+  public RestResponse update(SparkApplication app) {
+    applicationManageService.update(app);
+    return RestResponse.success(true);
+  }
+
+  @Operation(summary = "Get applications dashboard data")
+  @PostMapping("dashboard")
+  public RestResponse dashboard(Long teamId) {
+    Map<String, Serializable> dashboardMap = applicationInfoService.getDashboardDataMap(teamId);
+    return RestResponse.success(dashboardMap);
+  }
+
+  @Operation(summary = "List applications")
+  @ApiAccess
+  @PostMapping("list")
+  @RequiresPermissions("app:view")
+  public RestResponse list(SparkApplication app, RestRequest request) {
+    IPage<SparkApplication> applicationList = applicationManageService.page(app, request);
+    return RestResponse.success(applicationList);
+  }
+
+  @Operation(summary = "Mapping application")
+  @AppUpdated
+  @PostMapping("mapping")
+  @RequiresPermissions("app:mapping")
+  public RestResponse mapping(SparkApplication app) {
+    boolean flag = applicationManageService.mapping(app);
+    return RestResponse.success(flag);
+  }
+
+  @Operation(summary = "Revoke application")
+  @AppUpdated
+  @PermissionAction(id = "#app.id", type = PermissionTypeEnum.APP)
+  @PostMapping("revoke")
+  @RequiresPermissions("app:release")
+  public RestResponse revoke(SparkApplication app) {
+    applicationActionService.revoke(app.getId());
+    return RestResponse.success();
+  }
+
+  @PermissionAction(id = "#app.id", type = PermissionTypeEnum.APP)
+  @PostMapping(value = "check_start")
+  @RequiresPermissions("app:start")
+  public RestResponse checkStart(SparkApplication app) {
+    AppExistsStateEnum stateEnum = applicationInfoService.checkStart(app.getId());
+    return RestResponse.success(stateEnum.get());
+  }
+
+  @ApiAccess
+  @PermissionAction(id = "#app.id", type = PermissionTypeEnum.APP)
+  @PostMapping(value = "start")
+  @RequiresPermissions("app:start")
+  public RestResponse start(@Parameter(hidden = true) SparkApplication app) {
+    try {
+      applicationActionService.start(app, false);
+      return RestResponse.success(true);
+    } catch (Exception e) {
+      return RestResponse.success(false).message(e.getMessage());
+    }
+  }
+
+  @PermissionAction(id = "#app.id", type = PermissionTypeEnum.APP)
+  @PostMapping(value = "cancel")
+  @RequiresPermissions("app:cancel")
+  public RestResponse cancel(@Parameter(hidden = true) SparkApplication app) throws Exception {
+    applicationActionService.cancel(app);
+    return RestResponse.success();
+  }
+
+  @Operation(summary = "Clean application")
+  @AppUpdated
+  @ApiAccess
+  @PermissionAction(id = "#app.id", type = PermissionTypeEnum.APP)
+  @PostMapping("clean")
+  @RequiresPermissions("app:clean")
+  public RestResponse clean(SparkApplication app) {
+    applicationManageService.clean(app);
+    return RestResponse.success(true);
+  }
+
+  /** force stop(stop normal start or in progress) */
+  @Operation(summary = "Force stop application")
+  @PermissionAction(id = "#app.id", type = PermissionTypeEnum.APP)
+  @PostMapping("forcedStop")
+  @RequiresPermissions("app:cancel")
+  public RestResponse forcedStop(SparkApplication app) {
+    applicationActionService.forcedStop(app.getId());
+    return RestResponse.success();
+  }
+
+  @Operation(summary = "Get application on yarn proxy address")
+  @PostMapping("yarn")
+  public RestResponse yarn() {
+    return RestResponse.success(YarnUtils.getRMWebAppProxyURL());
+  }
+
+  @Operation(summary = "Get application on yarn name")
+  @PostMapping("name")
+  public RestResponse yarnName(SparkApplication app) {
+    String yarnName = applicationInfoService.getYarnName(app.getConfig());
+    return RestResponse.success(yarnName);
+  }
+
+  @Operation(summary = "Check the application exist status")
+  @PostMapping("checkName")
+  public RestResponse checkName(SparkApplication app) {
+    AppExistsStateEnum exists = applicationInfoService.checkExists(app);
+    return RestResponse.success(exists.get());
+  }
+
+  @Operation(summary = "Get application conf")
+  @PostMapping("readConf")
+  public RestResponse readConf(SparkApplication app) throws IOException {
+    String config = applicationInfoService.readConf(app.getConfig());
+    return RestResponse.success(config);
+  }
+
+  @Operation(summary = "Get application main-class")
+  @PostMapping("main")
+  public RestResponse getMain(SparkApplication application) {
+    String mainClass = applicationInfoService.getMain(application);
+    return RestResponse.success(mainClass);
+  }
+
+  @Operation(summary = "List application backups")
+  @PostMapping("backups")
+  public RestResponse backups(ApplicationBackUp backUp, RestRequest request) {
+    IPage<ApplicationBackUp> backups = backUpService.getPage(backUp, request);
+    return RestResponse.success(backups);
+  }
+
+  @Operation(summary = "List application operation logs")
+  @PostMapping("optionlog")
+  public RestResponse optionlog(ApplicationLog applicationLog, RestRequest request) {
+    IPage<ApplicationLog> applicationList = applicationLogService.getPage(applicationLog, request);
+    return RestResponse.success(applicationList);
+  }
+
+  @Operation(summary = "Delete application operation log")
+  @PermissionAction(id = "#applicationLog.appId", type = PermissionTypeEnum.APP)
+  @PostMapping("deleteOperationLog")
+  @RequiresPermissions("app:delete")
+  public RestResponse deleteOperationLog(Long id) {
+    Boolean deleted = applicationLogService.removeById(id);
+    return RestResponse.success(deleted);
+  }
+
+  @Operation(summary = "Delete application")
+  @PermissionAction(id = "#app.id", type = PermissionTypeEnum.APP)
+  @PostMapping("delete")
+  @RequiresPermissions("app:delete")
+  public RestResponse delete(SparkApplication app) throws InternalException {
+    Boolean deleted = applicationManageService.remove(app.getId());
+    return RestResponse.success(deleted);
+  }
+
+  @Operation(summary = "Backup application when deleted")
+  @PermissionAction(id = "#backUp.appId", type = PermissionTypeEnum.APP)
+  @PostMapping("deletebak")
+  public RestResponse deleteBak(ApplicationBackUp backUp) throws InternalException {
+    Boolean deleted = backUpService.removeById(backUp.getId());
+    return RestResponse.success(deleted);
+  }
+
+  @Operation(summary = "Check the application jar")
+  @PostMapping("checkjar")
+  public RestResponse checkjar(String jar) {
+    File file = new File(jar);
+    try {
+      Utils.requireCheckJarFile(file.toURI().toURL());
+      return RestResponse.success(true);
+    } catch (IOException e) {
+      return RestResponse.success(file).message(e.getLocalizedMessage());
+    }
+  }
+
+  @Operation(summary = "Upload the application jar")
+  @PostMapping("upload")
+  @RequiresPermissions("app:create")
+  public RestResponse upload(MultipartFile file) throws Exception {
+    String uploadPath = resourceService.upload(file);
+    return RestResponse.success(uploadPath);
+  }
+
+  @Hidden
+  @PostMapping("verifySchema")
+  public RestResponse verifySchema(String path) {
+    final URI uri = URI.create(path);
+    final String scheme = uri.getScheme();
+    final String pathPart = uri.getPath();
+    RestResponse restResponse = RestResponse.success(true);
+    String error = null;
+    if (scheme == null) {
+      error =
+          "The scheme (hdfs://, file://, etc) is null. Please specify the file system scheme explicitly in the URI.";
+    } else if (pathPart == null) {
+      error =
+          "The path to store the checkpoint data in is null. Please specify a directory path for the checkpoint data.";
+    } else if (pathPart.isEmpty() || "/".equals(pathPart)) {
+      error = "Cannot use the root directory for checkpoints.";
+    }
+    if (error != null) {
+      restResponse = RestResponse.success(false).message(error);
+    }
+    return restResponse;
+  }
+}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/controller/SparkEnvController.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/controller/SparkEnvController.java
new file mode 100644
index 0000000..a5c714e
--- /dev/null
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/controller/SparkEnvController.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.streampark.console.core.controller;
+
+import org.apache.streampark.console.base.domain.RestResponse;
+import org.apache.streampark.console.base.exception.ApiDetailException;
+import org.apache.streampark.console.core.entity.SparkEnv;
+import org.apache.streampark.console.core.enums.FlinkEnvCheckEnum;
+import org.apache.streampark.console.core.service.SparkEnvService;
+
+import io.swagger.v3.oas.annotations.Operation;
+import io.swagger.v3.oas.annotations.tags.Tag;
+import lombok.extern.slf4j.Slf4j;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.validation.annotation.Validated;
+import org.springframework.web.bind.annotation.PostMapping;
+import org.springframework.web.bind.annotation.RequestMapping;
+import org.springframework.web.bind.annotation.RestController;
+
+import java.util.List;
+
+@Tag(name = "SPARK_ENV_TAG")
+@Slf4j
+@Validated
+@RestController
+@RequestMapping("spark/env")
+public class SparkEnvController {
+
+  @Autowired private SparkEnvService sparkEnvService;
+
+  @Operation(summary = "Get spark environment")
+  @PostMapping("list")
+  public RestResponse list() {
+    List<SparkEnv> sparkEnvList = sparkEnvService.list();
+    return RestResponse.success(sparkEnvList);
+  }
+
+  @Operation(summary = "Verify spark environment")
+  @PostMapping("check")
+  public RestResponse check(SparkEnv version) {
+    FlinkEnvCheckEnum checkResp = sparkEnvService.check(version);
+    return RestResponse.success(checkResp.getCode());
+  }
+
+  @Operation(summary = "Create spark environment")
+  @PostMapping("create")
+  public RestResponse create(SparkEnv version) {
+    try {
+      sparkEnvService.create(version);
+    } catch (Exception e) {
+      throw new ApiDetailException(e);
+    }
+    return RestResponse.success(true);
+  }
+
+  @Operation(summary = "Get spark environment")
+  @PostMapping("get")
+  public RestResponse get(Long id) throws Exception {
+    SparkEnv sparkEnv = sparkEnvService.getById(id);
+    sparkEnv.unzipSparkConf();
+    return RestResponse.success(sparkEnv);
+  }
+
+  @Operation(summary = "Sync spark environment conf")
+  @PostMapping("sync")
+  public RestResponse sync(Long id) throws Exception {
+    sparkEnvService.syncConf(id);
+    return RestResponse.success();
+  }
+
+  @Operation(summary = "Update spark environment")
+  @PostMapping("update")
+  public RestResponse update(SparkEnv version) throws Exception {
+    try {
+      sparkEnvService.update(version);
+    } catch (Exception e) {
+      throw new ApiDetailException(e);
+    }
+    return RestResponse.success(true);
+  }
+
+  @Operation(summary = "Delete spark environment")
+  @PostMapping("delete")
+  public RestResponse delete(Long id) {
+    sparkEnvService.removeById(id);
+    return RestResponse.success();
+  }
+
+  @Operation(summary = "Check spark environment is valid, else throw exception")
+  @PostMapping("validity")
+  public RestResponse validity(SparkEnv version) {
+    sparkEnvService.validity(version.getId());
+    return RestResponse.success(true);
+  }
+
+  @Operation(summary = "Update spark environment as default")
+  @PostMapping("default")
+  public RestResponse setDefault(Long id) {
+    sparkEnvService.setDefault(id);
+    return RestResponse.success();
+  }
+}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/entity/ApplicationConfig.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/entity/ApplicationConfig.java
index 970e4f5..5a096fb 100644
--- a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/entity/ApplicationConfig.java
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/entity/ApplicationConfig.java
@@ -77,6 +77,14 @@
     application.setFormat(this.format);
   }
 
+  public void setToApplication(SparkApplication application) {
+    String unzipString = DeflaterUtils.unzipString(content);
+    String encode = Base64.getEncoder().encodeToString(unzipString.getBytes());
+    application.setConfig(encode);
+    application.setConfigId(this.id);
+    application.setFormat(this.format);
+  }
+
   public Map<String, String> readConfig() {
     Map<String, String> configs = renderConfigs();
 
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/entity/FlinkSql.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/entity/FlinkSql.java
index 8dc4278..ca025ac 100644
--- a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/entity/FlinkSql.java
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/entity/FlinkSql.java
@@ -71,6 +71,14 @@
     this.createTime = new Date();
   }
 
+  public FlinkSql(SparkApplication application) {
+    this.appId = application.getId();
+    this.sql = application.getSparkSql();
+    this.teamResource = application.getTeamResource();
+    this.dependency = application.getDependency();
+    this.createTime = new Date();
+  }
+
   public void decode() {
     this.setSql(DeflaterUtils.unzipString(this.sql));
   }
@@ -83,6 +91,14 @@
     application.setSqlId(this.id);
   }
 
+  public void setToApplication(SparkApplication application) {
+    String encode = Base64.getEncoder().encodeToString(this.sql.getBytes());
+    application.setSparkSql(encode);
+    application.setDependency(this.dependency);
+    application.setTeamResource(this.teamResource);
+    application.setSqlId(this.id);
+  }
+
   public ChangeTypeEnum checkChange(FlinkSql target) {
     // 1) determine if sql statement has changed
     boolean sqlDifference = !this.getSql().trim().equals(target.getSql().trim());
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/entity/SparkApplication.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/entity/SparkApplication.java
new file mode 100644
index 0000000..f8d3d6f
--- /dev/null
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/entity/SparkApplication.java
@@ -0,0 +1,599 @@
+/*
+ * 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.streampark.console.core.entity;
+
+import org.apache.streampark.common.Constant;
+import org.apache.streampark.common.conf.ConfigKeys;
+import org.apache.streampark.common.conf.Workspace;
+import org.apache.streampark.common.enums.ApplicationType;
+import org.apache.streampark.common.enums.FlinkDevelopmentMode;
+import org.apache.streampark.common.enums.SparkExecutionMode;
+import org.apache.streampark.common.enums.StorageType;
+import org.apache.streampark.common.fs.FsOperator;
+import org.apache.streampark.console.base.util.JacksonUtils;
+import org.apache.streampark.console.core.bean.AppControl;
+import org.apache.streampark.console.core.bean.Dependency;
+import org.apache.streampark.console.core.enums.FlinkAppStateEnum;
+import org.apache.streampark.console.core.enums.ReleaseStateEnum;
+import org.apache.streampark.console.core.enums.ResourceFromEnum;
+import org.apache.streampark.console.core.metrics.flink.JobsOverview;
+import org.apache.streampark.console.core.utils.YarnQueueLabelExpression;
+import org.apache.streampark.flink.kubernetes.model.K8sPodTemplates;
+import org.apache.streampark.flink.packer.maven.DependencyInfo;
+
+import org.apache.commons.collections.MapUtils;
+import org.apache.commons.lang3.StringUtils;
+
+import com.baomidou.mybatisplus.annotation.FieldStrategy;
+import com.baomidou.mybatisplus.annotation.IdType;
+import com.baomidou.mybatisplus.annotation.TableField;
+import com.baomidou.mybatisplus.annotation.TableId;
+import com.baomidou.mybatisplus.annotation.TableName;
+import com.baomidou.mybatisplus.core.toolkit.support.SFunction;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import lombok.Data;
+import lombok.Getter;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+
+@Data
+@TableName("t_spark_app")
+@Slf4j
+public class SparkApplication implements Serializable {
+
+  @TableId(type = IdType.AUTO)
+  private Long id;
+
+  private Long teamId;
+
+  /** 1) custom code 2) spark SQL */
+  private Integer jobType;
+
+  private Long projectId;
+  /** creator */
+  private Long userId;
+
+  /** The name of the frontend and program displayed in yarn */
+  private String jobName;
+
+  @TableField(updateStrategy = FieldStrategy.IGNORED)
+  private String appId;
+
+  @TableField(updateStrategy = FieldStrategy.IGNORED)
+  private String jobId;
+
+  /** The address of the jobmanager, that is, the direct access address of the Flink web UI */
+  @TableField(updateStrategy = FieldStrategy.IGNORED)
+  private String jobManagerUrl;
+
+  /** spark version */
+  private Long versionId;
+
+  /** k8s cluster id */
+  private String clusterId;
+
+  /** spark docker base image */
+  private String sparkImage;
+
+  /** The resource name of the spark job on k8s, equivalent to clusterId in application mode. */
+  private String k8sName;
+
+  /** k8s namespace */
+  private String k8sNamespace = Constant.DEFAULT;
+
+  /** The exposed type of the rest service of K8s(kubernetes.rest-service.exposed.type) */
+  private Integer k8sRestExposedType;
+  /** spark kubernetes pod template */
+  private String k8sPodTemplate;
+
+  private String k8sJmPodTemplate;
+  private String k8sTmPodTemplate;
+
+  @Getter private String ingressTemplate;
+  private String defaultModeIngress;
+
+  /** spark-hadoop integration on spark-k8s mode */
+  private Boolean k8sHadoopIntegration;
+
+  private Integer state;
+  /** task release status */
+  @TableField("`release`")
+  private Integer release;
+
+  /** determine if a task needs to be built */
+  private Boolean build;
+
+  /** max restart retries after job failed */
+  @TableField(updateStrategy = FieldStrategy.IGNORED)
+  private Integer restartSize;
+
+  /** has restart count */
+  private Integer restartCount;
+
+  private Integer optionState;
+
+  /** alert id */
+  @TableField(updateStrategy = FieldStrategy.IGNORED)
+  private Long alertId;
+
+  private String args;
+  /** application module */
+  private String module;
+
+  private String options;
+
+  @TableField(updateStrategy = FieldStrategy.IGNORED)
+  private String hotParams;
+
+  private Integer resolveOrder;
+  private Integer executionMode;
+  private String dynamicProperties;
+  private Integer appType;
+
+  /** determine if tracking status */
+  private Integer tracking;
+
+  private String jar;
+
+  /**
+   * for upload type tasks, checkSum needs to be recorded whether it needs to be republished after
+   * the update and modify.
+   */
+  private Long jarCheckSum;
+
+  private String mainClass;
+
+  private Date startTime;
+
+  @TableField(updateStrategy = FieldStrategy.IGNORED)
+  private Date endTime;
+
+  private Long duration;
+
+  /** checkpoint max failure interval */
+  private Integer cpMaxFailureInterval;
+
+  /** checkpoint failure rate interval */
+  private Integer cpFailureRateInterval;
+
+  /** Actions triggered after X minutes failed Y times: 1: send alert 2: restart */
+  private Integer cpFailureAction;
+
+  /** overview */
+  @TableField("TOTAL_TM")
+  private Integer totalTM;
+
+  @TableField("HADOOP_USER")
+  private String hadoopUser;
+
+  private Integer totalSlot;
+  private Integer availableSlot;
+  private Integer jmMemory;
+  private Integer tmMemory;
+  private Integer totalTask;
+
+  /** the cluster id bound to the task in remote mode */
+  @TableField(updateStrategy = FieldStrategy.IGNORED)
+  private Long sparkClusterId;
+
+  private String description;
+
+  private Date createTime;
+
+  private Date optionTime;
+
+  private Date modifyTime;
+
+  /** 1: cicd (build from csv) 2: upload (upload local jar job) */
+  private Integer resourceFrom;
+
+  private String tags;
+
+  private Boolean probing = false;
+
+  /** running job */
+  private transient JobsOverview.Task overview;
+
+  private transient String teamResource;
+  private transient String dependency;
+  private transient Long sqlId;
+  private transient String sparkSql;
+
+  private transient Integer[] stateArray;
+  private transient Integer[] jobTypeArray;
+  private transient Boolean backUp = false;
+  private transient Boolean restart = false;
+  private transient String userName;
+  private transient String nickName;
+  private transient String config;
+  private transient Long configId;
+  private transient String sparkVersion;
+  private transient String confPath;
+  private transient Integer format;
+  private transient String savePoint;
+  private transient Boolean savePointed = false;
+  private transient Boolean drain = false;
+  private transient Boolean nativeFormat = false;
+  private transient Long savePointTimeout = 60L;
+  private transient Boolean allowNonRestored = false;
+  private transient Integer restoreMode;
+  private transient String socketId;
+  private transient String projectName;
+  private transient String createTimeFrom;
+  private transient String createTimeTo;
+  private transient String backUpDescription;
+  private transient String yarnQueue;
+
+  /** spark Web UI Url */
+  private transient String sparkRestUrl;
+
+  /** refer to {@link org.apache.streampark.flink.packer.pipeline.BuildPipeline} */
+  private transient Integer buildStatus;
+
+  private transient AppControl appControl;
+
+  public void setDefaultModeIngress(String defaultModeIngress) {
+    this.defaultModeIngress = defaultModeIngress;
+  }
+
+  public void setK8sNamespace(String k8sNamespace) {
+    this.k8sNamespace = StringUtils.isBlank(k8sNamespace) ? Constant.DEFAULT : k8sNamespace;
+  }
+
+  public K8sPodTemplates getK8sPodTemplates() {
+    return K8sPodTemplates.of(k8sPodTemplate, k8sJmPodTemplate, k8sTmPodTemplate);
+  }
+
+  public void setState(Integer state) {
+    this.state = state;
+    this.tracking = shouldTracking() ? 1 : 0;
+  }
+
+  public void setYarnQueueByHotParams() {
+    if (!(SparkExecutionMode.YARN_CLIENT == this.getSparkExecutionMode()
+        || SparkExecutionMode.YARN_CLUSTER == this.getSparkExecutionMode())) {
+      return;
+    }
+
+    Map<String, Object> hotParamsMap = this.getHotParamsMap();
+    if (MapUtils.isNotEmpty(hotParamsMap)
+        && hotParamsMap.containsKey(ConfigKeys.KEY_YARN_APP_QUEUE())) {
+      String yarnQueue = hotParamsMap.get(ConfigKeys.KEY_YARN_APP_QUEUE()).toString();
+      String labelExpr =
+          Optional.ofNullable(hotParamsMap.get(ConfigKeys.KEY_YARN_APP_NODE_LABEL()))
+              .map(Object::toString)
+              .orElse(null);
+      this.setYarnQueue(YarnQueueLabelExpression.of(yarnQueue, labelExpr).toString());
+    }
+  }
+
+  /**
+   * Determine if a FlinkAppState requires tracking.
+   *
+   * @return 1: need to be tracked | 0: no need to be tracked.
+   */
+  public Boolean shouldTracking() {
+    switch (getStateEnum()) {
+      case ADDED:
+      case CREATED:
+      case FINISHED:
+      case FAILED:
+      case CANCELED:
+      case TERMINATED:
+      case POS_TERMINATED:
+        return false;
+      default:
+        return true;
+    }
+  }
+
+  /**
+   * Determine whether the application can be started to prevent repeated starts.
+   *
+   * @return true: can start | false: can not start.
+   */
+  public boolean isCanBeStart() {
+    switch (getStateEnum()) {
+      case ADDED:
+      case CREATED:
+      case FAILED:
+      case CANCELED:
+      case FINISHED:
+      case LOST:
+      case TERMINATED:
+      case SUCCEEDED:
+      case KILLED:
+      case POS_TERMINATED:
+        return true;
+      default:
+        return false;
+    }
+  }
+
+  @JsonIgnore
+  public ReleaseStateEnum getReleaseState() {
+    return ReleaseStateEnum.of(release);
+  }
+
+  @JsonIgnore
+  public FlinkDevelopmentMode getDevelopmentMode() {
+    return FlinkDevelopmentMode.of(jobType);
+  }
+
+  @JsonIgnore
+  public FlinkAppStateEnum getStateEnum() {
+    return FlinkAppStateEnum.of(state);
+  }
+
+  @JsonIgnore
+  public SparkExecutionMode getSparkExecutionMode() {
+    return SparkExecutionMode.of(executionMode);
+  }
+
+  public boolean cpFailedTrigger() {
+    return this.cpMaxFailureInterval != null
+        && this.cpFailureRateInterval != null
+        && this.cpFailureAction != null;
+  }
+
+  public boolean eqFlinkJob(SparkApplication other) {
+    if (this.isSparkSqlJob()
+        && other.isSparkSqlJob()
+        && this.getSparkSql().trim().equals(other.getSparkSql().trim())) {
+      return this.getDependencyObject().equals(other.getDependencyObject());
+    }
+    return false;
+  }
+
+  /** Local compilation and packaging working directory */
+  @JsonIgnore
+  public String getDistHome() {
+    String path =
+        String.format("%s/%s/%s", Workspace.APP_LOCAL_DIST(), projectId.toString(), getModule());
+    log.info("local distHome:{}", path);
+    return path;
+  }
+
+  @JsonIgnore
+  public String getLocalAppHome() {
+    String path = String.format("%s/%s", Workspace.local().APP_WORKSPACE(), id.toString());
+    log.info("local appHome:{}", path);
+    return path;
+  }
+
+  @JsonIgnore
+  public String getRemoteAppHome() {
+    String path = String.format("%s/%s", Workspace.remote().APP_WORKSPACE(), id.toString());
+    log.info("remote appHome:{}", path);
+    return path;
+  }
+
+  /** Automatically identify remoteAppHome or localAppHome based on app SparkExecutionMode */
+  @JsonIgnore
+  public String getAppHome() {
+    switch (this.getSparkExecutionMode()) {
+      case REMOTE:
+      case LOCAL:
+        return getLocalAppHome();
+      case YARN_CLIENT:
+      case YARN_CLUSTER:
+        return getRemoteAppHome();
+      default:
+        throw new UnsupportedOperationException(
+            "unsupported executionMode ".concat(getSparkExecutionMode().getName()));
+    }
+  }
+
+  @JsonIgnore
+  public String getAppLib() {
+    return getAppHome().concat("/lib");
+  }
+
+  @JsonIgnore
+  public ApplicationType getApplicationType() {
+    return ApplicationType.of(appType);
+  }
+
+  @JsonIgnore
+  @SneakyThrows
+  @SuppressWarnings("unchecked")
+  public Map<String, Object> getOptionMap() {
+    if (StringUtils.isBlank(this.options)) {
+      return Collections.emptyMap();
+    }
+    Map<String, Object> optionMap = JacksonUtils.read(this.options, Map.class);
+    optionMap.entrySet().removeIf(entry -> entry.getValue() == null);
+    return optionMap;
+  }
+
+  @JsonIgnore
+  public boolean isSparkSqlJob() {
+    return FlinkDevelopmentMode.FLINK_SQL.getMode().equals(this.getJobType());
+  }
+
+  @JsonIgnore
+  public boolean isCustomCodeJob() {
+    return FlinkDevelopmentMode.CUSTOM_CODE.getMode().equals(this.getJobType());
+  }
+
+  @JsonIgnore
+  public boolean isCustomCodeOrPySparkJob() {
+    return FlinkDevelopmentMode.CUSTOM_CODE.getMode().equals(this.getJobType())
+        || FlinkDevelopmentMode.PYFLINK.getMode().equals(this.getJobType());
+  }
+
+  @JsonIgnore
+  public boolean isUploadJob() {
+    return isCustomCodeOrPySparkJob()
+        && ResourceFromEnum.UPLOAD.getValue().equals(this.getResourceFrom());
+  }
+
+  @JsonIgnore
+  public boolean isCICDJob() {
+    return isCustomCodeOrPySparkJob()
+        && ResourceFromEnum.CICD.getValue().equals(this.getResourceFrom());
+  }
+
+  public boolean isStreamParkJob() {
+    return this.getAppType() == ApplicationType.STREAMPARK_FLINK.getType();
+  }
+
+  @JsonIgnore
+  @SneakyThrows
+  public Dependency getDependencyObject() {
+    return Dependency.toDependency(this.dependency);
+  }
+
+  @JsonIgnore
+  public DependencyInfo getDependencyInfo() {
+    return Dependency.toDependency(getDependency()).toJarPackDeps();
+  }
+
+  @JsonIgnore
+  public boolean isRunning() {
+    return FlinkAppStateEnum.RUNNING.getValue() == this.getState();
+  }
+
+  @JsonIgnore
+  public boolean isNeedRollback() {
+    return ReleaseStateEnum.NEED_ROLLBACK.get() == this.getRelease();
+  }
+
+  @JsonIgnore
+  public boolean isNeedRestartOnFailed() {
+    if (this.restartSize != null && this.restartCount != null) {
+      return this.restartSize > 0 && this.restartCount <= this.restartSize;
+    }
+    return false;
+  }
+
+  @JsonIgnore
+  public StorageType getStorageType() {
+    return getStorageType(getExecutionMode());
+  }
+
+  public static StorageType getStorageType(Integer execMode) {
+    SparkExecutionMode executionModeEnum = SparkExecutionMode.of(execMode);
+    switch (Objects.requireNonNull(executionModeEnum)) {
+      case YARN_CLUSTER:
+      case YARN_CLIENT:
+        return StorageType.HDFS;
+      case REMOTE:
+        return StorageType.LFS;
+      default:
+        throw new UnsupportedOperationException("Unsupported ".concat(executionModeEnum.getName()));
+    }
+  }
+
+  @JsonIgnore
+  public FsOperator getFsOperator() {
+    return FsOperator.of(getStorageType());
+  }
+
+  @JsonIgnore
+  public Workspace getWorkspace() {
+    return Workspace.of(getStorageType());
+  }
+
+  @JsonIgnore
+  @SneakyThrows
+  @SuppressWarnings("unchecked")
+  public Map<String, Object> getHotParamsMap() {
+    if (StringUtils.isNotBlank(this.hotParams)) {
+      Map<String, Object> hotParamsMap = JacksonUtils.read(this.hotParams, Map.class);
+      hotParamsMap.entrySet().removeIf(entry -> entry.getValue() == null);
+      return hotParamsMap;
+    }
+    return Collections.EMPTY_MAP;
+  }
+
+  @SneakyThrows
+  public void doSetHotParams() {
+    updateHotParams(this);
+  }
+
+  @SneakyThrows
+  public void updateHotParams(SparkApplication appParam) {
+    if (appParam != this) {
+      this.hotParams = null;
+    }
+    SparkExecutionMode executionModeEnum = appParam.getSparkExecutionMode();
+    Map<String, String> hotParams = new HashMap<>(0);
+    if (needFillYarnQueueLabel(executionModeEnum)) {
+      hotParams.putAll(YarnQueueLabelExpression.getQueueLabelMap(appParam.getYarnQueue()));
+    }
+    if (MapUtils.isNotEmpty(hotParams)) {
+      this.setHotParams(JacksonUtils.write(hotParams));
+    }
+  }
+
+  private boolean needFillYarnQueueLabel(SparkExecutionMode mode) {
+    return SparkExecutionMode.YARN_CLUSTER == mode || SparkExecutionMode.YARN_CLIENT == mode;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    return id.equals(((SparkApplication) o).id);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(id);
+  }
+
+  public static class SFunc {
+    public static final SFunction<SparkApplication, Long> ID = SparkApplication::getId;
+    public static final SFunction<SparkApplication, String> JOB_ID = SparkApplication::getJobId;
+    public static final SFunction<SparkApplication, Date> START_TIME =
+        SparkApplication::getStartTime;
+    public static final SFunction<SparkApplication, Date> END_TIME = SparkApplication::getEndTime;
+    public static final SFunction<SparkApplication, Long> DURATION = SparkApplication::getDuration;
+    public static final SFunction<SparkApplication, Integer> TOTAL_TASK =
+        SparkApplication::getTotalTask;
+    public static final SFunction<SparkApplication, Integer> TOTAL_TM =
+        SparkApplication::getTotalTM;
+    public static final SFunction<SparkApplication, Integer> TOTAL_SLOT =
+        SparkApplication::getTotalSlot;
+    public static final SFunction<SparkApplication, Integer> JM_MEMORY =
+        SparkApplication::getJmMemory;
+    public static final SFunction<SparkApplication, Integer> TM_MEMORY =
+        SparkApplication::getTmMemory;
+    public static final SFunction<SparkApplication, Integer> STATE = SparkApplication::getState;
+    public static final SFunction<SparkApplication, String> OPTIONS = SparkApplication::getOptions;
+    public static final SFunction<SparkApplication, Integer> AVAILABLE_SLOT =
+        SparkApplication::getAvailableSlot;
+    public static final SFunction<SparkApplication, Integer> EXECUTION_MODE =
+        SparkApplication::getExecutionMode;
+    public static final SFunction<SparkApplication, String> JOB_MANAGER_URL =
+        SparkApplication::getJobManagerUrl;
+  }
+}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/entity/SparkEnv.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/entity/SparkEnv.java
new file mode 100644
index 0000000..45ce197
--- /dev/null
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/entity/SparkEnv.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.streampark.console.core.entity;
+
+import org.apache.streampark.common.conf.SparkVersion;
+import org.apache.streampark.common.util.DeflaterUtils;
+import org.apache.streampark.common.util.PropertiesUtils;
+import org.apache.streampark.console.base.exception.ApiDetailException;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+
+import com.baomidou.mybatisplus.annotation.IdType;
+import com.baomidou.mybatisplus.annotation.TableId;
+import com.baomidou.mybatisplus.annotation.TableName;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import lombok.Data;
+
+import java.io.File;
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import java.util.Date;
+import java.util.Map;
+
+@Data
+@TableName("t_spark_env")
+public class SparkEnv implements Serializable {
+
+  @TableId(type = IdType.AUTO)
+  private Long id;
+
+  private String sparkName;
+
+  private String sparkHome;
+
+  private String sparkConf;
+
+  private String description;
+
+  private String scalaVersion;
+
+  private String version;
+
+  /** is default */
+  private Boolean isDefault;
+
+  private Date createTime;
+
+  private transient SparkVersion sparkVersion;
+
+  private transient String versionOfLarge;
+
+  private transient String versionOfMiddle;
+
+  private transient String versionOfLast;
+
+  private transient String streamParkScalaVersion = scala.util.Properties.versionNumberString();
+
+  public void doSetSparkConf() throws ApiDetailException {
+    try {
+      File yaml = new File(this.sparkHome.concat("/conf/spark-defaults.conf"));
+      String sparkConf = FileUtils.readFileToString(yaml, StandardCharsets.UTF_8);
+      this.sparkConf = DeflaterUtils.zipString(sparkConf);
+    } catch (Exception e) {
+      throw new ApiDetailException(e);
+    }
+  }
+
+  public void doSetVersion() {
+    this.setVersion(this.getSparkVersion().version());
+    this.setScalaVersion(this.getSparkVersion().scalaVersion());
+    if (!streamParkScalaVersion.startsWith(this.getSparkVersion().scalaVersion())) {
+      throw new UnsupportedOperationException(
+          String.format(
+              "The current Scala version of StreamPark is %s, but the scala version of Spark to be added is %s, which does not match, Please check",
+              streamParkScalaVersion, this.getSparkVersion().scalaVersion()));
+    }
+  }
+
+  public Map<String, String> convertSparkYamlAsMap() {
+    String sparkYamlString = DeflaterUtils.unzipString(sparkConf);
+    return PropertiesUtils.loadFlinkConfYaml(sparkYamlString);
+  }
+
+  @JsonIgnore
+  public SparkVersion getSparkVersion() {
+    if (this.sparkVersion == null) {
+      this.sparkVersion = new SparkVersion(this.sparkHome);
+    }
+    return this.sparkVersion;
+  }
+
+  public void unzipSparkConf() {
+    this.sparkConf = DeflaterUtils.unzipString(this.sparkConf);
+  }
+
+  public String getLargeVersion() {
+    if (StringUtils.isNotBlank(this.version)) {
+      return this.version.substring(0, this.version.lastIndexOf("."));
+    }
+    return null;
+  }
+
+  public String getVersionOfFirst() {
+    if (StringUtils.isNotBlank(this.version)) {
+      return this.version.split("\\.")[0];
+    }
+    return null;
+  }
+
+  public String getVersionOfMiddle() {
+    if (StringUtils.isNotBlank(this.version)) {
+      return this.version.split("\\.")[1];
+    }
+    return null;
+  }
+
+  public String getVersionOfLast() {
+    if (StringUtils.isNotBlank(this.version)) {
+      return this.version.split("\\.")[2];
+    }
+    return null;
+  }
+}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/enums/SparkAppStateEnum.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/enums/SparkAppStateEnum.java
new file mode 100644
index 0000000..12bf740
--- /dev/null
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/enums/SparkAppStateEnum.java
@@ -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.
+ */
+
+package org.apache.streampark.console.core.enums;
+
+import lombok.Getter;
+
+/** Describe the status of Spark Application */
+@Getter
+public enum SparkAppStateEnum {
+
+  /** Added new job to database. */
+  ADDED(0),
+
+  /**
+   * The job has been received by the Dispatcher, and is waiting for the job manager to be created.
+   */
+  INITIALIZING(1),
+
+  /** Job is newly created, no task has started to run. */
+  CREATED(2),
+
+  /** Application which is currently running. */
+  STARTING(3),
+
+  /** Application which is currently running. */
+  RESTARTING(4),
+
+  /** Some tasks are scheduled or running, some may be pending, some may be finished. */
+  RUNNING(5),
+
+  /** The job has failed and is currently waiting for the cleanup to complete. */
+  FAILING(6),
+
+  /** The job has failed with a non-recoverable task failure. */
+  FAILED(7),
+
+  /** Job is being cancelled. */
+  CANCELLING(8),
+
+  /** Job has been cancelled. */
+  CANCELED(9),
+
+  /** All the job's tasks have successfully finished. */
+  FINISHED(10),
+
+  /**
+   * The job has been suspended which means that it has been stopped but not been removed from a
+   * potential HA job store.
+   */
+  SUSPENDED(11),
+
+  /** The job is currently reconciling and waits for task execution report to recover state. */
+  RECONCILING(12),
+
+  /** Loss of mapping. */
+  LOST(13),
+
+  /** Mapping. */
+  MAPPING(14),
+
+  /** Other statuses. */
+  OTHER(15),
+
+  /** Has rollback. */
+  REVOKED(16),
+
+  /**
+   * Lost track of Spark job temporarily. A complete loss of Spark job tracking translates into LOST
+   * state.
+   */
+  @Deprecated
+  SILENT(17),
+
+  /** Spark job has terminated vaguely, maybe FINISHED, CANCELED or FAILED. */
+  TERMINATED(18),
+
+  /** Spark job has terminated vaguely, maybe FINISHED, CANCELED or FAILED. */
+  @Deprecated
+  POS_TERMINATED(19),
+
+  /** Job SUCCEEDED on yarn. */
+  SUCCEEDED(20),
+
+  /** Job auto Health probe */
+  PROBING(21),
+
+  /** Has killed in Yarn. */
+  KILLED(-9);
+
+  private final int value;
+
+  SparkAppStateEnum(int value) {
+    this.value = value;
+  }
+
+  public static SparkAppStateEnum of(Integer state) {
+    for (SparkAppStateEnum appState : values()) {
+      if (appState.value == state) {
+        return appState;
+      }
+    }
+    return SparkAppStateEnum.OTHER;
+  }
+
+  public static SparkAppStateEnum of(String name) {
+    for (SparkAppStateEnum appState : values()) {
+      if (appState.name().equals(name)) {
+        return appState;
+      }
+    }
+    return SparkAppStateEnum.OTHER;
+  }
+
+  public static boolean isEndState(Integer appState) {
+    SparkAppStateEnum sparkAppStateEnum = SparkAppStateEnum.of(appState);
+    return SparkAppStateEnum.CANCELED == sparkAppStateEnum
+        || SparkAppStateEnum.FAILED == sparkAppStateEnum
+        || SparkAppStateEnum.KILLED == sparkAppStateEnum
+        || SparkAppStateEnum.FINISHED == sparkAppStateEnum
+        || SparkAppStateEnum.SUCCEEDED == sparkAppStateEnum
+        || SparkAppStateEnum.LOST == sparkAppStateEnum
+        || SparkAppStateEnum.TERMINATED == sparkAppStateEnum;
+  }
+
+  public static boolean isLost(Integer appState) {
+    SparkAppStateEnum sparkAppStateEnum = SparkAppStateEnum.of(appState);
+    return SparkAppStateEnum.LOST == sparkAppStateEnum;
+  }
+}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/mapper/SparkApplicationMapper.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/mapper/SparkApplicationMapper.java
new file mode 100644
index 0000000..e260fac
--- /dev/null
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/mapper/SparkApplicationMapper.java
@@ -0,0 +1,64 @@
+/*
+ * 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.streampark.console.core.mapper;
+
+import org.apache.streampark.console.core.entity.SparkApplication;
+
+import org.apache.ibatis.annotations.Param;
+
+import com.baomidou.mybatisplus.core.mapper.BaseMapper;
+import com.baomidou.mybatisplus.core.metadata.IPage;
+import com.baomidou.mybatisplus.extension.plugins.pagination.Page;
+
+import java.util.List;
+
+public interface SparkApplicationMapper extends BaseMapper<SparkApplication> {
+
+  IPage<SparkApplication> selectPage(
+      Page<SparkApplication> page, @Param("app") SparkApplication application);
+
+  SparkApplication selectApp(@Param("id") Long id);
+
+  List<SparkApplication> selectAppsByTeamId(@Param("teamId") Long teamId);
+
+  void persistMetrics(@Param("app") SparkApplication application);
+
+  List<SparkApplication> selectProbeApps();
+
+  boolean mapping(@Param("app") SparkApplication appParam);
+
+  List<String> selectRecentK8sNamespaces(@Param("limitSize") Integer limit);
+
+  List<String> selectRecentK8sClusterIds(
+      @Param("executionMode") Integer executionMode, @Param("limitSize") Integer limit);
+
+  List<String> selectRecentK8sPodTemplates(@Param("limitSize") Integer limit);
+
+  List<String> selectRecentK8sJmPodTemplates(@Param("limitSize") Integer limit);
+
+  List<String> selectRecentK8sTmPodTemplates(@Param("limitSize") Integer limit);
+
+  void resetOptionState();
+
+  List<SparkApplication> selectAppsByProjectId(@Param("projectId") Long id);
+
+  boolean existsRunningJobByClusterId(@Param("clusterId") Long clusterId);
+
+  Integer countAffectedByClusterId(
+      @Param("clusterId") Long clusterId, @Param("dbType") String dbType);
+}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/mapper/SparkEnvMapper.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/mapper/SparkEnvMapper.java
new file mode 100644
index 0000000..cf52d7b
--- /dev/null
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/mapper/SparkEnvMapper.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.streampark.console.core.mapper;
+
+import org.apache.streampark.console.core.entity.SparkEnv;
+
+import org.apache.ibatis.annotations.Param;
+
+import com.baomidou.mybatisplus.core.mapper.BaseMapper;
+
+public interface SparkEnvMapper extends BaseMapper<SparkEnv> {
+
+  SparkEnv selectByAppId(@Param("appId") Long appId);
+
+  void setDefault(@Param("id") Long id);
+}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/runner/EnvInitializer.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/runner/EnvInitializer.java
index 3454dd8..a95b3c9 100644
--- a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/runner/EnvInitializer.java
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/runner/EnvInitializer.java
@@ -29,6 +29,7 @@
 import org.apache.streampark.common.zio.ZIOExt;
 import org.apache.streampark.console.base.util.WebUtils;
 import org.apache.streampark.console.core.entity.FlinkEnv;
+import org.apache.streampark.console.core.entity.SparkEnv;
 import org.apache.streampark.console.core.service.SettingService;
 import org.apache.streampark.flink.kubernetes.v2.fs.EmbeddedFileServer;
 
@@ -250,4 +251,29 @@
       fsOperator.upload(flinkLocalHome, flinkHome, false, true);
     }
   }
+
+  public void checkSparkEnv(StorageType storageType, SparkEnv sparkEnv) throws IOException {
+    String sparkLocalHome = sparkEnv.getSparkHome();
+    if (StringUtils.isBlank(sparkLocalHome)) {
+      throw new ExceptionInInitializerError(
+          "[StreamPark] SPARK_HOME is undefined,Make sure that Spark is installed.");
+    }
+    Workspace workspace = Workspace.of(storageType);
+    String appSpark = workspace.APP_SPARK();
+    FsOperator fsOperator = FsOperator.of(storageType);
+    if (!fsOperator.exists(appSpark)) {
+      log.info("checkSparkEnv, now mkdir [{}] starting ...", appSpark);
+      fsOperator.mkdirs(appSpark);
+    }
+    File sparkLocalDir = new File(sparkLocalHome);
+    if (Files.isSymbolicLink(sparkLocalDir.toPath())) {
+      sparkLocalDir = sparkLocalDir.getCanonicalFile();
+    }
+    String sparkName = sparkLocalDir.getName();
+    String sparkHome = appSpark.concat("/").concat(sparkName);
+    if (!fsOperator.exists(sparkHome)) {
+      log.info("{} is not exists,upload beginning....", sparkHome);
+      fsOperator.upload(sparkLocalHome, sparkHome, false, true);
+    }
+  }
 }
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/CommonService.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/CommonService.java
index 6b0852c..33bbbe7 100644
--- a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/CommonService.java
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/CommonService.java
@@ -18,6 +18,7 @@
 package org.apache.streampark.console.core.service;
 
 import org.apache.streampark.console.core.entity.FlinkEnv;
+import org.apache.streampark.console.core.entity.SparkEnv;
 import org.apache.streampark.console.system.entity.User;
 
 /** Base Service */
@@ -44,4 +45,6 @@
    * @return Jar
    */
   String getSqlClientJar(FlinkEnv flinkEnv);
+
+  String getSqlClientJar(SparkEnv flinkEnv);
 }
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/SparkAppBuildPipeService.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/SparkAppBuildPipeService.java
new file mode 100644
index 0000000..c571c1d
--- /dev/null
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/SparkAppBuildPipeService.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.streampark.console.core.service;
+
+import org.apache.streampark.console.core.entity.AppBuildPipeline;
+import org.apache.streampark.flink.packer.pipeline.PipelineStatusEnum;
+
+import com.baomidou.mybatisplus.extension.service.IService;
+
+import javax.annotation.Nonnull;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+/** Applications can be built asynchronously, can manage pipeline and get info */
+public interface SparkAppBuildPipeService extends IService<AppBuildPipeline> {
+
+  /**
+   * Build application. This is an async call method.
+   *
+   * @param appId application id
+   * @param forceBuild forced start pipeline or not
+   * @return Whether the pipeline was successfully started
+   */
+  boolean buildApplication(@Nonnull Long appId, boolean forceBuild) throws Exception;
+
+  /**
+   * Get current build pipeline instance of specified application
+   *
+   * @param appId application id
+   * @return ApplicationBuildPipeline instance
+   */
+  Optional<AppBuildPipeline> getCurrentBuildPipeline(@Nonnull Long appId);
+
+  /**
+   * Whether the application can currently start a new building progress
+   *
+   * @param appId application id
+   * @return Whether construction can be started at this time
+   */
+  boolean allowToBuildNow(@Nonnull Long appId);
+
+  /**
+   * List pipeline status on application id list
+   *
+   * @param appIds list of application ids
+   * @return Map structure, key is application id, value is for the pipeline state
+   */
+  Map<Long, PipelineStatusEnum> listAppIdPipelineStatusMap(List<Long> appIds);
+
+  /**
+   * Delete appBuildPipeline By application id
+   *
+   * @param appId
+   */
+  void removeByAppId(Long appId);
+}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/SparkEnvService.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/SparkEnvService.java
new file mode 100644
index 0000000..ee1a044
--- /dev/null
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/SparkEnvService.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.streampark.console.core.service;
+
+import org.apache.streampark.console.core.entity.SparkEnv;
+import org.apache.streampark.console.core.enums.FlinkEnvCheckEnum;
+
+import com.baomidou.mybatisplus.extension.service.IService;
+
+import java.io.IOException;
+
+public interface SparkEnvService extends IService<SparkEnv> {
+
+  /**
+   * Checks if a specific version of Flink exists.
+   *
+   * @param version The version of Flink to check.
+   * @return Returns enum value indicating the existence of the specified version.
+   */
+  FlinkEnvCheckEnum check(SparkEnv version);
+
+  /**
+   * Create a new instance.
+   *
+   * @param version The version of SparkEnv to use.
+   * @throws Exception if an error occurs during the creation process.
+   * @return true if the instance is successfully created, false otherwise.
+   */
+  boolean create(SparkEnv version) throws Exception;
+
+  /**
+   * Deletes a Flink environment with the provided ID.
+   *
+   * @param id the ID of the Flink environment to delete
+   */
+  void removeById(Long id);
+
+  /**
+   * Updates the specified version of Flink environment.
+   *
+   * @param version the version of Flink environment to update
+   * @throws IOException if an I/O error occurs during the update process
+   */
+  void update(SparkEnv version) throws IOException;
+
+  /**
+   * Get flink version by application id.
+   *
+   * @param appId the ID of the application
+   * @return the SparkEnv object representing the version of Flink associated with the given app ID
+   */
+  SparkEnv getByAppId(Long appId);
+
+  /**
+   * Sets the specified Flink version as the default.
+   *
+   * @param id The ID of the Flink version to set as the default.
+   */
+  void setDefault(Long id);
+
+  /**
+   * Retrieves the default version of SparkEnv.
+   *
+   * @return the default version of SparkEnv
+   */
+  SparkEnv getDefault();
+
+  /**
+   * Retrieves a Flink environment by ID, if available. If the ID is null or not found, the method
+   * returns the default Flink environment.
+   *
+   * @param id The ID of the Flink environment to retrieve. If null, the default environment will be
+   *     retrieved.
+   * @return The Flink environment with the specified ID, or the default environment if the ID is
+   *     null or not found.
+   */
+  SparkEnv getByIdOrDefault(Long id);
+
+  /**
+   * Synchronizes the configuration file for the given id.
+   *
+   * @param id The id of the configuration file to be synchronized.
+   * @throws IOException If an I/O error occurs while synchronizing the configuration file.
+   */
+  void syncConf(Long id) throws IOException;
+
+  /**
+   * Checks the validity of the given ID.
+   *
+   * @param id The ID to check for validity.
+   */
+  void validity(Long id);
+}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/YarnQueueService.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/YarnQueueService.java
index fd02cac..b44111b 100644
--- a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/YarnQueueService.java
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/YarnQueueService.java
@@ -18,6 +18,7 @@
 package org.apache.streampark.console.core.service;
 
 import org.apache.streampark.common.enums.FlinkExecutionMode;
+import org.apache.streampark.common.enums.SparkExecutionMode;
 import org.apache.streampark.console.base.domain.RestRequest;
 import org.apache.streampark.console.core.bean.ResponseResult;
 import org.apache.streampark.console.core.entity.YarnQueue;
@@ -74,6 +75,8 @@
    */
   void checkQueueLabel(FlinkExecutionMode executionModeEnum, String queueLabel);
 
+  void checkQueueLabel(SparkExecutionMode executionModeEnum, String queueLabel);
+
   /**
    * Determine whether it is the default queue by the given queue label
    *
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/SparkApplicationActionService.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/SparkApplicationActionService.java
new file mode 100644
index 0000000..41cc5cd
--- /dev/null
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/SparkApplicationActionService.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.streampark.console.core.service.application;
+
+import org.apache.streampark.console.base.exception.ApplicationException;
+import org.apache.streampark.console.core.entity.SparkApplication;
+
+import com.baomidou.mybatisplus.extension.service.IService;
+
+/**
+ * This interface represents an Application Operation Service. It extends the IService interface for
+ * handling Application entities.
+ */
+public interface SparkApplicationActionService extends IService<SparkApplication> {
+
+  /**
+   * Starts the specified application.
+   *
+   * @param appParam The application to start.
+   * @param auto True if the application should start automatically, False otherwise.
+   * @throws Exception If an error occurs while starting the application.
+   */
+  void start(SparkApplication appParam, boolean auto) throws Exception;
+
+  /**
+   * Restarts the given application.
+   *
+   * @param appParam The application to restart.
+   * @throws Exception If an error occurs while restarting the application.
+   */
+  void restart(SparkApplication appParam) throws Exception;
+
+  /**
+   * Revokes access for the given application.
+   *
+   * @param appId The application's id for which access needs to be revoked.
+   * @throws ApplicationException if an error occurs while revoking access.
+   */
+  void revoke(Long appId) throws ApplicationException;
+
+  /**
+   * Cancels the given application. Throws an exception if cancellation fails.
+   *
+   * @param appParam the application to be canceled
+   * @throws Exception if cancellation fails
+   */
+  void cancel(SparkApplication appParam) throws Exception;
+
+  /**
+   * Forces the given application to stop.
+   *
+   * @param id the application's id which need to be stopped
+   */
+  void forcedStop(Long id);
+}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/SparkApplicationInfoService.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/SparkApplicationInfoService.java
new file mode 100644
index 0000000..1906d75
--- /dev/null
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/SparkApplicationInfoService.java
@@ -0,0 +1,221 @@
+/*
+ * 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.streampark.console.core.service.application;
+
+import org.apache.streampark.console.base.exception.ApplicationException;
+import org.apache.streampark.console.core.entity.SparkApplication;
+import org.apache.streampark.console.core.enums.AppExistsStateEnum;
+
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+
+import com.baomidou.mybatisplus.extension.service.IService;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * This interface defines the methods that can be used for various utility operations related to an
+ * application.
+ */
+public interface SparkApplicationInfoService extends IService<SparkApplication> {
+
+  /**
+   * Checks the environment for the given application.
+   *
+   * @param appParam the application to check the environment for
+   * @return true if the environment is valid for the application, false otherwise
+   * @throws ApplicationException if an error occurs while checking the environment
+   */
+  boolean checkEnv(SparkApplication appParam) throws ApplicationException;
+
+  /**
+   * Checks the savepoint path for the given application.
+   *
+   * @param appParam the application to check the savepoint path for
+   * @return the check message
+   * @throws Exception if an error occurs while checking the savepoint path
+   */
+  String checkSavepointPath(SparkApplication appParam) throws Exception;
+
+  /**
+   * Checks if the given application meets the required alterations.
+   *
+   * @param appParam The application to be checked.
+   * @return True if the application meets the required alterations, false otherwise.
+   * @throws ApplicationException If an error occurs while checking the alterations.
+   */
+  boolean checkAlter(SparkApplication appParam);
+
+  /**
+   * Checks if a record exists in the database with the given team ID.
+   *
+   * @param teamId The ID of the team to check.
+   * @return true if a record with the given team ID exists, false otherwise.
+   */
+  boolean existsByTeamId(Long teamId);
+
+  /**
+   * Checks if a record exists in the database with the given user ID.
+   *
+   * @param userId The ID of the user to check.
+   * @return true if a record with the given user ID exists, false otherwise.
+   */
+  boolean existsByUserId(Long userId);
+
+  /**
+   * Checks if a job exists for a given Spark environment ID.
+   *
+   * @param sparkEnvId The ID of the Spark environment.
+   * @return true if a job exists for the given Spark environment ID; otherwise, false.
+   */
+  boolean existsBySparkEnvId(Long sparkEnvId);
+
+  /**
+   * Checks if a job is running for a given cluster ID.
+   *
+   * @param clusterId The ID of the cluster.
+   * @return true if a job is running for the given cluster ID; otherwise, false.
+   */
+  boolean existsRunningByClusterId(Long clusterId);
+
+  /**
+   * Checks if there is a job that is associated with the given cluster ID.
+   *
+   * @param clusterId The ID of the cluster.
+   * @return True if a job exists for the given cluster ID, false otherwise.
+   */
+  boolean existsByClusterId(Long clusterId);
+
+  /**
+   * Counts the number of items associated with the given cluster ID.
+   *
+   * @param clusterId The ID of the cluster.
+   * @return The number of items associated with the given cluster ID.
+   */
+  Integer countByClusterId(Long clusterId);
+
+  /**
+   * Counts the number of items associated with the given cluster ID and database type.
+   *
+   * @param clusterId The ID of the cluster.
+   * @param dbType The type of the database.
+   * @return The number of items associated with the given cluster ID and database type.
+   */
+  Integer countAffectedByClusterId(Long clusterId, String dbType);
+
+  /**
+   * Gets the YARN name for the given application.
+   *
+   * @param appConfig The application's config for which to retrieve the YARN name.
+   * @return The YARN name of the application as a String.
+   */
+  String getYarnName(String appConfig);
+
+  /**
+   * Checks if the given application exists in the system.
+   *
+   * @param appParam The application to check for existence.
+   * @return AppExistsState indicating the existence state of the application.
+   */
+  AppExistsStateEnum checkExists(SparkApplication appParam);
+
+  /**
+   * Reads the configuration for the given application and returns it as a String.
+   *
+   * @param appConfig The application's config for which the configuration needs to be read.
+   * @return The configuration for the given application as a String.
+   * @throws IOException If an I/O error occurs while reading the configuration.
+   */
+  String readConf(String appConfig) throws IOException;
+
+  /**
+   * Retrieves the main configuration value for the given Application.
+   *
+   * @param appParam the Application object for which to fetch the main configuration value
+   * @return the main configuration value as a String
+   */
+  String getMain(SparkApplication appParam);
+
+  /**
+   * Returns the dashboard for the specified team.
+   *
+   * @param teamId the ID of the team
+   * @return a map containing the dashboard data
+   */
+  Map<String, Serializable> getDashboardDataMap(Long teamId);
+
+  /**
+   * Retrieves the list of recent Kubernetes namespaces.
+   *
+   * @return The list of recent Kubernetes namespaces as a List of Strings.
+   */
+  List<String> listRecentK8sNamespace();
+
+  /**
+   * Retrieves the list of recent K8s cluster IDs based on the specified execution mode.
+   *
+   * @param executionMode The execution mode to filter the recent K8s cluster IDs. 1: Production
+   *     mode 2: Test mode 3: Development mode -1: All modes
+   * @return The list of recent K8s cluster IDs based on the specified execution mode.
+   */
+  List<String> listRecentK8sClusterId(Integer executionMode);
+
+  /**
+   * Retrieves the recent K8s pod templates.
+   *
+   * @return a List of Strings representing the recent K8s pod templates.
+   */
+  List<String> listRecentK8sPodTemplate();
+
+  /**
+   * Retrieves the list of recent Kubernetes Job Manager Pod templates.
+   *
+   * @return A List of string values representing the recent Kubernetes Job Manager Pod templates.
+   */
+  List<String> listRecentK8sJmPodTemplate();
+
+  /**
+   * Retrieves the list of recent K8s TM pod templates.
+   *
+   * @return The list of recent K8s TM pod templates as a List of String objects.
+   */
+  List<String> listRecentK8sTmPodTemplate();
+
+  /**
+   * Uploads a list of jars to the server for historical reference.
+   *
+   * @return A list of strings representing the names of the uploaded jars.
+   */
+  List<String> listHistoryUploadJars();
+
+  /**
+   * check application before start
+   *
+   * @param id the application's id which need to check before start.
+   * @return org.apache.streampark.console.core.enums.AppExistsStateEnum
+   */
+  AppExistsStateEnum checkStart(Long id);
+
+  /**
+   * @param appName
+   * @return running,submitted, accepted job list in YARN
+   */
+  List<ApplicationReport> getYarnAppReport(String appName);
+}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/SparkApplicationManageService.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/SparkApplicationManageService.java
new file mode 100644
index 0000000..346c5d6
--- /dev/null
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/SparkApplicationManageService.java
@@ -0,0 +1,174 @@
+/*
+ * 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.streampark.console.core.service.application;
+
+import org.apache.streampark.common.enums.SparkExecutionMode;
+import org.apache.streampark.console.base.domain.RestRequest;
+import org.apache.streampark.console.core.entity.SparkApplication;
+
+import com.baomidou.mybatisplus.core.metadata.IPage;
+import com.baomidou.mybatisplus.extension.service.IService;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * The ApplicationManageService interface provides methods to manage applications information. It
+ * extends the IService interface with the Application entity.
+ */
+public interface SparkApplicationManageService extends IService<SparkApplication> {
+
+  /**
+   * Retrieves a page of applications based on the provided parameters.
+   *
+   * @param appParam The application object to be used for filtering the results.
+   * @param request The REST request object containing additional parameters or headers.
+   * @return A page of Application objects based on the provided parameters.
+   */
+  IPage<SparkApplication> page(SparkApplication appParam, RestRequest request);
+
+  /**
+   * Creates a new application.
+   *
+   * @param appParam The application to create.
+   * @return True if the application was successfully created, false otherwise.
+   * @throws IOException If an I/O error occurs.
+   */
+  boolean create(SparkApplication appParam) throws IOException;
+
+  /**
+   * Copies the given Application.
+   *
+   * @param appParam the Application to be copied
+   * @return the size of the copied Application in bytes as a Long value
+   * @throws IOException if there was an error during the copy process
+   */
+  Long copy(SparkApplication appParam) throws IOException;
+
+  /**
+   * Updates the given application.
+   *
+   * @param appParam the application to be updated
+   * @return true if the update was successful, false otherwise
+   */
+  boolean update(SparkApplication appParam);
+
+  /**
+   * Sets the given application to be effective.
+   *
+   * @param appParam the application to be set effective
+   */
+  void toEffective(SparkApplication appParam);
+
+  /**
+   * Persists the metrics of the given application.
+   *
+   * @param appParam The application which metrics need to be persisted.
+   */
+  void persistMetrics(SparkApplication appParam);
+
+  /**
+   * Maps the given application.
+   *
+   * @param appParam The application to be mapped.
+   * @return True if the mapping was successful, false otherwise.
+   */
+  boolean mapping(SparkApplication appParam);
+
+  /**
+   * Checks if the given application is ready to build and update.
+   *
+   * @param appParam the application to check for readiness
+   * @return true if the application is ready to build and update, false otherwise
+   */
+  boolean checkBuildAndUpdate(SparkApplication appParam);
+
+  /**
+   * Deletes the given Application from the system.
+   *
+   * @param appId The Application's id which need to be deleted.
+   * @return True if the deletion was successful, false otherwise.
+   */
+  Boolean remove(Long appId);
+
+  /**
+   * Retrieves the Application with the specified details from the system.
+   *
+   * @param id The Application object's id.
+   * @return The Application object that matches the specified details, or null if no matching
+   *     Application is found.
+   */
+  SparkApplication getApp(Long id);
+
+  /**
+   * Updates the release of the given application.
+   *
+   * @param appParam The application to update the release for.
+   */
+  void updateRelease(SparkApplication appParam);
+
+  /**
+   * Cleans the application by performing necessary cleanup tasks.
+   *
+   * @param appParam The application to clean.
+   */
+  void clean(SparkApplication appParam);
+
+  /**
+   * Retrieves a list of applications by project ID.
+   *
+   * @param id The project ID to search for applications.
+   * @return A list of applications associated with the project ID.
+   */
+  List<SparkApplication> listByProjectId(Long id);
+
+  /**
+   * Changes the ownership of all applications associated with a user.
+   *
+   * @param userId The ID of the user whose applications will be changed.
+   * @param targetUserId The ID of the user who will become the new owner of the applications.
+   */
+  void changeOwnership(Long userId, Long targetUserId);
+
+  /**
+   * Retrieves a list of applications based on the specified team ID.
+   *
+   * @param teamId The ID of the team to retrieve the applications for.
+   * @return A list of Application objects associated with the given team ID.
+   */
+  List<SparkApplication> listByTeamId(Long teamId);
+
+  /**
+   * Retrieves a list of applications by team ID and execution modes.
+   *
+   * @param teamId The ID of the team to filter by
+   * @param executionModeEnums The collection of execution modes to filter by
+   * @return A list of applications that belong to the specified team and have the specified
+   *     execution modes
+   */
+  List<SparkApplication> listByTeamIdAndExecutionModes(
+      Long teamId, Collection<SparkExecutionMode> executionModeEnums);
+
+  /**
+   * Retrieves a list of applications be probing or need to probe.
+   *
+   * @return a list of applications be probing or need to probe.
+   */
+  List<SparkApplication> listProbeApps();
+}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/impl/SparkApplicationActionServiceImpl.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/impl/SparkApplicationActionServiceImpl.java
new file mode 100644
index 0000000..b653751
--- /dev/null
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/impl/SparkApplicationActionServiceImpl.java
@@ -0,0 +1,672 @@
+/*
+ * 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.streampark.console.core.service.application.impl;
+
+import org.apache.streampark.common.Constant;
+import org.apache.streampark.common.conf.ConfigKeys;
+import org.apache.streampark.common.conf.Workspace;
+import org.apache.streampark.common.enums.ApplicationType;
+import org.apache.streampark.common.enums.ClusterState;
+import org.apache.streampark.common.enums.FlinkDevelopmentMode;
+import org.apache.streampark.common.enums.ResolveOrder;
+import org.apache.streampark.common.enums.SparkExecutionMode;
+import org.apache.streampark.common.fs.FsOperator;
+import org.apache.streampark.common.util.AssertUtils;
+import org.apache.streampark.common.util.DeflaterUtils;
+import org.apache.streampark.common.util.ExceptionUtils;
+import org.apache.streampark.common.util.HadoopUtils;
+import org.apache.streampark.common.util.PropertiesUtils;
+import org.apache.streampark.console.base.exception.ApiAlertException;
+import org.apache.streampark.console.base.exception.ApplicationException;
+import org.apache.streampark.console.core.entity.AppBuildPipeline;
+import org.apache.streampark.console.core.entity.ApplicationConfig;
+import org.apache.streampark.console.core.entity.ApplicationLog;
+import org.apache.streampark.console.core.entity.FlinkCluster;
+import org.apache.streampark.console.core.entity.FlinkSql;
+import org.apache.streampark.console.core.entity.Resource;
+import org.apache.streampark.console.core.entity.SavePoint;
+import org.apache.streampark.console.core.entity.SparkApplication;
+import org.apache.streampark.console.core.entity.SparkEnv;
+import org.apache.streampark.console.core.enums.CheckPointTypeEnum;
+import org.apache.streampark.console.core.enums.ConfigFileTypeEnum;
+import org.apache.streampark.console.core.enums.FlinkAppStateEnum;
+import org.apache.streampark.console.core.enums.OperationEnum;
+import org.apache.streampark.console.core.enums.OptionStateEnum;
+import org.apache.streampark.console.core.enums.ReleaseStateEnum;
+import org.apache.streampark.console.core.mapper.SparkApplicationMapper;
+import org.apache.streampark.console.core.service.AppBuildPipeService;
+import org.apache.streampark.console.core.service.ApplicationConfigService;
+import org.apache.streampark.console.core.service.ApplicationLogService;
+import org.apache.streampark.console.core.service.CommonService;
+import org.apache.streampark.console.core.service.FlinkClusterService;
+import org.apache.streampark.console.core.service.FlinkSqlService;
+import org.apache.streampark.console.core.service.ResourceService;
+import org.apache.streampark.console.core.service.SparkEnvService;
+import org.apache.streampark.console.core.service.VariableService;
+import org.apache.streampark.console.core.service.application.SparkApplicationActionService;
+import org.apache.streampark.console.core.service.application.SparkApplicationInfoService;
+import org.apache.streampark.console.core.watcher.FlinkAppHttpWatcher;
+import org.apache.streampark.console.core.watcher.FlinkClusterWatcher;
+import org.apache.streampark.flink.packer.pipeline.BuildResult;
+import org.apache.streampark.flink.packer.pipeline.ShadedBuildResponse;
+import org.apache.streampark.spark.client.SparkClient;
+import org.apache.streampark.spark.client.bean.CancelRequest;
+import org.apache.streampark.spark.client.bean.CancelResponse;
+import org.apache.streampark.spark.client.bean.SubmitRequest;
+import org.apache.streampark.spark.client.bean.SubmitResponse;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.configuration.RestOptions;
+import org.apache.flink.runtime.jobgraph.SavepointConfigOptions;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+
+import com.baomidou.mybatisplus.core.conditions.update.LambdaUpdateWrapper;
+import com.baomidou.mybatisplus.core.toolkit.Wrappers;
+import com.baomidou.mybatisplus.extension.service.impl.ServiceImpl;
+import com.google.common.collect.Sets;
+import lombok.extern.slf4j.Slf4j;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.beans.factory.annotation.Qualifier;
+import org.springframework.stereotype.Service;
+
+import java.io.File;
+import java.net.URI;
+import java.util.Date;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executor;
+
+@Slf4j
+@Service
+public class SparkApplicationActionServiceImpl
+    extends ServiceImpl<SparkApplicationMapper, SparkApplication>
+    implements SparkApplicationActionService {
+
+  @Qualifier("streamparkDeployExecutor")
+  @Autowired
+  private Executor executorService;
+
+  @Autowired private SparkApplicationInfoService applicationInfoService;
+
+  @Autowired private ApplicationConfigService configService;
+
+  @Autowired private ApplicationLogService applicationLogService;
+
+  @Autowired private SparkEnvService sparkEnvService;
+
+  @Autowired private FlinkSqlService flinkSqlService;
+
+  @Autowired private CommonService commonService;
+
+  @Autowired private AppBuildPipeService appBuildPipeService;
+
+  @Autowired private FlinkClusterService flinkClusterService;
+
+  @Autowired private VariableService variableService;
+
+  @Autowired private ResourceService resourceService;
+
+  @Autowired private FlinkClusterWatcher flinkClusterWatcher;
+
+  private final Map<Long, CompletableFuture<SubmitResponse>> startFutureMap =
+      new ConcurrentHashMap<>();
+
+  private final Map<Long, CompletableFuture<CancelResponse>> cancelFutureMap =
+      new ConcurrentHashMap<>();
+
+  @Override
+  public void revoke(Long appId) throws ApplicationException {
+    SparkApplication application = getById(appId);
+    ApiAlertException.throwIfNull(
+        application, String.format("The application id=%s not found, revoke failed.", appId));
+
+    // 1) delete files that have been published to workspace
+    application.getFsOperator().delete(application.getAppHome());
+
+    // 2) rollback the files to the workspace
+    //    backUpService.revoke(application);
+
+    // 3) restore related status
+    LambdaUpdateWrapper<SparkApplication> updateWrapper = Wrappers.lambdaUpdate();
+    updateWrapper.eq(SparkApplication::getId, application.getId());
+    if (application.isSparkSqlJob()) {
+      updateWrapper.set(SparkApplication::getRelease, ReleaseStateEnum.FAILED.get());
+    } else {
+      updateWrapper.set(SparkApplication::getRelease, ReleaseStateEnum.NEED_RELEASE.get());
+    }
+    if (!application.isRunning()) {
+      updateWrapper.set(SparkApplication::getState, FlinkAppStateEnum.REVOKED.getValue());
+    }
+    baseMapper.update(null, updateWrapper);
+  }
+
+  @Override
+  public void restart(SparkApplication appParam) throws Exception {
+    this.cancel(appParam);
+    this.start(appParam, false);
+  }
+
+  @Override
+  public void forcedStop(Long id) {
+    CompletableFuture<SubmitResponse> startFuture = startFutureMap.remove(id);
+    CompletableFuture<CancelResponse> cancelFuture = cancelFutureMap.remove(id);
+    SparkApplication application = this.baseMapper.selectApp(id);
+    if (startFuture != null) {
+      startFuture.cancel(true);
+    }
+    if (cancelFuture != null) {
+      cancelFuture.cancel(true);
+    }
+    if (startFuture == null && cancelFuture == null) {
+      this.doStopped(id);
+    }
+  }
+
+  @Override
+  public void cancel(SparkApplication appParam) throws Exception {
+    FlinkAppHttpWatcher.setOptionState(appParam.getId(), OptionStateEnum.CANCELLING);
+    SparkApplication application = getById(appParam.getId());
+    application.setState(FlinkAppStateEnum.CANCELLING.getValue());
+
+    ApplicationLog applicationLog = new ApplicationLog();
+    applicationLog.setOptionName(OperationEnum.CANCEL.getValue());
+    applicationLog.setAppId(application.getId());
+    applicationLog.setJobManagerUrl(application.getJobManagerUrl());
+    applicationLog.setOptionTime(new Date());
+    applicationLog.setYarnAppId(application.getClusterId());
+    applicationLog.setUserId(commonService.getUserId());
+
+    if (appParam.getSavePointed()) {
+      FlinkAppHttpWatcher.addSavepoint(application.getId());
+      application.setOptionState(OptionStateEnum.SAVEPOINTING.getValue());
+    } else {
+      application.setOptionState(OptionStateEnum.CANCELLING.getValue());
+    }
+
+    application.setOptionTime(new Date());
+    this.baseMapper.updateById(application);
+
+    Long userId = commonService.getUserId();
+    if (!application.getUserId().equals(userId)) {
+      FlinkAppHttpWatcher.addCanceledApp(application.getId(), userId);
+    }
+
+    SparkEnv sparkEnv = sparkEnvService.getById(application.getVersionId());
+
+    String clusterId = null;
+    if (SparkExecutionMode.isYarnMode(application.getExecutionMode())) {
+      clusterId = application.getAppId();
+    }
+
+    Map<String, Object> properties = new HashMap<>();
+
+    if (SparkExecutionMode.isRemoteMode(application.getSparkExecutionMode())) {
+      FlinkCluster cluster = flinkClusterService.getById(application.getSparkClusterId());
+      ApiAlertException.throwIfNull(
+          cluster,
+          String.format(
+              "The clusterId=%s cannot be find, maybe the clusterId is wrong or "
+                  + "the cluster has been deleted. Please contact the Admin.",
+              application.getSparkClusterId()));
+      URI activeAddress = cluster.getRemoteURI();
+      properties.put(RestOptions.ADDRESS.key(), activeAddress.getHost());
+      properties.put(RestOptions.PORT.key(), activeAddress.getPort());
+    }
+
+    CancelRequest cancelRequest =
+        new CancelRequest(
+            application.getId(),
+            sparkEnv.getSparkVersion(),
+            SparkExecutionMode.of(application.getExecutionMode()),
+            properties,
+            clusterId,
+            application.getJobId(),
+            appParam.getDrain(),
+            appParam.getNativeFormat());
+
+    final Date triggerTime = new Date();
+    CompletableFuture<CancelResponse> cancelFuture =
+        CompletableFuture.supplyAsync(() -> SparkClient.cancel(cancelRequest), executorService);
+
+    cancelFutureMap.put(application.getId(), cancelFuture);
+
+    cancelFuture.whenComplete(
+        (cancelResponse, throwable) -> {
+          cancelFutureMap.remove(application.getId());
+
+          if (throwable != null) {
+            String exception = ExceptionUtils.stringifyException(throwable);
+            applicationLog.setException(exception);
+            applicationLog.setSuccess(false);
+            applicationLogService.save(applicationLog);
+
+            if (throwable instanceof CancellationException) {
+              doStopped(application.getId());
+            } else {
+              log.error("stop flink job failed.", throwable);
+              application.setOptionState(OptionStateEnum.NONE.getValue());
+              application.setState(FlinkAppStateEnum.FAILED.getValue());
+              updateById(application);
+
+              FlinkAppHttpWatcher.unWatching(application.getId());
+            }
+            return;
+          }
+
+          applicationLog.setSuccess(true);
+          // save log...
+          applicationLogService.save(applicationLog);
+
+          if (cancelResponse != null && cancelResponse.savePointDir() != null) {
+            String savePointDir = cancelResponse.savePointDir();
+            log.info("savePoint path: {}", savePointDir);
+            SavePoint savePoint = new SavePoint();
+            savePoint.setPath(savePointDir);
+            savePoint.setAppId(application.getId());
+            savePoint.setLatest(true);
+            savePoint.setType(CheckPointTypeEnum.SAVEPOINT.get());
+            savePoint.setCreateTime(new Date());
+            savePoint.setTriggerTime(triggerTime);
+          }
+        });
+  }
+
+  @Override
+  public void start(SparkApplication appParam, boolean auto) throws Exception {
+    // 1) check application
+    final SparkApplication application = getById(appParam.getId());
+    AssertUtils.notNull(application);
+    ApiAlertException.throwIfTrue(
+        !application.isCanBeStart(), "[StreamPark] The application cannot be started repeatedly.");
+
+    if (SparkExecutionMode.isRemoteMode(application.getSparkExecutionMode())) {
+      checkBeforeStart(application);
+    }
+
+    if (SparkExecutionMode.isYarnMode(application.getSparkExecutionMode())) {
+
+      ApiAlertException.throwIfTrue(
+          !applicationInfoService.getYarnAppReport(application.getJobName()).isEmpty(),
+          "[StreamPark] The same task name is already running in the yarn queue");
+    }
+
+    AppBuildPipeline buildPipeline = appBuildPipeService.getById(application.getId());
+    AssertUtils.notNull(buildPipeline);
+
+    SparkEnv sparkEnv = sparkEnvService.getByIdOrDefault(application.getVersionId());
+
+    ApiAlertException.throwIfNull(sparkEnv, "[StreamPark] can no found flink version");
+
+    // if manually started, clear the restart flag
+    if (!auto) {
+      application.setRestartCount(0);
+    } else {
+      if (!application.isNeedRestartOnFailed()) {
+        return;
+      }
+      appParam.setSavePointed(true);
+      application.setRestartCount(application.getRestartCount() + 1);
+    }
+
+    // 2) update app state to starting...
+    starting(application);
+
+    String jobId = new JobID().toHexString();
+    ApplicationLog applicationLog = new ApplicationLog();
+    applicationLog.setOptionName(OperationEnum.START.getValue());
+    applicationLog.setAppId(application.getId());
+    applicationLog.setOptionTime(new Date());
+    applicationLog.setUserId(commonService.getUserId());
+
+    // set the latest to Effective, (it will only become the current effective at this time)
+    //    applicationManageService.toEffective(application);
+
+    Map<String, Object> extraParameter = new HashMap<>(0);
+    if (application.isSparkSqlJob()) {
+      FlinkSql flinkSql = flinkSqlService.getEffective(application.getId(), true);
+      // Get the sql of the replaced placeholder
+      String realSql = variableService.replaceVariable(application.getTeamId(), flinkSql.getSql());
+      flinkSql.setSql(DeflaterUtils.zipString(realSql));
+      extraParameter.put(ConfigKeys.KEY_FLINK_SQL(null), flinkSql.getSql());
+    }
+
+    Tuple2<String, String> userJarAndAppConf = getUserJarAndAppConf(sparkEnv, application);
+    String flinkUserJar = userJarAndAppConf.f0;
+    String appConf = userJarAndAppConf.f1;
+
+    BuildResult buildResult = buildPipeline.getBuildResult();
+    if (SparkExecutionMode.YARN_CLUSTER == application.getSparkExecutionMode()
+        || SparkExecutionMode.YARN_CLIENT == application.getSparkExecutionMode()) {
+      buildResult = new ShadedBuildResponse(null, flinkUserJar, true);
+    }
+
+    // Get the args after placeholder replacement
+    String applicationArgs =
+        variableService.replaceVariable(application.getTeamId(), application.getArgs());
+
+    SubmitRequest submitRequest =
+        new SubmitRequest(
+            sparkEnv.getSparkVersion(),
+            SparkExecutionMode.of(application.getExecutionMode()),
+            getProperties(application),
+            sparkEnv.getSparkConf(),
+            FlinkDevelopmentMode.of(application.getJobType()),
+            application.getId(),
+            jobId,
+            application.getJobName(),
+            appConf,
+            application.getApplicationType(),
+            applicationArgs,
+            application.getHadoopUser(),
+            buildResult,
+            extraParameter);
+
+    CompletableFuture<SubmitResponse> future =
+        CompletableFuture.supplyAsync(() -> SparkClient.submit(submitRequest), executorService);
+
+    startFutureMap.put(application.getId(), future);
+
+    future.whenComplete(
+        (response, throwable) -> {
+          // 1) remove Future
+          startFutureMap.remove(application.getId());
+
+          // 2) exception
+          if (throwable != null) {
+            String exception = ExceptionUtils.stringifyException(throwable);
+            applicationLog.setException(exception);
+            applicationLog.setSuccess(false);
+            applicationLogService.save(applicationLog);
+            if (throwable instanceof CancellationException) {
+              doStopped(application.getId());
+            } else {
+              SparkApplication app = getById(appParam.getId());
+              app.setState(FlinkAppStateEnum.FAILED.getValue());
+              app.setOptionState(OptionStateEnum.NONE.getValue());
+              updateById(app);
+              FlinkAppHttpWatcher.unWatching(appParam.getId());
+            }
+            return;
+          }
+
+          // 3) success
+          applicationLog.setSuccess(true);
+          if (response.sparkConfig() != null) {
+            String jmMemory = response.sparkConfig().get(ConfigKeys.KEY_FLINK_JM_PROCESS_MEMORY());
+            if (jmMemory != null) {
+              application.setJmMemory(MemorySize.parse(jmMemory).getMebiBytes());
+            }
+            String tmMemory = response.sparkConfig().get(ConfigKeys.KEY_FLINK_TM_PROCESS_MEMORY());
+            if (tmMemory != null) {
+              application.setTmMemory(MemorySize.parse(tmMemory).getMebiBytes());
+            }
+          }
+          application.setAppId(response.clusterId());
+          if (StringUtils.isNoneEmpty(response.jobId())) {
+            application.setJobId(response.jobId());
+          }
+
+          if (StringUtils.isNoneEmpty(response.jobManagerUrl())) {
+            application.setJobManagerUrl(response.jobManagerUrl());
+            applicationLog.setJobManagerUrl(response.jobManagerUrl());
+          }
+          applicationLog.setYarnAppId(response.clusterId());
+          application.setStartTime(new Date());
+          application.setEndTime(null);
+
+          // if start completed, will be added task to tracking queue
+          FlinkAppHttpWatcher.setOptionState(appParam.getId(), OptionStateEnum.STARTING);
+          //            FlinkAppHttpWatcher.doWatching(application);
+
+          // update app
+          updateById(application);
+          // save log
+          applicationLogService.save(applicationLog);
+        });
+  }
+
+  /**
+   * Check whether a job with the same name is running in the yarn queue
+   *
+   * @param jobName
+   * @return
+   */
+  private boolean checkAppRepeatInYarn(String jobName) {
+    try {
+      YarnClient yarnClient = HadoopUtils.yarnClient();
+      Set<String> types =
+          Sets.newHashSet(
+              ApplicationType.STREAMPARK_FLINK.getName(), ApplicationType.APACHE_FLINK.getName());
+      EnumSet<YarnApplicationState> states =
+          EnumSet.of(YarnApplicationState.RUNNING, YarnApplicationState.ACCEPTED);
+      List<ApplicationReport> applications = yarnClient.getApplications(types, states);
+      for (ApplicationReport report : applications) {
+        if (report.getName().equals(jobName)) {
+          return true;
+        }
+      }
+      return false;
+    } catch (Exception e) {
+      throw new RuntimeException("The yarn api is abnormal. Ensure that yarn is running properly.");
+    }
+  }
+
+  private void starting(SparkApplication application) {
+    application.setState(FlinkAppStateEnum.STARTING.getValue());
+    application.setOptionTime(new Date());
+    updateById(application);
+  }
+
+  private Tuple2<String, String> getUserJarAndAppConf(
+      SparkEnv sparkEnv, SparkApplication application) {
+    SparkExecutionMode executionModeEnum = application.getSparkExecutionMode();
+    ApplicationConfig applicationConfig = configService.getEffective(application.getId());
+
+    ApiAlertException.throwIfNull(
+        executionModeEnum, "ExecutionMode can't be null, start application failed.");
+
+    String flinkUserJar = null;
+    String appConf = null;
+
+    switch (application.getDevelopmentMode()) {
+      case FLINK_SQL:
+        FlinkSql flinkSql = flinkSqlService.getEffective(application.getId(), false);
+        AssertUtils.notNull(flinkSql);
+        // 1) dist_userJar
+        String sqlDistJar = commonService.getSqlClientJar(sparkEnv);
+        // 2) appConfig
+        appConf =
+            applicationConfig == null
+                ? null
+                : String.format("yaml://%s", applicationConfig.getContent());
+        // 3) client
+        if (SparkExecutionMode.YARN_CLUSTER == executionModeEnum) {
+          String clientPath = Workspace.remote().APP_CLIENT();
+          flinkUserJar = String.format("%s/%s", clientPath, sqlDistJar);
+        }
+        break;
+
+      case PYFLINK:
+        Resource resource =
+            resourceService.findByResourceName(application.getTeamId(), application.getJar());
+
+        ApiAlertException.throwIfNull(
+            resource, "pyflink file can't be null, start application failed.");
+
+        ApiAlertException.throwIfNull(
+            resource.getFilePath(), "pyflink file can't be null, start application failed.");
+
+        ApiAlertException.throwIfFalse(
+            resource.getFilePath().endsWith(Constant.PYTHON_SUFFIX),
+            "pyflink format error, must be a \".py\" suffix, start application failed.");
+
+        flinkUserJar = resource.getFilePath();
+        break;
+
+      case CUSTOM_CODE:
+        if (application.isUploadJob()) {
+          appConf =
+              String.format(
+                  "json://{\"%s\":\"%s\"}",
+                  ConfigKeys.KEY_FLINK_APPLICATION_MAIN_CLASS(), application.getMainClass());
+        } else {
+          switch (application.getApplicationType()) {
+            case STREAMPARK_SPARK:
+              ConfigFileTypeEnum fileType = ConfigFileTypeEnum.of(applicationConfig.getFormat());
+              if (fileType != null && ConfigFileTypeEnum.UNKNOWN != fileType) {
+                appConf =
+                    String.format(
+                        "%s://%s", fileType.getTypeName(), applicationConfig.getContent());
+              } else {
+                throw new IllegalArgumentException(
+                    "application' config type error,must be ( yaml| properties| hocon )");
+              }
+              break;
+            case APACHE_SPARK:
+              appConf =
+                  String.format(
+                      "json://{\"%s\":\"%s\"}",
+                      ConfigKeys.KEY_FLINK_APPLICATION_MAIN_CLASS(), application.getMainClass());
+              break;
+            default:
+              throw new IllegalArgumentException(
+                  "[StreamPark] ApplicationType must be (StreamPark flink | Apache flink)... ");
+          }
+        }
+
+        if (SparkExecutionMode.YARN_CLUSTER == executionModeEnum) {
+          switch (application.getApplicationType()) {
+            case STREAMPARK_SPARK:
+              flinkUserJar =
+                  String.format(
+                      "%s/%s",
+                      application.getAppLib(), application.getModule().concat(Constant.JAR_SUFFIX));
+              break;
+            case APACHE_SPARK:
+              flinkUserJar = String.format("%s/%s", application.getAppHome(), application.getJar());
+              if (!FsOperator.hdfs().exists(flinkUserJar)) {
+                resource =
+                    resourceService.findByResourceName(
+                        application.getTeamId(), application.getJar());
+                if (resource != null && StringUtils.isNotBlank(resource.getFilePath())) {
+                  flinkUserJar =
+                      String.format(
+                          "%s/%s",
+                          application.getAppHome(), new File(resource.getFilePath()).getName());
+                }
+              }
+              break;
+            default:
+              throw new IllegalArgumentException(
+                  "[StreamPark] ApplicationType must be (StreamPark flink | Apache flink)... ");
+          }
+        }
+        break;
+    }
+    return Tuple2.of(flinkUserJar, appConf);
+  }
+
+  private Map<String, Object> getProperties(SparkApplication application) {
+    Map<String, Object> properties = new HashMap<>(application.getOptionMap());
+    if (SparkExecutionMode.isRemoteMode(application.getSparkExecutionMode())) {
+      FlinkCluster cluster = flinkClusterService.getById(application.getSparkClusterId());
+      ApiAlertException.throwIfNull(
+          cluster,
+          String.format(
+              "The clusterId=%s can't be find, maybe the clusterId is wrong or "
+                  + "the cluster has been deleted. Please contact the Admin.",
+              application.getSparkClusterId()));
+      URI activeAddress = cluster.getRemoteURI();
+      properties.put(RestOptions.ADDRESS.key(), activeAddress.getHost());
+      properties.put(RestOptions.PORT.key(), activeAddress.getPort());
+    } else if (SparkExecutionMode.isYarnMode(application.getSparkExecutionMode())) {
+      String yarnQueue =
+          (String) application.getHotParamsMap().get(ConfigKeys.KEY_YARN_APP_QUEUE());
+      String yarnLabelExpr =
+          (String) application.getHotParamsMap().get(ConfigKeys.KEY_YARN_APP_NODE_LABEL());
+      Optional.ofNullable(yarnQueue)
+          .ifPresent(yq -> properties.put(ConfigKeys.KEY_YARN_APP_QUEUE(), yq));
+      Optional.ofNullable(yarnLabelExpr)
+          .ifPresent(yLabel -> properties.put(ConfigKeys.KEY_YARN_APP_NODE_LABEL(), yLabel));
+    }
+
+    if (application.getAllowNonRestored()) {
+      properties.put(SavepointConfigOptions.SAVEPOINT_IGNORE_UNCLAIMED_STATE.key(), true);
+    }
+
+    Map<String, String> dynamicProperties =
+        PropertiesUtils.extractDynamicPropertiesAsJava(application.getDynamicProperties());
+    properties.putAll(dynamicProperties);
+    ResolveOrder resolveOrder = ResolveOrder.of(application.getResolveOrder());
+    if (resolveOrder != null) {
+      properties.put(CoreOptions.CLASSLOADER_RESOLVE_ORDER.key(), resolveOrder.getName());
+    }
+
+    return properties;
+  }
+
+  private void doStopped(Long id) {
+    SparkApplication application = getById(id);
+    application.setOptionState(OptionStateEnum.NONE.getValue());
+    application.setState(FlinkAppStateEnum.CANCELED.getValue());
+    application.setOptionTime(new Date());
+    updateById(application);
+    // re-tracking flink job on kubernetes and logging exception
+    FlinkAppHttpWatcher.unWatching(application.getId());
+    // kill application
+    if (SparkExecutionMode.isYarnMode(application.getSparkExecutionMode())) {
+      try {
+        List<ApplicationReport> applications =
+            applicationInfoService.getYarnAppReport(application.getJobName());
+        if (!applications.isEmpty()) {
+          YarnClient yarnClient = HadoopUtils.yarnClient();
+          yarnClient.killApplication(applications.get(0).getApplicationId());
+        }
+      } catch (Exception ignored) {
+      }
+    }
+  }
+
+  /* check flink cluster before job start job */
+  private void checkBeforeStart(SparkApplication application) {
+    SparkEnv sparkEnv = sparkEnvService.getByAppId(application.getId());
+    ApiAlertException.throwIfNull(sparkEnv, "[StreamPark] can no found flink version");
+
+    ApiAlertException.throwIfFalse(
+        flinkClusterService.existsByFlinkEnvId(sparkEnv.getId()),
+        "[StreamPark] The flink cluster don't exist, please check it");
+
+    FlinkCluster flinkCluster = flinkClusterService.getById(application.getSparkClusterId());
+    ApiAlertException.throwIfFalse(
+        flinkClusterWatcher.getClusterState(flinkCluster) == ClusterState.RUNNING,
+        "[StreamPark] The flink cluster not running, please start it");
+  }
+}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/impl/SparkApplicationInfoServiceImpl.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/impl/SparkApplicationInfoServiceImpl.java
new file mode 100644
index 0000000..49147b0
--- /dev/null
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/impl/SparkApplicationInfoServiceImpl.java
@@ -0,0 +1,476 @@
+/*
+ * 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.streampark.console.core.service.application.impl;
+
+import org.apache.streampark.common.Constant;
+import org.apache.streampark.common.conf.K8sFlinkConfig;
+import org.apache.streampark.common.conf.Workspace;
+import org.apache.streampark.common.enums.ApplicationType;
+import org.apache.streampark.common.enums.SparkExecutionMode;
+import org.apache.streampark.common.fs.LfsOperator;
+import org.apache.streampark.common.util.ExceptionUtils;
+import org.apache.streampark.common.util.HadoopUtils;
+import org.apache.streampark.common.util.Utils;
+import org.apache.streampark.common.util.YarnUtils;
+import org.apache.streampark.console.base.exception.ApiAlertException;
+import org.apache.streampark.console.base.exception.ApiDetailException;
+import org.apache.streampark.console.base.exception.ApplicationException;
+import org.apache.streampark.console.core.entity.Application;
+import org.apache.streampark.console.core.entity.FlinkCluster;
+import org.apache.streampark.console.core.entity.SparkApplication;
+import org.apache.streampark.console.core.entity.SparkEnv;
+import org.apache.streampark.console.core.enums.AppExistsStateEnum;
+import org.apache.streampark.console.core.enums.FlinkAppStateEnum;
+import org.apache.streampark.console.core.mapper.SparkApplicationMapper;
+import org.apache.streampark.console.core.metrics.flink.JobsOverview;
+import org.apache.streampark.console.core.runner.EnvInitializer;
+import org.apache.streampark.console.core.service.FlinkClusterService;
+import org.apache.streampark.console.core.service.SavePointService;
+import org.apache.streampark.console.core.service.SparkEnvService;
+import org.apache.streampark.console.core.service.application.SparkApplicationInfoService;
+import org.apache.streampark.console.core.watcher.FlinkAppHttpWatcher;
+import org.apache.streampark.console.core.watcher.FlinkClusterWatcher;
+import org.apache.streampark.console.core.watcher.FlinkK8sObserverStub;
+import org.apache.streampark.flink.core.conf.ParameterCli;
+import org.apache.streampark.flink.kubernetes.FlinkK8sWatcher;
+import org.apache.streampark.flink.kubernetes.model.FlinkMetricCV;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+
+import com.baomidou.mybatisplus.core.conditions.query.LambdaQueryWrapper;
+import com.baomidou.mybatisplus.extension.service.impl.ServiceImpl;
+import com.google.common.collect.Sets;
+import lombok.extern.slf4j.Slf4j;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Service;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.Serializable;
+import java.net.URI;
+import java.util.Arrays;
+import java.util.Base64;
+import java.util.Comparator;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.streampark.common.enums.StorageType.LFS;
+
+@Slf4j
+@Service
+public class SparkApplicationInfoServiceImpl
+    extends ServiceImpl<SparkApplicationMapper, SparkApplication>
+    implements SparkApplicationInfoService {
+
+  private static final int DEFAULT_HISTORY_RECORD_LIMIT = 25;
+
+  private static final int DEFAULT_HISTORY_POD_TMPL_RECORD_LIMIT = 5;
+
+  private static final Pattern JOB_NAME_PATTERN =
+      Pattern.compile("^[.\\x{4e00}-\\x{9fa5}A-Za-z\\d_\\-\\s]+$");
+
+  private static final Pattern SINGLE_SPACE_PATTERN = Pattern.compile("^\\S+(\\s\\S+)*$");
+
+  @Autowired private SparkEnvService sparkEnvService;
+
+  @Autowired private SavePointService savePointService;
+
+  @Autowired private EnvInitializer envInitializer;
+
+  @Autowired private FlinkK8sWatcher k8SFlinkTrackMonitor;
+
+  @Autowired private FlinkK8sObserverStub flinkK8sObserver;
+
+  @Autowired private FlinkClusterService flinkClusterService;
+
+  @Autowired private FlinkClusterWatcher flinkClusterWatcher;
+
+  @Override
+  public Map<String, Serializable> getDashboardDataMap(Long teamId) {
+    JobsOverview.Task overview = new JobsOverview.Task();
+    Integer totalJmMemory = 0;
+    Integer totalTmMemory = 0;
+    Integer totalTm = 0;
+    Integer totalSlot = 0;
+    Integer availableSlot = 0;
+    Integer runningJob = 0;
+
+    // stat metrics from other than kubernetes mode
+    for (Application app : FlinkAppHttpWatcher.getWatchingApps()) {
+      if (!teamId.equals(app.getTeamId())) {
+        continue;
+      }
+      if (app.getJmMemory() != null) {
+        totalJmMemory += app.getJmMemory();
+      }
+      if (app.getTmMemory() != null) {
+        totalTmMemory += app.getTmMemory() * (app.getTotalTM() == null ? 1 : app.getTotalTM());
+      }
+      if (app.getTotalTM() != null) {
+        totalTm += app.getTotalTM();
+      }
+      if (app.getTotalSlot() != null) {
+        totalSlot += app.getTotalSlot();
+      }
+      if (app.getAvailableSlot() != null) {
+        availableSlot += app.getAvailableSlot();
+      }
+      if (app.getState() == FlinkAppStateEnum.RUNNING.getValue()) {
+        runningJob++;
+      }
+      JobsOverview.Task task = app.getOverview();
+      if (task != null) {
+        overview.setTotal(overview.getTotal() + task.getTotal());
+        overview.setCreated(overview.getCreated() + task.getCreated());
+        overview.setScheduled(overview.getScheduled() + task.getScheduled());
+        overview.setDeploying(overview.getDeploying() + task.getDeploying());
+        overview.setRunning(overview.getRunning() + task.getRunning());
+        overview.setFinished(overview.getFinished() + task.getFinished());
+        overview.setCanceling(overview.getCanceling() + task.getCanceling());
+        overview.setCanceled(overview.getCanceled() + task.getCanceled());
+        overview.setFailed(overview.getFailed() + task.getFailed());
+        overview.setReconciling(overview.getReconciling() + task.getReconciling());
+      }
+    }
+
+    // merge metrics from flink kubernetes cluster
+    FlinkMetricCV k8sMetric;
+    if (K8sFlinkConfig.isV2Enabled()) {
+      k8sMetric = flinkK8sObserver.getAggClusterMetricCV(teamId);
+    } else {
+      k8sMetric = k8SFlinkTrackMonitor.getAccGroupMetrics(teamId.toString());
+    }
+    if (k8sMetric != null) {
+      totalJmMemory += k8sMetric.totalJmMemory();
+      totalTmMemory += k8sMetric.totalTmMemory();
+      totalTm += k8sMetric.totalTm();
+      totalSlot += k8sMetric.totalSlot();
+      availableSlot += k8sMetric.availableSlot();
+      runningJob += k8sMetric.runningJob();
+      overview.setTotal(overview.getTotal() + k8sMetric.totalJob());
+      overview.setRunning(overview.getRunning() + k8sMetric.runningJob());
+      overview.setFinished(overview.getFinished() + k8sMetric.finishedJob());
+      overview.setCanceled(overview.getCanceled() + k8sMetric.cancelledJob());
+      overview.setFailed(overview.getFailed() + k8sMetric.failedJob());
+    }
+
+    // result json
+    Map<String, Serializable> dashboardDataMap = new HashMap<>(8);
+    dashboardDataMap.put("task", overview);
+    dashboardDataMap.put("jmMemory", totalJmMemory);
+    dashboardDataMap.put("tmMemory", totalTmMemory);
+    dashboardDataMap.put("totalTM", totalTm);
+    dashboardDataMap.put("availableSlot", availableSlot);
+    dashboardDataMap.put("totalSlot", totalSlot);
+    dashboardDataMap.put("runningJob", runningJob);
+
+    return dashboardDataMap;
+  }
+
+  @Override
+  public boolean checkEnv(SparkApplication appParam) throws ApplicationException {
+    SparkApplication application = getById(appParam.getId());
+    try {
+      SparkEnv sparkEnv;
+      if (application.getVersionId() != null) {
+        sparkEnv = sparkEnvService.getByIdOrDefault(application.getVersionId());
+      } else {
+        sparkEnv = sparkEnvService.getDefault();
+      }
+      if (sparkEnv == null) {
+        return false;
+      }
+      envInitializer.checkSparkEnv(application.getStorageType(), sparkEnv);
+      envInitializer.storageInitialize(application.getStorageType());
+
+      if (SparkExecutionMode.REMOTE == application.getSparkExecutionMode()) {
+        FlinkCluster flinkCluster = flinkClusterService.getById(application.getSparkClusterId());
+        boolean conned = flinkClusterWatcher.verifyClusterConnection(flinkCluster);
+        if (!conned) {
+          throw new ApiAlertException("the target cluster is unavailable, please check!");
+        }
+      }
+      return true;
+    } catch (Exception e) {
+      log.error(ExceptionUtils.stringifyException(e));
+      throw new ApiDetailException(e);
+    }
+  }
+
+  @Override
+  public boolean checkAlter(SparkApplication appParam) {
+    Long appId = appParam.getId();
+    if (FlinkAppStateEnum.CANCELED != appParam.getStateEnum()) {
+      return false;
+    }
+    long cancelUserId = FlinkAppHttpWatcher.getCanceledJobUserId(appId);
+    long appUserId = appParam.getUserId();
+    return cancelUserId != -1 && cancelUserId != appUserId;
+  }
+
+  @Override
+  public boolean existsByTeamId(Long teamId) {
+    return baseMapper.exists(
+        new LambdaQueryWrapper<SparkApplication>().eq(SparkApplication::getTeamId, teamId));
+  }
+
+  @Override
+  public boolean existsByUserId(Long userId) {
+    return baseMapper.exists(
+        new LambdaQueryWrapper<SparkApplication>().eq(SparkApplication::getUserId, userId));
+  }
+
+  @Override
+  public boolean existsRunningByClusterId(Long clusterId) {
+    return baseMapper.existsRunningJobByClusterId(clusterId)
+        || FlinkAppHttpWatcher.getWatchingApps().stream()
+            .anyMatch(
+                application ->
+                    clusterId.equals(application.getFlinkClusterId())
+                        && FlinkAppStateEnum.RUNNING == application.getStateEnum());
+  }
+
+  @Override
+  public boolean existsByClusterId(Long clusterId) {
+    return baseMapper.exists(
+        new LambdaQueryWrapper<SparkApplication>()
+            .eq(SparkApplication::getSparkClusterId, clusterId));
+  }
+
+  @Override
+  public Integer countByClusterId(Long clusterId) {
+    return baseMapper
+        .selectCount(
+            new LambdaQueryWrapper<SparkApplication>()
+                .eq(SparkApplication::getSparkClusterId, clusterId))
+        .intValue();
+  }
+
+  @Override
+  public Integer countAffectedByClusterId(Long clusterId, String dbType) {
+    return baseMapper.countAffectedByClusterId(clusterId, dbType);
+  }
+
+  @Override
+  public boolean existsBySparkEnvId(Long sparkEnvId) {
+    return baseMapper.exists(
+        new LambdaQueryWrapper<SparkApplication>().eq(SparkApplication::getVersionId, sparkEnvId));
+  }
+
+  @Override
+  public List<String> listRecentK8sNamespace() {
+    return baseMapper.selectRecentK8sNamespaces(DEFAULT_HISTORY_RECORD_LIMIT);
+  }
+
+  @Override
+  public List<String> listRecentK8sClusterId(Integer executionMode) {
+    return baseMapper.selectRecentK8sClusterIds(executionMode, DEFAULT_HISTORY_RECORD_LIMIT);
+  }
+
+  @Override
+  public List<String> listRecentK8sPodTemplate() {
+    return baseMapper.selectRecentK8sPodTemplates(DEFAULT_HISTORY_POD_TMPL_RECORD_LIMIT);
+  }
+
+  @Override
+  public List<String> listRecentK8sJmPodTemplate() {
+    return baseMapper.selectRecentK8sJmPodTemplates(DEFAULT_HISTORY_POD_TMPL_RECORD_LIMIT);
+  }
+
+  @Override
+  public List<String> listRecentK8sTmPodTemplate() {
+    return baseMapper.selectRecentK8sTmPodTemplates(DEFAULT_HISTORY_POD_TMPL_RECORD_LIMIT);
+  }
+
+  @Override
+  public List<String> listHistoryUploadJars() {
+    return Arrays.stream(LfsOperator.listDir(Workspace.of(LFS).APP_UPLOADS()))
+        .filter(File::isFile)
+        .sorted(Comparator.comparingLong(File::lastModified).reversed())
+        .map(File::getName)
+        .filter(fn -> fn.endsWith(Constant.JAR_SUFFIX))
+        .limit(DEFAULT_HISTORY_RECORD_LIMIT)
+        .collect(Collectors.toList());
+  }
+
+  @Override
+  public AppExistsStateEnum checkStart(Long id) {
+    SparkApplication application = getById(id);
+    if (application == null) {
+      return AppExistsStateEnum.INVALID;
+    }
+    if (SparkExecutionMode.isYarnMode(application.getExecutionMode())) {
+      boolean exists = !getYarnAppReport(application.getJobName()).isEmpty();
+      return exists ? AppExistsStateEnum.IN_YARN : AppExistsStateEnum.NO;
+    }
+    // todo on k8s check...
+    return AppExistsStateEnum.NO;
+  }
+
+  @Override
+  public List<ApplicationReport> getYarnAppReport(String appName) {
+    try {
+      YarnClient yarnClient = HadoopUtils.yarnClient();
+      Set<String> types =
+          Sets.newHashSet(
+              ApplicationType.STREAMPARK_SPARK.getName(), ApplicationType.APACHE_SPARK.getName());
+      EnumSet<YarnApplicationState> states =
+          EnumSet.of(
+              YarnApplicationState.NEW,
+              YarnApplicationState.NEW_SAVING,
+              YarnApplicationState.SUBMITTED,
+              YarnApplicationState.ACCEPTED,
+              YarnApplicationState.RUNNING);
+      Set<String> yarnTag = Sets.newHashSet("streampark");
+      List<ApplicationReport> applications = yarnClient.getApplications(types, states, yarnTag);
+      return applications.stream()
+          .filter(report -> report.getName().equals(appName))
+          .collect(Collectors.toList());
+    } catch (Exception e) {
+      throw new RuntimeException(
+          "getYarnAppReport failed. Ensure that yarn is running properly. ", e);
+    }
+  }
+
+  @Override
+  public String getYarnName(String appConfig) {
+    String[] args = new String[2];
+    args[0] = "--name";
+    args[1] = appConfig;
+    return ParameterCli.read(args);
+  }
+
+  /**
+   * Check if the current jobName and other key identifiers already exist in the database and
+   * yarn/k8s.
+   *
+   * @param appParam The application to check for existence.
+   * @return The state of the application's existence.
+   */
+  @Override
+  public AppExistsStateEnum checkExists(SparkApplication appParam) {
+
+    if (!checkJobName(appParam.getJobName())) {
+      return AppExistsStateEnum.INVALID;
+    }
+
+    boolean existsByJobName = this.existsByJobName(appParam.getJobName());
+
+    if (appParam.getId() != null) {
+      SparkApplication app = getById(appParam.getId());
+      if (app.getJobName().equals(appParam.getJobName())) {
+        return AppExistsStateEnum.NO;
+      }
+
+      if (existsByJobName) {
+        return AppExistsStateEnum.IN_DB;
+      }
+
+      // has stopped status
+      if (FlinkAppStateEnum.isEndState(app.getState())) {
+        // check whether jobName exists on yarn
+        if (SparkExecutionMode.isYarnMode(appParam.getExecutionMode())
+            && YarnUtils.isContains(appParam.getJobName())) {
+          return AppExistsStateEnum.IN_YARN;
+        }
+      }
+    } else {
+      if (existsByJobName) {
+        return AppExistsStateEnum.IN_DB;
+      }
+
+      // check whether jobName exists on yarn
+      if (SparkExecutionMode.isYarnMode(appParam.getExecutionMode())
+          && YarnUtils.isContains(appParam.getJobName())) {
+        return AppExistsStateEnum.IN_YARN;
+      }
+    }
+    return AppExistsStateEnum.NO;
+  }
+
+  private boolean existsByJobName(String jobName) {
+    return baseMapper.exists(
+        new LambdaQueryWrapper<SparkApplication>().eq(SparkApplication::getJobName, jobName));
+  }
+
+  @Override
+  public String readConf(String appConfig) throws IOException {
+    File file = new File(appConfig);
+    String conf = org.apache.streampark.common.util.FileUtils.readFile(file);
+    return Base64.getEncoder().encodeToString(conf.getBytes());
+  }
+
+  @Override
+  public String getMain(SparkApplication appParam) {
+    File jarFile = null;
+    if (appParam.getProjectId() == null) {
+      jarFile = new File(appParam.getJar());
+    }
+    return Utils.getJarManClass(jarFile);
+  }
+
+  @Override
+  public String checkSavepointPath(SparkApplication appParam) throws Exception {
+    String savepointPath = appParam.getSavePoint();
+    if (StringUtils.isBlank(savepointPath)) {
+      //      savepointPath = savePointService.getSavePointPath(appParam);
+    }
+
+    if (StringUtils.isNotBlank(savepointPath)) {
+      final URI uri = URI.create(savepointPath);
+      final String scheme = uri.getScheme();
+      final String pathPart = uri.getPath();
+      String error = null;
+      if (scheme == null) {
+        error =
+            "This state.savepoints.dir value "
+                + savepointPath
+                + " scheme (hdfs://, file://, etc) of  is null. Please specify the file system scheme explicitly in the URI.";
+      } else if (pathPart == null) {
+        error =
+            "This state.savepoints.dir value "
+                + savepointPath
+                + " path part to store the checkpoint data in is null. Please specify a directory path for the checkpoint data.";
+      } else if (pathPart.isEmpty() || "/".equals(pathPart)) {
+        error =
+            "This state.savepoints.dir value "
+                + savepointPath
+                + " Cannot use the root directory for checkpoints.";
+      }
+      return error;
+    } else {
+      return "When custom savepoint is not set, state.savepoints.dir needs to be set in properties or flink-conf.yaml of application";
+    }
+  }
+
+  private Boolean checkJobName(String jobName) {
+    if (!StringUtils.isBlank(jobName.trim())) {
+      return JOB_NAME_PATTERN.matcher(jobName).matches()
+          && SINGLE_SPACE_PATTERN.matcher(jobName).matches();
+    }
+    return false;
+  }
+}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/impl/SparkApplicationManageServiceImpl.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/impl/SparkApplicationManageServiceImpl.java
new file mode 100644
index 0000000..c7142aa
--- /dev/null
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/impl/SparkApplicationManageServiceImpl.java
@@ -0,0 +1,727 @@
+/*
+ * 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.streampark.console.core.service.application.impl;
+
+import org.apache.streampark.common.conf.Workspace;
+import org.apache.streampark.common.enums.SparkExecutionMode;
+import org.apache.streampark.common.enums.StorageType;
+import org.apache.streampark.common.fs.HdfsOperator;
+import org.apache.streampark.common.util.DeflaterUtils;
+import org.apache.streampark.console.base.domain.RestRequest;
+import org.apache.streampark.console.base.exception.ApiAlertException;
+import org.apache.streampark.console.base.mybatis.pager.MybatisPager;
+import org.apache.streampark.console.base.util.WebUtils;
+import org.apache.streampark.console.core.bean.AppControl;
+import org.apache.streampark.console.core.entity.ApplicationConfig;
+import org.apache.streampark.console.core.entity.FlinkSql;
+import org.apache.streampark.console.core.entity.Resource;
+import org.apache.streampark.console.core.entity.SparkApplication;
+import org.apache.streampark.console.core.enums.CandidateTypeEnum;
+import org.apache.streampark.console.core.enums.ChangeTypeEnum;
+import org.apache.streampark.console.core.enums.FlinkAppStateEnum;
+import org.apache.streampark.console.core.enums.OptionStateEnum;
+import org.apache.streampark.console.core.enums.ReleaseStateEnum;
+import org.apache.streampark.console.core.mapper.SparkApplicationMapper;
+import org.apache.streampark.console.core.service.AppBuildPipeService;
+import org.apache.streampark.console.core.service.ApplicationBackUpService;
+import org.apache.streampark.console.core.service.ApplicationConfigService;
+import org.apache.streampark.console.core.service.ApplicationLogService;
+import org.apache.streampark.console.core.service.CommonService;
+import org.apache.streampark.console.core.service.EffectiveService;
+import org.apache.streampark.console.core.service.FlinkSqlService;
+import org.apache.streampark.console.core.service.ProjectService;
+import org.apache.streampark.console.core.service.ResourceService;
+import org.apache.streampark.console.core.service.SettingService;
+import org.apache.streampark.console.core.service.YarnQueueService;
+import org.apache.streampark.console.core.service.application.SparkApplicationManageService;
+import org.apache.streampark.flink.packer.pipeline.PipelineStatusEnum;
+
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.commons.lang3.StringUtils;
+
+import com.baomidou.mybatisplus.core.conditions.query.LambdaQueryWrapper;
+import com.baomidou.mybatisplus.core.conditions.update.LambdaUpdateWrapper;
+import com.baomidou.mybatisplus.core.metadata.IPage;
+import com.baomidou.mybatisplus.core.toolkit.Wrappers;
+import com.baomidou.mybatisplus.core.toolkit.support.SFunction;
+import com.baomidou.mybatisplus.extension.plugins.pagination.Page;
+import com.baomidou.mybatisplus.extension.service.impl.ServiceImpl;
+import com.google.common.annotations.VisibleForTesting;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Service;
+import org.springframework.transaction.annotation.Propagation;
+import org.springframework.transaction.annotation.Transactional;
+
+import javax.annotation.PostConstruct;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+@Slf4j
+@Service
+@Transactional(propagation = Propagation.SUPPORTS, readOnly = true, rollbackFor = Exception.class)
+public class SparkApplicationManageServiceImpl
+    extends ServiceImpl<SparkApplicationMapper, SparkApplication>
+    implements SparkApplicationManageService {
+
+  private static final String ERROR_APP_QUEUE_HINT =
+      "Queue label '%s' isn't available for teamId '%d', please add it into the team first.";
+
+  @Autowired private ProjectService projectService;
+
+  @Autowired private ApplicationBackUpService backUpService;
+
+  @Autowired private ApplicationConfigService configService;
+
+  @Autowired private ApplicationLogService applicationLogService;
+
+  @Autowired private FlinkSqlService flinkSqlService;
+
+  @Autowired private EffectiveService effectiveService;
+
+  @Autowired private SettingService settingService;
+
+  @Autowired private CommonService commonService;
+
+  @Autowired private AppBuildPipeService appBuildPipeService;
+
+  @Autowired private YarnQueueService yarnQueueService;
+
+  @Autowired private ResourceService resourceService;
+
+  @PostConstruct
+  public void resetOptionState() {
+    this.baseMapper.resetOptionState();
+  }
+
+  @Override
+  public void toEffective(SparkApplication appParam) {
+    // set latest to Effective
+    ApplicationConfig config = configService.getLatest(appParam.getId());
+    if (config != null) {
+      this.configService.toEffective(appParam.getId(), config.getId());
+    }
+    if (appParam.isSparkSqlJob()) {
+      FlinkSql flinkSql = flinkSqlService.getCandidate(appParam.getId(), null);
+      if (flinkSql != null) {
+        flinkSqlService.toEffective(appParam.getId(), flinkSql.getId());
+        // clean candidate
+        flinkSqlService.cleanCandidate(flinkSql.getId());
+      }
+    }
+  }
+
+  @Override
+  public void persistMetrics(SparkApplication appParam) {
+    this.baseMapper.persistMetrics(appParam);
+  }
+
+  @Override
+  public boolean mapping(SparkApplication appParam) {
+    boolean mapping = this.baseMapper.mapping(appParam);
+    SparkApplication application = getById(appParam.getId());
+    return mapping;
+  }
+
+  @Override
+  public Boolean remove(Long appId) {
+
+    SparkApplication application = getById(appId);
+
+    // 1) remove flink sql
+    flinkSqlService.removeByAppId(application.getId());
+
+    // 2) remove log
+    applicationLogService.removeByAppId(application.getId());
+
+    // 3) remove config
+    configService.removeByAppId(application.getId());
+
+    // 4) remove effective
+    effectiveService.removeByAppId(application.getId());
+
+    // remove related hdfs
+    // 5) remove backup
+    //    backUpService.remove(application);
+
+    // 6) remove savepoint
+    //    savePointService.remove(application);
+
+    // 7) remove BuildPipeline
+    appBuildPipeService.removeByAppId(application.getId());
+
+    // 8) remove app
+    removeApp(application);
+    return true;
+  }
+
+  private void removeApp(SparkApplication application) {
+    Long appId = application.getId();
+    removeById(appId);
+    try {
+      application
+          .getFsOperator()
+          .delete(application.getWorkspace().APP_WORKSPACE().concat("/").concat(appId.toString()));
+      // try to delete yarn-application, and leave no trouble.
+      String path =
+          Workspace.of(StorageType.HDFS).APP_WORKSPACE().concat("/").concat(appId.toString());
+      if (HdfsOperator.exists(path)) {
+        HdfsOperator.delete(path);
+      }
+    } catch (Exception e) {
+      // skip
+    }
+  }
+
+  @Override
+  public IPage<SparkApplication> page(SparkApplication appParam, RestRequest request) {
+    if (appParam.getTeamId() == null) {
+      return null;
+    }
+    Page<SparkApplication> page = MybatisPager.getPage(request);
+
+    if (ArrayUtils.isNotEmpty(appParam.getStateArray())
+        && Arrays.stream(appParam.getStateArray())
+            .anyMatch(x -> x == FlinkAppStateEnum.FINISHED.getValue())) {
+      Integer[] newArray =
+          ArrayUtils.insert(
+              appParam.getStateArray().length,
+              appParam.getStateArray(),
+              FlinkAppStateEnum.POS_TERMINATED.getValue());
+      appParam.setStateArray(newArray);
+    }
+    this.baseMapper.selectPage(page, appParam);
+    List<SparkApplication> records = page.getRecords();
+    long now = System.currentTimeMillis();
+
+    List<Long> appIds = records.stream().map(SparkApplication::getId).collect(Collectors.toList());
+    Map<Long, PipelineStatusEnum> pipeStates =
+        appBuildPipeService.listAppIdPipelineStatusMap(appIds);
+
+    List<SparkApplication> newRecords =
+        records.stream()
+            .peek(
+                record -> {
+                  if (pipeStates.containsKey(record.getId())) {
+                    record.setBuildStatus(pipeStates.get(record.getId()).getCode());
+                  }
+
+                  AppControl appControl =
+                      new AppControl()
+                          .setAllowBuild(
+                              record.getBuildStatus() == null
+                                  || !PipelineStatusEnum.running
+                                      .getCode()
+                                      .equals(record.getBuildStatus()))
+                          .setAllowStart(
+                              !record.shouldTracking()
+                                  && PipelineStatusEnum.success
+                                      .getCode()
+                                      .equals(record.getBuildStatus()))
+                          .setAllowStop(record.isRunning());
+                  record.setAppControl(appControl);
+                })
+            .collect(Collectors.toList());
+    page.setRecords(newRecords);
+    return page;
+  }
+
+  @Override
+  public void changeOwnership(Long userId, Long targetUserId) {
+    LambdaUpdateWrapper<SparkApplication> updateWrapper =
+        new LambdaUpdateWrapper<SparkApplication>()
+            .eq(SparkApplication::getUserId, userId)
+            .set(SparkApplication::getUserId, targetUserId);
+    this.baseMapper.update(null, updateWrapper);
+  }
+
+  @SneakyThrows
+  @Override
+  public boolean create(SparkApplication appParam) {
+    ApiAlertException.throwIfNull(
+        appParam.getTeamId(), "The teamId can't be null. Create application failed.");
+    appParam.setUserId(commonService.getUserId());
+    appParam.setState(FlinkAppStateEnum.ADDED.getValue());
+    appParam.setRelease(ReleaseStateEnum.NEED_RELEASE.get());
+    appParam.setOptionState(OptionStateEnum.NONE.getValue());
+    appParam.setCreateTime(new Date());
+    appParam.setModifyTime(new Date());
+    appParam.setDefaultModeIngress(settingService.getIngressModeDefault());
+
+    boolean success = validateQueueIfNeeded(appParam);
+    ApiAlertException.throwIfFalse(
+        success,
+        String.format(ERROR_APP_QUEUE_HINT, appParam.getYarnQueue(), appParam.getTeamId()));
+
+    appParam.doSetHotParams();
+    if (appParam.isUploadJob()) {
+      String jarPath =
+          String.format(
+              "%s/%d/%s", Workspace.local().APP_UPLOADS(), appParam.getTeamId(), appParam.getJar());
+      if (!new File(jarPath).exists()) {
+        Resource resource =
+            resourceService.findByResourceName(appParam.getTeamId(), appParam.getJar());
+        if (resource != null && StringUtils.isNotBlank(resource.getFilePath())) {
+          jarPath = resource.getFilePath();
+        }
+      }
+      appParam.setJarCheckSum(org.apache.commons.io.FileUtils.checksumCRC32(new File(jarPath)));
+    }
+
+    if (save(appParam)) {
+      if (appParam.isSparkSqlJob()) {
+        FlinkSql flinkSql = new FlinkSql(appParam);
+        flinkSqlService.create(flinkSql);
+      }
+      if (appParam.getConfig() != null) {
+        //        configService.create(appParam, true);
+      }
+      return true;
+    } else {
+      throw new ApiAlertException("create application failed");
+    }
+  }
+
+  private boolean existsByJobName(String jobName) {
+    return baseMapper.exists(
+        new LambdaQueryWrapper<SparkApplication>().eq(SparkApplication::getJobName, jobName));
+  }
+
+  @SuppressWarnings("checkstyle:WhitespaceAround")
+  @Override
+  @SneakyThrows
+  public Long copy(SparkApplication appParam) {
+    boolean existsByJobName = this.existsByJobName(appParam.getJobName());
+    ApiAlertException.throwIfFalse(
+        !existsByJobName,
+        "[StreamPark] Application names can't be repeated, copy application failed.");
+
+    SparkApplication oldApp = getById(appParam.getId());
+    SparkApplication newApp = new SparkApplication();
+    String jobName = appParam.getJobName();
+
+    newApp.setJobName(jobName);
+    newApp.setClusterId(jobName);
+    newApp.setArgs(appParam.getArgs() != null ? appParam.getArgs() : oldApp.getArgs());
+    newApp.setVersionId(oldApp.getVersionId());
+
+    newApp.setSparkClusterId(oldApp.getSparkClusterId());
+    newApp.setRestartSize(oldApp.getRestartSize());
+    newApp.setJobType(oldApp.getJobType());
+    newApp.setOptions(oldApp.getOptions());
+    newApp.setDynamicProperties(oldApp.getDynamicProperties());
+    newApp.setResolveOrder(oldApp.getResolveOrder());
+    newApp.setExecutionMode(oldApp.getExecutionMode());
+    newApp.setSparkImage(oldApp.getSparkImage());
+    newApp.setK8sNamespace(oldApp.getK8sNamespace());
+    newApp.setK8sRestExposedType(oldApp.getK8sRestExposedType());
+    newApp.setK8sPodTemplate(oldApp.getK8sPodTemplate());
+    newApp.setK8sJmPodTemplate(oldApp.getK8sJmPodTemplate());
+    newApp.setK8sTmPodTemplate(oldApp.getK8sTmPodTemplate());
+    newApp.setK8sHadoopIntegration(oldApp.getK8sHadoopIntegration());
+    newApp.setDescription(oldApp.getDescription());
+    newApp.setAlertId(oldApp.getAlertId());
+    newApp.setCpFailureAction(oldApp.getCpFailureAction());
+    newApp.setCpFailureRateInterval(oldApp.getCpFailureRateInterval());
+    newApp.setCpMaxFailureInterval(oldApp.getCpMaxFailureInterval());
+    newApp.setMainClass(oldApp.getMainClass());
+    newApp.setAppType(oldApp.getAppType());
+    newApp.setResourceFrom(oldApp.getResourceFrom());
+    newApp.setProjectId(oldApp.getProjectId());
+    newApp.setModule(oldApp.getModule());
+    newApp.setUserId(commonService.getUserId());
+    newApp.setState(FlinkAppStateEnum.ADDED.getValue());
+    newApp.setRelease(ReleaseStateEnum.NEED_RELEASE.get());
+    newApp.setOptionState(OptionStateEnum.NONE.getValue());
+    newApp.setCreateTime(new Date());
+    newApp.setModifyTime(new Date());
+    newApp.setHotParams(oldApp.getHotParams());
+
+    newApp.setJar(oldApp.getJar());
+    newApp.setJarCheckSum(oldApp.getJarCheckSum());
+    newApp.setTags(oldApp.getTags());
+    newApp.setTeamId(oldApp.getTeamId());
+    newApp.setHadoopUser(oldApp.getHadoopUser());
+
+    boolean saved = save(newApp);
+    if (saved) {
+      if (newApp.isSparkSqlJob()) {
+        FlinkSql copyFlinkSql = flinkSqlService.getLatestFlinkSql(appParam.getId(), true);
+        newApp.setSparkSql(copyFlinkSql.getSql());
+        newApp.setTeamResource(copyFlinkSql.getTeamResource());
+        newApp.setDependency(copyFlinkSql.getDependency());
+        FlinkSql flinkSql = new FlinkSql(newApp);
+        flinkSqlService.create(flinkSql);
+      }
+      ApplicationConfig copyConfig = configService.getEffective(appParam.getId());
+      if (copyConfig != null) {
+        ApplicationConfig config = new ApplicationConfig();
+        config.setAppId(newApp.getId());
+        config.setFormat(copyConfig.getFormat());
+        config.setContent(copyConfig.getContent());
+        config.setCreateTime(new Date());
+        config.setVersion(1);
+        configService.save(config);
+        configService.setLatestOrEffective(true, config.getId(), newApp.getId());
+      }
+      return newApp.getId();
+    } else {
+      throw new ApiAlertException(
+          "create application from copy failed, copy source app: " + oldApp.getJobName());
+    }
+  }
+
+  @Override
+  public boolean update(SparkApplication appParam) {
+    SparkApplication application = getById(appParam.getId());
+
+    /* If the original mode is remote, k8s-session, yarn-session, check cluster status */
+    SparkExecutionMode sparkExecutionMode = application.getSparkExecutionMode();
+
+    boolean success = validateQueueIfNeeded(application, appParam);
+    ApiAlertException.throwIfFalse(
+        success,
+        String.format(ERROR_APP_QUEUE_HINT, appParam.getYarnQueue(), appParam.getTeamId()));
+
+    application.setRelease(ReleaseStateEnum.NEED_RELEASE.get());
+
+    // 1) jar job jar file changed
+    if (application.isUploadJob()) {
+      if (!Objects.equals(application.getJar(), appParam.getJar())) {
+        application.setBuild(true);
+      } else {
+        File jarFile = new File(WebUtils.getAppTempDir(), appParam.getJar());
+        if (jarFile.exists()) {
+          try {
+            long checkSum = org.apache.commons.io.FileUtils.checksumCRC32(jarFile);
+            if (!Objects.equals(checkSum, application.getJarCheckSum())) {
+              application.setBuild(true);
+            }
+          } catch (IOException e) {
+            log.error("Error in checksumCRC32 for {}.", jarFile);
+            throw new RuntimeException(e);
+          }
+        }
+      }
+    }
+
+    // 3) flink version changed
+    if (!application.getBuild()
+        && !Objects.equals(application.getVersionId(), appParam.getVersionId())) {
+      application.setBuild(true);
+    }
+
+    // 4) yarn application mode change
+    if (!application.getBuild() && isYarnApplicationModeChange(application, appParam)) {
+      application.setBuild(true);
+    }
+
+    appParam.setJobType(application.getJobType());
+    // changes to the following parameters need to be re-release to take effect
+    application.setJobName(appParam.getJobName());
+    application.setVersionId(appParam.getVersionId());
+    application.setArgs(appParam.getArgs());
+    application.setOptions(appParam.getOptions());
+    application.setDynamicProperties(appParam.getDynamicProperties());
+    application.setResolveOrder(appParam.getResolveOrder());
+    application.setExecutionMode(appParam.getExecutionMode());
+    application.setClusterId(appParam.getClusterId());
+    application.setSparkImage(appParam.getSparkImage());
+    application.setK8sNamespace(appParam.getK8sNamespace());
+    application.updateHotParams(appParam);
+    application.setK8sRestExposedType(appParam.getK8sRestExposedType());
+    application.setK8sPodTemplate(appParam.getK8sPodTemplate());
+    application.setK8sJmPodTemplate(appParam.getK8sJmPodTemplate());
+    application.setK8sTmPodTemplate(appParam.getK8sTmPodTemplate());
+    application.setK8sHadoopIntegration(appParam.getK8sHadoopIntegration());
+
+    // changes to the following parameters do not affect running tasks
+    application.setModifyTime(new Date());
+    application.setDescription(appParam.getDescription());
+    application.setAlertId(appParam.getAlertId());
+    application.setRestartSize(appParam.getRestartSize());
+    application.setCpFailureAction(appParam.getCpFailureAction());
+    application.setCpFailureRateInterval(appParam.getCpFailureRateInterval());
+    application.setCpMaxFailureInterval(appParam.getCpMaxFailureInterval());
+    application.setTags(appParam.getTags());
+
+    switch (appParam.getSparkExecutionMode()) {
+      case YARN_CLUSTER:
+      case YARN_CLIENT:
+        application.setHadoopUser(appParam.getHadoopUser());
+        break;
+      case REMOTE:
+        application.setSparkClusterId(appParam.getSparkClusterId());
+        break;
+      default:
+        break;
+    }
+
+    // Flink Sql job...
+    if (application.isSparkSqlJob()) {
+      updateFlinkSqlJob(application, appParam);
+      return true;
+    }
+
+    if (application.isStreamParkJob()) {
+      //      configService.update(appParam, application.isRunning());
+    } else {
+      application.setJar(appParam.getJar());
+      application.setMainClass(appParam.getMainClass());
+    }
+    this.updateById(application);
+    return true;
+  }
+
+  /**
+   * update FlinkSql type jobs, there are 3 aspects to consider<br>
+   * 1. flink sql has changed <br>
+   * 2. dependency has changed<br>
+   * 3. parameter has changed<br>
+   *
+   * @param application
+   * @param appParam
+   */
+  private void updateFlinkSqlJob(SparkApplication application, SparkApplication appParam) {
+    FlinkSql effectiveFlinkSql = flinkSqlService.getEffective(application.getId(), true);
+    if (effectiveFlinkSql == null) {
+      effectiveFlinkSql = flinkSqlService.getCandidate(application.getId(), CandidateTypeEnum.NEW);
+      flinkSqlService.removeById(effectiveFlinkSql.getId());
+      FlinkSql sql = new FlinkSql(appParam);
+      flinkSqlService.create(sql);
+      application.setBuild(true);
+    } else {
+      // get previous flink sql and decode
+      FlinkSql copySourceFlinkSql = flinkSqlService.getById(appParam.getSqlId());
+      ApiAlertException.throwIfNull(
+          copySourceFlinkSql, "Flink sql is null, update flink sql job failed.");
+      copySourceFlinkSql.decode();
+
+      // get submit flink sql
+      FlinkSql targetFlinkSql = new FlinkSql(appParam);
+
+      // judge sql and dependency has changed
+      ChangeTypeEnum changeTypeEnum = copySourceFlinkSql.checkChange(targetFlinkSql);
+
+      log.info("updateFlinkSqlJob changeTypeEnum: {}", changeTypeEnum);
+
+      // if has been changed
+      if (changeTypeEnum.hasChanged()) {
+        // check if there is a candidate version for the newly added record
+        FlinkSql newFlinkSql =
+            flinkSqlService.getCandidate(application.getId(), CandidateTypeEnum.NEW);
+        // If the candidate version of the new record exists, it will be deleted directly,
+        // and only one candidate version will be retained. If the new candidate version is not
+        // effective,
+        // if it is edited again and the next record comes in, the previous candidate version will
+        // be deleted.
+        if (newFlinkSql != null) {
+          // delete all records about candidates
+          flinkSqlService.removeById(newFlinkSql.getId());
+        }
+        FlinkSql historyFlinkSql =
+            flinkSqlService.getCandidate(application.getId(), CandidateTypeEnum.HISTORY);
+        // remove candidate flags that already exist but are set as candidates
+        if (historyFlinkSql != null) {
+          flinkSqlService.cleanCandidate(historyFlinkSql.getId());
+        }
+        FlinkSql sql = new FlinkSql(appParam);
+        flinkSqlService.create(sql);
+        if (changeTypeEnum.isDependencyChanged()) {
+          application.setBuild(true);
+        }
+      } else {
+        // judge version has changed
+        boolean versionChanged = !effectiveFlinkSql.getId().equals(appParam.getSqlId());
+        if (versionChanged) {
+          // sql and dependency not changed, but version changed, means that rollback to the version
+          CandidateTypeEnum type = CandidateTypeEnum.HISTORY;
+          flinkSqlService.setCandidate(type, appParam.getId(), appParam.getSqlId());
+          application.setRelease(ReleaseStateEnum.NEED_ROLLBACK.get());
+          application.setBuild(true);
+        }
+      }
+    }
+    this.updateById(application);
+    //    this.configService.update(appParam, application.isRunning());
+  }
+
+  @Override
+  public void updateRelease(SparkApplication appParam) {
+    LambdaUpdateWrapper<SparkApplication> updateWrapper = Wrappers.lambdaUpdate();
+    updateWrapper.eq(SparkApplication::getId, appParam.getId());
+    updateWrapper.set(SparkApplication::getRelease, appParam.getRelease());
+    updateWrapper.set(SparkApplication::getBuild, appParam.getBuild());
+    if (appParam.getOptionState() != null) {
+      updateWrapper.set(SparkApplication::getOptionState, appParam.getOptionState());
+    }
+    this.update(updateWrapper);
+  }
+
+  @Override
+  public List<SparkApplication> listByProjectId(Long id) {
+    return baseMapper.selectAppsByProjectId(id);
+  }
+
+  @Override
+  public List<SparkApplication> listByTeamId(Long teamId) {
+    return baseMapper.selectAppsByTeamId(teamId);
+  }
+
+  @Override
+  public List<SparkApplication> listByTeamIdAndExecutionModes(
+      Long teamId, Collection<SparkExecutionMode> executionModeEnums) {
+    return getBaseMapper()
+        .selectList(
+            new LambdaQueryWrapper<SparkApplication>()
+                .eq((SFunction<SparkApplication, Long>) SparkApplication::getTeamId, teamId)
+                .in(
+                    SparkApplication::getExecutionMode,
+                    executionModeEnums.stream()
+                        .map(SparkExecutionMode::getMode)
+                        .collect(Collectors.toSet())));
+  }
+
+  public List<SparkApplication> listProbeApps() {
+    return this.baseMapper.selectProbeApps();
+  }
+
+  @Override
+  public boolean checkBuildAndUpdate(SparkApplication appParam) {
+    boolean build = appParam.getBuild();
+    if (!build) {
+      LambdaUpdateWrapper<SparkApplication> updateWrapper = Wrappers.lambdaUpdate();
+      updateWrapper.eq(SparkApplication::getId, appParam.getId());
+      if (appParam.isRunning()) {
+        updateWrapper.set(SparkApplication::getRelease, ReleaseStateEnum.NEED_RESTART.get());
+      } else {
+        updateWrapper.set(SparkApplication::getRelease, ReleaseStateEnum.DONE.get());
+        updateWrapper.set(SparkApplication::getOptionState, OptionStateEnum.NONE.getValue());
+      }
+      this.update(updateWrapper);
+
+      // If the current task is not running, or the task has just been added,
+      // directly set the candidate version to the official version
+      FlinkSql flinkSql = flinkSqlService.getEffective(appParam.getId(), false);
+      if (!appParam.isRunning() || flinkSql == null) {
+        this.toEffective(appParam);
+      }
+    }
+    return build;
+  }
+
+  @Override
+  public void clean(SparkApplication appParam) {
+    appParam.setRelease(ReleaseStateEnum.DONE.get());
+    this.updateRelease(appParam);
+  }
+
+  @Override
+  public SparkApplication getApp(Long id) {
+    SparkApplication application = this.baseMapper.selectApp(id);
+    ApplicationConfig config = configService.getEffective(id);
+    config = config == null ? configService.getLatest(id) : config;
+    if (config != null) {
+      config.setToApplication(application);
+    }
+    if (application.isSparkSqlJob()) {
+      FlinkSql flinkSql = flinkSqlService.getEffective(application.getId(), true);
+      if (flinkSql == null) {
+        flinkSql = flinkSqlService.getCandidate(application.getId(), CandidateTypeEnum.NEW);
+        flinkSql.setSql(DeflaterUtils.unzipString(flinkSql.getSql()));
+      }
+      flinkSql.setToApplication(application);
+    } else {
+      if (application.isCICDJob()) {
+        String path =
+            this.projectService.getAppConfPath(application.getProjectId(), application.getModule());
+        application.setConfPath(path);
+      }
+    }
+
+    application.setYarnQueueByHotParams();
+
+    return application;
+  }
+
+  /**
+   * Check queue label validation when create the application if needed.
+   *
+   * @param appParam the app to create.
+   * @return <code>true</code> if validate it successfully, <code>false</code> else.
+   */
+  @VisibleForTesting
+  public boolean validateQueueIfNeeded(SparkApplication appParam) {
+    yarnQueueService.checkQueueLabel(appParam.getSparkExecutionMode(), appParam.getYarnQueue());
+    if (!isYarnNotDefaultQueue(appParam)) {
+      return true;
+    }
+    return yarnQueueService.existByTeamIdQueueLabel(appParam.getTeamId(), appParam.getYarnQueue());
+  }
+
+  /**
+   * Check queue label validation when update the application if needed.
+   *
+   * @param oldApp the old app to update.
+   * @param newApp the new app payload.
+   * @return <code>true</code> if validate it successfully, <code>false</code> else.
+   */
+  @VisibleForTesting
+  public boolean validateQueueIfNeeded(SparkApplication oldApp, SparkApplication newApp) {
+    yarnQueueService.checkQueueLabel(newApp.getSparkExecutionMode(), newApp.getYarnQueue());
+    if (!isYarnNotDefaultQueue(newApp)) {
+      return true;
+    }
+
+    oldApp.setYarnQueueByHotParams();
+    if (SparkExecutionMode.isYarnMode(newApp.getSparkExecutionMode())
+        && StringUtils.equals(oldApp.getYarnQueue(), newApp.getYarnQueue())) {
+      return true;
+    }
+    return yarnQueueService.existByTeamIdQueueLabel(newApp.getTeamId(), newApp.getYarnQueue());
+  }
+
+  /**
+   * Judge the execution mode whether is the Yarn PerJob or Application mode with not default or
+   * empty queue label.
+   *
+   * @param application application entity.
+   * @return If the executionMode is (Yarn PerJob or application mode) and the queue label is not
+   *     (empty or default), return true, false else.
+   */
+  private boolean isYarnNotDefaultQueue(SparkApplication application) {
+    return SparkExecutionMode.isYarnMode(application.getSparkExecutionMode())
+        && !yarnQueueService.isDefaultQueue(application.getYarnQueue());
+  }
+
+  private boolean isYarnApplicationModeChange(
+      SparkApplication application, SparkApplication appParam) {
+    return !application.getExecutionMode().equals(appParam.getExecutionMode())
+        && (SparkExecutionMode.YARN_CLIENT == appParam.getSparkExecutionMode()
+            || SparkExecutionMode.YARN_CLUSTER == application.getSparkExecutionMode());
+  }
+}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/CommonServiceImpl.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/CommonServiceImpl.java
index 0ef911c..b4e6e50 100644
--- a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/CommonServiceImpl.java
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/CommonServiceImpl.java
@@ -20,6 +20,7 @@
 import org.apache.streampark.console.base.exception.ApiAlertException;
 import org.apache.streampark.console.base.util.WebUtils;
 import org.apache.streampark.console.core.entity.FlinkEnv;
+import org.apache.streampark.console.core.entity.SparkEnv;
 import org.apache.streampark.console.core.service.CommonService;
 import org.apache.streampark.console.system.authentication.JWTUtil;
 import org.apache.streampark.console.system.entity.User;
@@ -42,6 +43,8 @@
 
   private String sqlClientJar = null;
 
+  private String sparkSqlClientJar = null;
+
   @Autowired private UserService userService;
 
   @Override
@@ -82,4 +85,31 @@
     }
     return sqlClientJar;
   }
+
+  @Override
+  public String getSqlClientJar(SparkEnv sparkEnv) {
+    if (sparkSqlClientJar == null) {
+      File localClient = WebUtils.getAppClientDir();
+      ApiAlertException.throwIfFalse(
+          localClient.exists(), "[StreamPark] " + localClient + " no exists. please check.");
+      List<String> jars =
+          Arrays.stream(Objects.requireNonNull(localClient.list()))
+              .filter(
+                  x ->
+                      x.matches(
+                          "streampark-spark-sqlclient_" + sparkEnv.getScalaVersion() + "-.*\\.jar"))
+              .collect(Collectors.toList());
+
+      ApiAlertException.throwIfTrue(
+          jars.isEmpty(),
+          "[StreamPark] can't found streampark-flink-sqlclient jar in " + localClient);
+
+      ApiAlertException.throwIfTrue(
+          jars.size() > 1,
+          "[StreamPark] found multiple streampark-flink-sqlclient jar in " + localClient);
+
+      sparkSqlClientJar = jars.get(0);
+    }
+    return sparkSqlClientJar;
+  }
 }
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/SparkAppBuildPipeServiceImpl.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/SparkAppBuildPipeServiceImpl.java
new file mode 100644
index 0000000..ff0a529
--- /dev/null
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/SparkAppBuildPipeServiceImpl.java
@@ -0,0 +1,567 @@
+/*
+ * 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.streampark.console.core.service.impl;
+
+import org.apache.streampark.common.Constant;
+import org.apache.streampark.common.conf.Workspace;
+import org.apache.streampark.common.enums.ApplicationType;
+import org.apache.streampark.common.enums.FlinkDevelopmentMode;
+import org.apache.streampark.common.enums.SparkExecutionMode;
+import org.apache.streampark.common.fs.FsOperator;
+import org.apache.streampark.common.util.AssertUtils;
+import org.apache.streampark.common.util.ExceptionUtils;
+import org.apache.streampark.common.util.FileUtils;
+import org.apache.streampark.console.base.exception.ApiAlertException;
+import org.apache.streampark.console.base.util.JacksonUtils;
+import org.apache.streampark.console.base.util.WebUtils;
+import org.apache.streampark.console.core.bean.Dependency;
+import org.apache.streampark.console.core.entity.AppBuildPipeline;
+import org.apache.streampark.console.core.entity.ApplicationConfig;
+import org.apache.streampark.console.core.entity.ApplicationLog;
+import org.apache.streampark.console.core.entity.FlinkSql;
+import org.apache.streampark.console.core.entity.Message;
+import org.apache.streampark.console.core.entity.Resource;
+import org.apache.streampark.console.core.entity.SparkApplication;
+import org.apache.streampark.console.core.entity.SparkEnv;
+import org.apache.streampark.console.core.enums.CandidateTypeEnum;
+import org.apache.streampark.console.core.enums.NoticeTypeEnum;
+import org.apache.streampark.console.core.enums.OptionStateEnum;
+import org.apache.streampark.console.core.enums.ReleaseStateEnum;
+import org.apache.streampark.console.core.enums.ResourceTypeEnum;
+import org.apache.streampark.console.core.mapper.ApplicationBuildPipelineMapper;
+import org.apache.streampark.console.core.service.ApplicationBackUpService;
+import org.apache.streampark.console.core.service.ApplicationConfigService;
+import org.apache.streampark.console.core.service.ApplicationLogService;
+import org.apache.streampark.console.core.service.CommonService;
+import org.apache.streampark.console.core.service.FlinkSqlService;
+import org.apache.streampark.console.core.service.MessageService;
+import org.apache.streampark.console.core.service.ResourceService;
+import org.apache.streampark.console.core.service.SettingService;
+import org.apache.streampark.console.core.service.SparkAppBuildPipeService;
+import org.apache.streampark.console.core.service.SparkEnvService;
+import org.apache.streampark.console.core.service.application.SparkApplicationInfoService;
+import org.apache.streampark.console.core.service.application.SparkApplicationManageService;
+import org.apache.streampark.console.core.watcher.FlinkAppHttpWatcher;
+import org.apache.streampark.flink.packer.maven.Artifact;
+import org.apache.streampark.flink.packer.maven.DependencyInfo;
+import org.apache.streampark.flink.packer.pipeline.BuildPipeline;
+import org.apache.streampark.flink.packer.pipeline.BuildResult;
+import org.apache.streampark.flink.packer.pipeline.PipeSnapshot;
+import org.apache.streampark.flink.packer.pipeline.PipeWatcher;
+import org.apache.streampark.flink.packer.pipeline.PipelineStatusEnum;
+import org.apache.streampark.flink.packer.pipeline.SparkYarnApplicationBuildRequest;
+import org.apache.streampark.flink.packer.pipeline.impl.SparkYarnApplicationBuildPipeline;
+
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.lang3.StringUtils;
+
+import com.baomidou.mybatisplus.core.conditions.query.LambdaQueryWrapper;
+import com.baomidou.mybatisplus.extension.service.impl.ServiceImpl;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import lombok.extern.slf4j.Slf4j;
+import org.jetbrains.annotations.NotNull;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.beans.factory.annotation.Qualifier;
+import org.springframework.stereotype.Service;
+import org.springframework.transaction.annotation.Propagation;
+import org.springframework.transaction.annotation.Transactional;
+
+import javax.annotation.Nonnull;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ExecutorService;
+import java.util.stream.Collectors;
+
+import static org.apache.streampark.console.core.enums.OperationEnum.RELEASE;
+
+@Service
+@Slf4j
+@Transactional(propagation = Propagation.SUPPORTS, rollbackFor = Exception.class)
+public class SparkAppBuildPipeServiceImpl
+    extends ServiceImpl<ApplicationBuildPipelineMapper, AppBuildPipeline>
+    implements SparkAppBuildPipeService {
+
+  @Autowired private SparkEnvService sparkEnvService;
+
+  @Autowired private FlinkSqlService flinkSqlService;
+
+  @Autowired private ApplicationBackUpService backUpService;
+
+  @Autowired private CommonService commonService;
+
+  @Autowired private SettingService settingService;
+
+  @Autowired private MessageService messageService;
+
+  @Autowired private SparkApplicationManageService applicationManageService;
+
+  @Autowired private SparkApplicationInfoService applicationInfoService;
+
+  @Autowired private ApplicationLogService applicationLogService;
+
+  @Autowired private FlinkAppHttpWatcher flinkAppHttpWatcher;
+
+  @Autowired private ApplicationConfigService applicationConfigService;
+
+  @Autowired private ResourceService resourceService;
+
+  @Qualifier("streamparkBuildPipelineExecutor")
+  @Autowired
+  private ExecutorService executorService;
+
+  /**
+   * Build application. This is an async call method.
+   *
+   * @param appId application id
+   * @param forceBuild forced start pipeline or not
+   * @return Whether the pipeline was successfully started
+   */
+  @Override
+  public boolean buildApplication(@NotNull Long appId, boolean forceBuild) {
+    // check the build environment
+    checkBuildEnv(appId, forceBuild);
+
+    SparkApplication app = applicationManageService.getById(appId);
+    ApplicationLog applicationLog = new ApplicationLog();
+    applicationLog.setOptionName(RELEASE.getValue());
+    applicationLog.setAppId(app.getId());
+    applicationLog.setOptionTime(new Date());
+    applicationLog.setUserId(commonService.getUserId());
+
+    // check if you need to go through the build process (if the jar and pom have changed,
+    // you need to go through the build process, if other common parameters are modified,
+    // you don't need to go through the build process)
+    boolean needBuild = applicationManageService.checkBuildAndUpdate(app);
+    if (!needBuild) {
+      applicationLog.setSuccess(true);
+      applicationLogService.save(applicationLog);
+      return true;
+    }
+
+    // 1) spark sql setDependency
+    FlinkSql newFlinkSql = flinkSqlService.getCandidate(app.getId(), CandidateTypeEnum.NEW);
+    FlinkSql effectiveFlinkSql = flinkSqlService.getEffective(app.getId(), false);
+    if (app.isSparkSqlJob()) {
+      FlinkSql flinkSql = newFlinkSql == null ? effectiveFlinkSql : newFlinkSql;
+      AssertUtils.notNull(flinkSql);
+      app.setDependency(flinkSql.getDependency());
+      app.setTeamResource(flinkSql.getTeamResource());
+    }
+
+    // create pipeline instance
+    BuildPipeline pipeline = createPipelineInstance(app);
+
+    // clear history
+    removeByAppId(app.getId());
+    // register pipeline progress event watcher.
+    // save snapshot of pipeline to db when status of pipeline was changed.
+    pipeline.registerWatcher(
+        new PipeWatcher() {
+          @Override
+          public void onStart(PipeSnapshot snapshot) {
+            AppBuildPipeline buildPipeline =
+                AppBuildPipeline.fromPipeSnapshot(snapshot).setAppId(app.getId());
+            saveEntity(buildPipeline);
+
+            app.setRelease(ReleaseStateEnum.RELEASING.get());
+            applicationManageService.updateRelease(app);
+
+            if (flinkAppHttpWatcher.isWatchingApp(app.getId())) {
+              flinkAppHttpWatcher.init();
+            }
+
+            // 1) checkEnv
+            applicationInfoService.checkEnv(app);
+
+            // 2) some preparatory work
+            String appUploads = app.getWorkspace().APP_UPLOADS();
+
+            if (app.isCustomCodeOrPySparkJob()) {
+              // customCode upload jar to appHome...
+              String appHome = app.getAppHome();
+              FsOperator fsOperator = app.getFsOperator();
+              fsOperator.delete(appHome);
+              if (app.isUploadJob()) {
+                String uploadJar = appUploads.concat("/").concat(app.getJar());
+                File localJar =
+                    new File(
+                        String.format(
+                            "%s/%d/%s",
+                            Workspace.local().APP_UPLOADS(), app.getTeamId(), app.getJar()));
+                if (!localJar.exists()) {
+                  Resource resource =
+                      resourceService.findByResourceName(app.getTeamId(), app.getJar());
+                  if (resource != null && StringUtils.isNotBlank(resource.getFilePath())) {
+                    localJar = new File(resource.getFilePath());
+                    uploadJar = appUploads.concat("/").concat(localJar.getName());
+                  }
+                }
+                // upload jar copy to appHome
+                checkOrElseUploadJar(app.getFsOperator(), localJar, uploadJar, appUploads);
+
+                switch (app.getApplicationType()) {
+                  case STREAMPARK_SPARK:
+                    fsOperator.mkdirs(app.getAppLib());
+                    fsOperator.copy(uploadJar, app.getAppLib(), false, true);
+                    break;
+                  case APACHE_SPARK:
+                    fsOperator.mkdirs(appHome);
+                    fsOperator.copy(uploadJar, appHome, false, true);
+                    break;
+                  default:
+                    throw new IllegalArgumentException(
+                        "[StreamPark] unsupported ApplicationType of custom code: "
+                            + app.getApplicationType());
+                }
+              } else {
+                fsOperator.upload(app.getDistHome(), appHome);
+              }
+            } else {
+              if (!app.getDependencyObject().getJar().isEmpty()) {
+                String localUploads = Workspace.local().APP_UPLOADS();
+                // copy jar to local upload dir
+                for (String jar : app.getDependencyObject().getJar()) {
+                  File localJar = new File(WebUtils.getAppTempDir(), jar);
+                  File uploadJar = new File(localUploads, jar);
+                  if (!localJar.exists() && !uploadJar.exists()) {
+                    throw new ApiAlertException("Missing file: " + jar + ", please upload again");
+                  }
+                  if (localJar.exists()) {
+                    checkOrElseUploadJar(
+                        FsOperator.lfs(), localJar, uploadJar.getAbsolutePath(), localUploads);
+                  }
+                }
+              }
+            }
+          }
+
+          @Override
+          public void onStepStateChange(PipeSnapshot snapshot) {
+            AppBuildPipeline buildPipeline =
+                AppBuildPipeline.fromPipeSnapshot(snapshot).setAppId(app.getId());
+            saveEntity(buildPipeline);
+          }
+
+          @Override
+          public void onFinish(PipeSnapshot snapshot, BuildResult result) {
+            AppBuildPipeline buildPipeline =
+                AppBuildPipeline.fromPipeSnapshot(snapshot)
+                    .setAppId(app.getId())
+                    .setBuildResult(result);
+            saveEntity(buildPipeline);
+            if (result.pass()) {
+              // running job ...
+              if (app.isRunning()) {
+                app.setRelease(ReleaseStateEnum.NEED_RESTART.get());
+              } else {
+                app.setOptionState(OptionStateEnum.NONE.getValue());
+                app.setRelease(ReleaseStateEnum.DONE.get());
+                // If the current task is not running, or the task has just been added, directly set
+                // the candidate version to the official version
+                if (app.isSparkSqlJob()) {
+                  applicationManageService.toEffective(app);
+                } else {
+                  if (app.isStreamParkJob()) {
+                    ApplicationConfig config = applicationConfigService.getLatest(app.getId());
+                    if (config != null) {
+                      config.setToApplication(app);
+                      applicationConfigService.toEffective(app.getId(), app.getConfigId());
+                    }
+                  }
+                }
+              }
+              applicationLog.setSuccess(true);
+              app.setBuild(false);
+
+            } else {
+              Message message =
+                  new Message(
+                      commonService.getUserId(),
+                      app.getId(),
+                      app.getJobName().concat(" release failed"),
+                      ExceptionUtils.stringifyException(snapshot.error().exception()),
+                      NoticeTypeEnum.EXCEPTION);
+              messageService.push(message);
+              app.setRelease(ReleaseStateEnum.FAILED.get());
+              app.setOptionState(OptionStateEnum.NONE.getValue());
+              app.setBuild(true);
+              applicationLog.setException(
+                  ExceptionUtils.stringifyException(snapshot.error().exception()));
+              applicationLog.setSuccess(false);
+            }
+            applicationManageService.updateRelease(app);
+            applicationLogService.save(applicationLog);
+            if (flinkAppHttpWatcher.isWatchingApp(app.getId())) {
+              flinkAppHttpWatcher.init();
+            }
+          }
+        });
+    // save pipeline instance snapshot to db before release it.
+    AppBuildPipeline buildPipeline =
+        AppBuildPipeline.initFromPipeline(pipeline).setAppId(app.getId());
+    boolean saved = saveEntity(buildPipeline);
+    // async release pipeline
+    executorService.submit((Runnable) pipeline::launch);
+    return saved;
+  }
+
+  /**
+   * check the build environment
+   *
+   * @param appId application id
+   * @param forceBuild forced start pipeline or not
+   */
+  private void checkBuildEnv(Long appId, boolean forceBuild) {
+    SparkApplication app = applicationManageService.getById(appId);
+
+    // 1) check flink version
+    SparkEnv env = sparkEnvService.getById(app.getVersionId());
+    boolean checkVersion = env.getSparkVersion().checkVersion(false);
+    ApiAlertException.throwIfFalse(
+        checkVersion, "Unsupported flink version:" + env.getSparkVersion().version());
+
+    // 2) check env
+    boolean envOk = applicationInfoService.checkEnv(app);
+    ApiAlertException.throwIfFalse(
+        envOk, "Check flink env failed, please check the flink version of this job");
+
+    // 3) Whether the application can currently start a new building progress
+    ApiAlertException.throwIfTrue(
+        !forceBuild && !allowToBuildNow(appId),
+        "The job is invalid, or the job cannot be built while it is running");
+  }
+
+  /** create building pipeline instance */
+  private BuildPipeline createPipelineInstance(@Nonnull SparkApplication app) {
+    SparkEnv sparkEnv = sparkEnvService.getByIdOrDefault(app.getVersionId());
+    String sparkUserJar = retrieveSparkUserJar(sparkEnv, app);
+
+    if (!FileUtils.exists(sparkUserJar)) {
+      Resource resource = resourceService.findByResourceName(app.getTeamId(), app.getJar());
+      if (resource != null && StringUtils.isNotBlank(resource.getFilePath())) {
+        sparkUserJar = resource.getFilePath();
+      }
+    }
+
+    SparkExecutionMode executionModeEnum = app.getSparkExecutionMode();
+    String mainClass = Constant.STREAMPARK_SPARKSQL_CLIENT_CLASS;
+    switch (executionModeEnum) {
+      case YARN_CLUSTER:
+      case YARN_CLIENT:
+        String yarnProvidedPath = app.getAppLib();
+        String localWorkspace = app.getLocalAppHome().concat("/lib");
+        if (FlinkDevelopmentMode.CUSTOM_CODE == app.getDevelopmentMode()
+            && ApplicationType.APACHE_FLINK == app.getApplicationType()) {
+          yarnProvidedPath = app.getAppHome();
+          localWorkspace = app.getLocalAppHome();
+        }
+        SparkYarnApplicationBuildRequest yarnAppRequest =
+            new SparkYarnApplicationBuildRequest(
+                app.getJobName(),
+                mainClass,
+                localWorkspace,
+                yarnProvidedPath,
+                app.getDevelopmentMode(),
+                getMergedDependencyInfo(app));
+        log.info("Submit params to building pipeline : {}", yarnAppRequest);
+        return SparkYarnApplicationBuildPipeline.of(yarnAppRequest);
+      default:
+        throw new UnsupportedOperationException(
+            "Unsupported Building Application for ExecutionMode: " + app.getSparkExecutionMode());
+    }
+  }
+
+  private String retrieveSparkUserJar(SparkEnv sparkEnv, SparkApplication app) {
+    switch (app.getDevelopmentMode()) {
+      case CUSTOM_CODE:
+        switch (app.getApplicationType()) {
+          case STREAMPARK_SPARK:
+            return String.format(
+                "%s/%s", app.getAppLib(), app.getModule().concat(Constant.JAR_SUFFIX));
+          case APACHE_SPARK:
+            return String.format("%s/%s", app.getAppHome(), app.getJar());
+          default:
+            throw new IllegalArgumentException(
+                "[StreamPark] unsupported ApplicationType of custom code: "
+                    + app.getApplicationType());
+        }
+      case PYFLINK:
+        return String.format("%s/%s", app.getAppHome(), app.getJar());
+      case FLINK_SQL:
+        String sqlDistJar = commonService.getSqlClientJar(sparkEnv);
+        if (app.getSparkExecutionMode() == SparkExecutionMode.YARN_CLUSTER) {
+          String clientPath = Workspace.remote().APP_CLIENT();
+          return String.format("%s/%s", clientPath, sqlDistJar);
+        }
+        return Workspace.local().APP_CLIENT().concat("/").concat(sqlDistJar);
+      default:
+        throw new UnsupportedOperationException(
+            "[StreamPark] unsupported JobType: " + app.getDevelopmentMode());
+    }
+  }
+
+  @Override
+  public Optional<AppBuildPipeline> getCurrentBuildPipeline(@Nonnull Long appId) {
+    return Optional.ofNullable(getById(appId));
+  }
+
+  @Override
+  public boolean allowToBuildNow(@Nonnull Long appId) {
+    return getCurrentBuildPipeline(appId)
+        .map(pipeline -> PipelineStatusEnum.running != pipeline.getPipelineStatus())
+        .orElse(true);
+  }
+
+  @Override
+  public Map<Long, PipelineStatusEnum> listAppIdPipelineStatusMap(List<Long> appIds) {
+    if (CollectionUtils.isEmpty(appIds)) {
+      return Collections.emptyMap();
+    }
+    LambdaQueryWrapper<AppBuildPipeline> queryWrapper =
+        new LambdaQueryWrapper<AppBuildPipeline>().in(AppBuildPipeline::getAppId, appIds);
+
+    List<AppBuildPipeline> appBuildPipelines = baseMapper.selectList(queryWrapper);
+    if (CollectionUtils.isEmpty(appBuildPipelines)) {
+      return Collections.emptyMap();
+    }
+    return appBuildPipelines.stream()
+        .collect(Collectors.toMap(AppBuildPipeline::getAppId, AppBuildPipeline::getPipelineStatus));
+  }
+
+  @Override
+  public void removeByAppId(Long appId) {
+    baseMapper.delete(
+        new LambdaQueryWrapper<AppBuildPipeline>().eq(AppBuildPipeline::getAppId, appId));
+  }
+
+  /**
+   * save or update build pipeline
+   *
+   * @param pipe application build pipeline
+   * @return value after the save or update
+   */
+  public boolean saveEntity(AppBuildPipeline pipe) {
+    AppBuildPipeline old = getById(pipe.getAppId());
+    if (old == null) {
+      return save(pipe);
+    }
+    return updateById(pipe);
+  }
+
+  /**
+   * Check if the jar exists, and upload a copy if it does not exist
+   *
+   * @param fsOperator
+   * @param localJar
+   * @param targetJar
+   * @param targetDir
+   */
+  private void checkOrElseUploadJar(
+      FsOperator fsOperator, File localJar, String targetJar, String targetDir) {
+    if (!fsOperator.exists(targetJar)) {
+      fsOperator.upload(localJar.getAbsolutePath(), targetDir, false, true);
+    } else {
+      // The file exists to check whether it is consistent, and if it is inconsistent, re-upload it
+      if (!FileUtils.equals(localJar, new File(targetJar))) {
+        fsOperator.upload(localJar.getAbsolutePath(), targetDir, false, true);
+      }
+    }
+  }
+
+  /**
+   * Gets and parses dependencies on the application
+   *
+   * @param application
+   * @return DependencyInfo
+   */
+  private DependencyInfo getMergedDependencyInfo(SparkApplication application) {
+    DependencyInfo dependencyInfo = application.getDependencyInfo();
+    if (StringUtils.isBlank(application.getTeamResource())) {
+      return dependencyInfo;
+    }
+
+    try {
+      String[] resourceIds = JacksonUtils.read(application.getTeamResource(), String[].class);
+
+      List<Artifact> mvnArtifacts = new ArrayList<Artifact>();
+      List<String> jarLibs = new ArrayList<String>();
+
+      Arrays.stream(resourceIds)
+          .forEach(
+              resourceId -> {
+                Resource resource = resourceService.getById(resourceId);
+
+                if (resource.getResourceType() != ResourceTypeEnum.GROUP) {
+                  mergeDependency(application, mvnArtifacts, jarLibs, resource);
+                } else {
+                  try {
+                    String[] groupElements =
+                        JacksonUtils.read(resource.getResource(), String[].class);
+                    Arrays.stream(groupElements)
+                        .forEach(
+                            resourceIdInGroup ->
+                                mergeDependency(
+                                    application,
+                                    mvnArtifacts,
+                                    jarLibs,
+                                    resourceService.getById(resourceIdInGroup)));
+                  } catch (JsonProcessingException e) {
+                    throw new ApiAlertException("Parse resource group failed.", e);
+                  }
+                }
+              });
+      return dependencyInfo.merge(mvnArtifacts, jarLibs);
+    } catch (Exception e) {
+      log.warn("Merge team dependency failed.", e);
+      return dependencyInfo;
+    }
+  }
+
+  private static void mergeDependency(
+      SparkApplication application,
+      List<Artifact> mvnArtifacts,
+      List<String> jarLibs,
+      Resource resource) {
+    Dependency dependency = Dependency.toDependency(resource.getResource());
+    dependency
+        .getPom()
+        .forEach(
+            pom ->
+                mvnArtifacts.add(
+                    new Artifact(
+                        pom.getGroupId(),
+                        pom.getArtifactId(),
+                        pom.getVersion(),
+                        pom.getClassifier())));
+    dependency
+        .getJar()
+        .forEach(
+            jar ->
+                jarLibs.add(
+                    String.format(
+                        "%s/%d/%s",
+                        Workspace.local().APP_UPLOADS(), application.getTeamId(), jar)));
+  }
+}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/SparkEnvServiceImpl.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/SparkEnvServiceImpl.java
new file mode 100644
index 0000000..bfe2398
--- /dev/null
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/SparkEnvServiceImpl.java
@@ -0,0 +1,163 @@
+/*
+ * 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.streampark.console.core.service.impl;
+
+import org.apache.streampark.console.base.exception.ApiAlertException;
+import org.apache.streampark.console.core.entity.SparkEnv;
+import org.apache.streampark.console.core.enums.FlinkEnvCheckEnum;
+import org.apache.streampark.console.core.mapper.SparkEnvMapper;
+import org.apache.streampark.console.core.service.FlinkClusterService;
+import org.apache.streampark.console.core.service.SparkEnvService;
+import org.apache.streampark.console.core.service.application.ApplicationInfoService;
+
+import com.baomidou.mybatisplus.core.conditions.query.LambdaQueryWrapper;
+import com.baomidou.mybatisplus.extension.service.impl.ServiceImpl;
+import lombok.extern.slf4j.Slf4j;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Service;
+import org.springframework.transaction.annotation.Propagation;
+import org.springframework.transaction.annotation.Transactional;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Date;
+
+@Slf4j
+@Service
+@Transactional(propagation = Propagation.SUPPORTS, readOnly = true, rollbackFor = Exception.class)
+public class SparkEnvServiceImpl extends ServiceImpl<SparkEnvMapper, SparkEnv>
+    implements SparkEnvService {
+
+  @Autowired private FlinkClusterService flinkClusterService;
+  @Autowired private ApplicationInfoService applicationInfoService;
+
+  /**
+   * two places will be checked: <br>
+   * 1) name repeated <br>
+   * 2) spark jars repeated <br>
+   */
+  @Override
+  public FlinkEnvCheckEnum check(SparkEnv version) {
+    // 1) check name
+    LambdaQueryWrapper<SparkEnv> queryWrapper =
+        new LambdaQueryWrapper<SparkEnv>().eq(SparkEnv::getSparkName, version.getSparkName());
+    if (version.getId() != null) {
+      queryWrapper.ne(SparkEnv::getId, version.getId());
+    }
+    if (this.count(queryWrapper) > 0) {
+      return FlinkEnvCheckEnum.NAME_REPEATED;
+    }
+
+    String lib = version.getSparkHome().concat("/jars");
+    File sparkLib = new File(lib);
+    // 2) spark/jars path exists and is a directory
+    if (!sparkLib.exists() || !sparkLib.isDirectory()) {
+      return FlinkEnvCheckEnum.INVALID_PATH;
+    }
+
+    return FlinkEnvCheckEnum.OK;
+  }
+
+  @Override
+  public boolean create(SparkEnv version) throws Exception {
+    long count = this.baseMapper.selectCount(null);
+    version.setIsDefault(count == 0);
+    version.setCreateTime(new Date());
+    version.doSetSparkConf();
+    version.doSetVersion();
+    return save(version);
+  }
+
+  @Override
+  public void removeById(Long id) {
+    SparkEnv sparkEnv = getById(id);
+    checkOrElseAlert(sparkEnv);
+    Long count = this.baseMapper.selectCount(null);
+    ApiAlertException.throwIfFalse(
+        !(count > 1 && sparkEnv.getIsDefault()),
+        "The spark home is set as default, please change it first.");
+
+    this.baseMapper.deleteById(id);
+  }
+
+  @Override
+  public void update(SparkEnv version) throws IOException {
+    SparkEnv sparkEnv = getById(version.getId());
+    checkOrElseAlert(sparkEnv);
+    sparkEnv.setDescription(version.getDescription());
+    sparkEnv.setSparkName(version.getSparkName());
+    if (!version.getSparkHome().equals(sparkEnv.getSparkHome())) {
+      sparkEnv.setSparkHome(version.getSparkHome());
+      sparkEnv.doSetSparkConf();
+      sparkEnv.doSetVersion();
+    }
+    updateById(sparkEnv);
+  }
+
+  @Override
+  public void setDefault(Long id) {
+    this.baseMapper.setDefault(id);
+  }
+
+  @Override
+  public SparkEnv getByAppId(Long appId) {
+    return this.baseMapper.selectByAppId(appId);
+  }
+
+  @Override
+  public SparkEnv getDefault() {
+    return this.baseMapper.selectOne(
+        new LambdaQueryWrapper<SparkEnv>().eq(SparkEnv::getIsDefault, true));
+  }
+
+  @Override
+  public SparkEnv getByIdOrDefault(Long id) {
+    SparkEnv sparkEnv = getById(id);
+    return sparkEnv == null ? getDefault() : sparkEnv;
+  }
+
+  @Override
+  public void syncConf(Long id) {
+    SparkEnv sparkEnv = getById(id);
+    sparkEnv.doSetSparkConf();
+    updateById(sparkEnv);
+  }
+
+  @Override
+  public void validity(Long id) {
+    SparkEnv sparkEnv = getById(id);
+    checkOrElseAlert(sparkEnv);
+  }
+
+  private void checkOrElseAlert(SparkEnv sparkEnv) {
+
+    // 1.check exists
+    ApiAlertException.throwIfNull(sparkEnv, "The spark home does not exist, please check.");
+
+    // todo : To be developed
+    // 2.check if it is being used by any spark cluster
+    //    ApiAlertException.throwIfTrue(
+    //        flinkClusterService.existsByFlinkEnvId(sparkEnv.getId()),
+    //        "The spark home is still in use by some spark cluster, please check.");
+    //
+    //    // 3.check if it is being used by any application
+    //    ApiAlertException.throwIfTrue(
+    //        applicationInfoService.existsBySparkEnvId(sparkEnv.getId()),
+    //        "The spark home is still in use by some application, please check.");
+  }
+}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/YarnQueueServiceImpl.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/YarnQueueServiceImpl.java
index d52bdc9..66235ea 100644
--- a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/YarnQueueServiceImpl.java
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/YarnQueueServiceImpl.java
@@ -18,6 +18,7 @@
 package org.apache.streampark.console.core.service.impl;
 
 import org.apache.streampark.common.enums.FlinkExecutionMode;
+import org.apache.streampark.common.enums.SparkExecutionMode;
 import org.apache.streampark.common.util.AssertUtils;
 import org.apache.streampark.console.base.domain.RestRequest;
 import org.apache.streampark.console.base.exception.ApiAlertException;
@@ -183,6 +184,13 @@
   }
 
   @Override
+  public void checkQueueLabel(SparkExecutionMode executionModeEnum, String queueLabel) {
+    if (SparkExecutionMode.isYarnMode(executionModeEnum)) {
+      ApiAlertException.throwIfFalse(isValid(queueLabel, true), ERR_FORMAT_HINTS);
+    }
+  }
+
+  @Override
   public boolean isDefaultQueue(String queueLabel) {
     return StringUtils.equals(DEFAULT_QUEUE, queueLabel) || StringUtils.isBlank(queueLabel);
   }
diff --git a/streampark-console/streampark-console-service/src/main/resources/db/schema-h2.sql b/streampark-console/streampark-console-service/src/main/resources/db/schema-h2.sql
index c666f50..3029648 100644
--- a/streampark-console/streampark-console-service/src/main/resources/db/schema-h2.sql
+++ b/streampark-console/streampark-console-service/src/main/resources/db/schema-h2.sql
@@ -144,6 +144,7 @@
   unique (`flink_name`)
 );
 
+
 -- ----------------------------
 -- Table structure for t_flink_log
 -- ----------------------------
@@ -489,3 +490,89 @@
   unique key (`team_id`,`queue_label`),
   primary key (`id`)
 );
+
+-- ----------------------------
+-- Table structure for t_spark_env
+-- ----------------------------
+create table if not exists `t_spark_env` (
+                                             `id` bigint generated by default as identity not null,
+                                             `spark_name` varchar(128)  not null comment 'spark instance name',
+    `spark_home` varchar(255)  not null comment 'spark home path',
+    `version` varchar(64)  not null comment 'spark version',
+    `scala_version` varchar(64)  not null comment 'scala version of spark',
+    `spark_conf` text  not null comment 'spark-conf',
+    `is_default` tinyint not null default 0 comment 'whether default version or not',
+    `description` varchar(255)  default null comment 'description',
+    `create_time` datetime not null default current_timestamp comment 'create time',
+    primary key(`id`),
+    unique (`spark_name`)
+    );
+
+-- ----------------------------
+-- Table structure for t_spark_app
+-- ----------------------------
+create table if not exists `t_spark_app` (
+                                             `id` bigint generated by default as identity not null,
+                                             `team_id` bigint not null,
+                                             `job_type` tinyint default null,
+                                             `execution_mode` tinyint default null,
+                                             `resource_from` tinyint default null,
+                                             `project_id` bigint default null,
+                                             `job_name` varchar(255)  default null,
+    `module` varchar(255)  default null,
+    `jar` varchar(255)  default null,
+    `jar_check_sum` bigint default null,
+    `main_class` varchar(255)  default null,
+    `args` text,
+    `options` text,
+    `hot_params` text ,
+    `user_id` bigint default null,
+    `app_id` varchar(64)  default null,
+    `app_type` tinyint default null,
+    `duration` bigint default null,
+    `job_id` varchar(64)  default null,
+    `job_manager_url` varchar(255)  default null,
+    `version_id` bigint default null,
+    `cluster_id` varchar(45)  default null,
+    `k8s_name` varchar(63)  default null,
+    `k8s_namespace` varchar(63)  default null,
+    `spark_image` varchar(128)  default null,
+    `state` int default null,
+    `restart_size` int default null,
+    `restart_count` int default null,
+    `cp_threshold` int default null,
+    `cp_max_failure_interval` int default null,
+    `cp_failure_rate_interval` int default null,
+    `cp_failure_action` tinyint default null,
+    `dynamic_properties` text ,
+    `description` varchar(255)  default null,
+    `resolve_order` tinyint default null,
+    `k8s_rest_exposed_type` tinyint default null,
+    `jm_memory` int default null,
+    `tm_memory` int default null,
+    `total_task` int default null,
+    `total_tm` int default null,
+    `total_slot` int default null,
+    `available_slot` int default null,
+    `option_state` tinyint default null,
+    `tracking` tinyint default null,
+    `create_time` datetime not null default current_timestamp comment 'create time',
+    `modify_time` datetime not null default current_timestamp comment 'modify time',
+    `option_time` datetime default null,
+    `release` tinyint default 1,
+    `build` tinyint default 1,
+    `start_time` datetime default null,
+    `end_time` datetime default null,
+    `alert_id` bigint default null,
+    `k8s_pod_template` text ,
+    `k8s_jm_pod_template` text ,
+    `k8s_tm_pod_template` text ,
+    `k8s_hadoop_integration` tinyint default 0,
+    `spark_cluster_id` bigint default null,
+    `ingress_template` text ,
+    `default_mode_ingress` text ,
+    `tags` varchar(500) default null,
+    `probing` tinyint default 0,
+    `hadoop_user` varchar(500) default null,
+    primary key(`id`)
+    );
diff --git a/streampark-console/streampark-console-service/src/main/resources/mapper/core/SparkApplicationMapper.xml b/streampark-console/streampark-console-service/src/main/resources/mapper/core/SparkApplicationMapper.xml
new file mode 100644
index 0000000..bb161aa
--- /dev/null
+++ b/streampark-console/streampark-console-service/src/main/resources/mapper/core/SparkApplicationMapper.xml
@@ -0,0 +1,347 @@
+<?xml version="1.0" encoding="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.
+  -->
+<!DOCTYPE mapper PUBLIC "-//mybatis.org//DTD Mapper 3.0//EN" "http://mybatis.org/dtd/mybatis-3-mapper.dtd">
+<mapper namespace="org.apache.streampark.console.core.mapper.SparkApplicationMapper">
+    <resultMap id="BaseResultMap" type="org.apache.streampark.console.core.entity.SparkApplication">
+        <id column="id" jdbcType="BIGINT" property="id"/>
+        <result column="team_id" jdbcType="BIGINT" property="teamId"/>
+        <result column="project_id" jdbcType="BIGINT" property="projectId"/>
+        <result column="module" jdbcType="VARCHAR" property="module"/>
+        <result column="args" jdbcType="LONGVARCHAR" property="args"/>
+        <result column="options" jdbcType="LONGVARCHAR" property="options"/>
+        <result column="dynamic_properties" jdbcType="LONGVARCHAR" property="dynamicProperties"/>
+        <result column="hot_params" jdbcType="VARCHAR" property="hotParams"/>
+        <result column="job_name" jdbcType="VARCHAR" property="jobName"/>
+        <result column="app_id" jdbcType="VARCHAR" property="appId"/>
+        <result column="version_id" jdbcType="BIGINT" property="versionId"/>
+        <result column="cluster_id" jdbcType="VARCHAR" property="clusterId"/>
+        <result column="spark_cluster_id" jdbcType="BIGINT" property="sparkClusterId"/>
+        <result column="spark_image" jdbcType="VARCHAR" property="sparkImage"/>
+        <result column="k8s_name" jdbcType="VARCHAR" property="k8sName"/>
+        <result column="k8s_namespace" jdbcType="VARCHAR" property="k8sNamespace"/>
+        <result column="app_type" jdbcType="INTEGER" property="appType"/>
+        <result column="job_type" jdbcType="INTEGER" property="jobType"/>
+        <result column="resource_from" jdbcType="INTEGER" property="resourceFrom"/>
+        <result column="execution_mode" jdbcType="INTEGER" property="executionMode"/>
+        <result column="tracking" jdbcType="INTEGER" property="tracking"/>
+        <result column="jar" jdbcType="VARCHAR" property="jar"/>
+        <result column="jar_check_sum" jdbcType="VARCHAR" property="jarCheckSum"/>
+        <result column="main_class" jdbcType="VARCHAR" property="mainClass"/>
+        <result column="job_id" jdbcType="VARCHAR" property="jobId"/>
+        <result column="job_manager_url" jdbcType="VARCHAR" property="jobManagerUrl"/>
+        <result column="user_id" jdbcType="BIGINT" property="userId"/>
+        <result column="start_time" jdbcType="DATE" property="startTime"/>
+        <result column="end_time" jdbcType="DATE" property="endTime"/>
+        <result column="duration" jdbcType="BIGINT" property="duration"/>
+        <result column="state" jdbcType="INTEGER" property="state"/>
+        <result column="cp_max_failure_interval" jdbcType="INTEGER" property="cpMaxFailureInterval"/>
+        <result column="cp_failure_rate_interval" jdbcType="INTEGER" property="cpFailureRateInterval"/>
+        <result column="cp_failure_action" jdbcType="INTEGER" property="cpFailureAction"/>
+        <result column="restart_size" jdbcType="INTEGER" property="restartSize"/>
+        <result column="restart_count" jdbcType="INTEGER" property="restartCount"/>
+        <result column="release" jdbcType="INTEGER" property="release"/>
+        <result column="build" jdbcType="BOOLEAN" property="build"/>
+        <result column="resolve_order" jdbcType="INTEGER" property="resolveOrder"/>
+        <result column="total_tm" jdbcType="INTEGER" property="totalTM"/>
+        <result column="total_slot" jdbcType="INTEGER" property="totalSlot"/>
+        <result column="available_slot" jdbcType="INTEGER" property="availableSlot"/>
+        <result column="total_task" jdbcType="INTEGER" property="totalTask"/>
+        <result column="jm_memory" jdbcType="INTEGER" property="jmMemory"/>
+        <result column="tm_memory" jdbcType="INTEGER" property="tmMemory"/>
+        <result column="option_state" jdbcType="INTEGER" property="optionState"/>
+        <result column="alert_id" jdbcType="BIGINT" property="alertId"/>
+        <result column="description" jdbcType="VARCHAR" property="description"/>
+        <result column="create_time" jdbcType="DATE" property="createTime"/>
+        <result column="option_time" jdbcType="DATE" property="optionTime"/>
+        <result column="k8s_rest_exposed_type" jdbcType="INTEGER" property="k8sRestExposedType"/>
+        <result column="k8s_pod_template" jdbcType="LONGVARCHAR" property="k8sPodTemplate"/>
+        <result column="k8s_jm_pod_template" jdbcType="LONGVARCHAR" property="k8sJmPodTemplate"/>
+        <result column="k8s_tm_pod_template" jdbcType="LONGVARCHAR" property="k8sTmPodTemplate"/>
+        <result column="k8s_hadoop_integration" jdbcType="TINYINT" property="k8sHadoopIntegration"/>
+        <result column="rest_url" jdbcType="VARCHAR" property="restUrl"/>
+        <result column="rest_port" jdbcType="INTEGER" property="restPort"/>
+        <result column="tags" jdbcType="VARCHAR" property="tags"/>
+        <result column="probing" jdbcType="INTEGER" property="probing"/>
+    </resultMap>
+
+    <update id="resetOptionState">
+        update t_spark_app
+        set option_state = 0
+    </update>
+
+    <select id="existsRunningJobByClusterId" resultType="java.lang.Boolean" parameterType="java.lang.Long">
+        select
+            CASE
+                WHEN  count(1) > 0 THEN true ELSE false
+            END
+        from t_spark_app
+            where spark_cluster_id = #{clusterId}
+             and state = 5
+             limit 1
+    </select>
+
+    <select id="countAffectedByClusterId" resultType="java.lang.Integer" parameterType="java.lang.Long">
+        select
+            count(1)
+        from t_spark_app
+        where
+            spark_cluster_id = #{clusterId}
+            and state in (5, 7, 9)
+            <choose>
+                <when test="dbType == 'pgsql'">
+                    and (end_time IS NULL or EXTRACT(EPOCH FROM (NOW() - end_time)) &lt;= 2)
+                </when>
+                <when test="dbType == 'mysql' or dbType == 'h2'">
+                    and (end_time IS NULL or TIMESTAMPDIFF(SECOND, end_time, NOW()) &lt;= 2)
+                </when>
+            </choose>
+            limit 1
+    </select>
+
+    <select id="selectAppsByProjectId" resultType="org.apache.streampark.console.core.entity.SparkApplication" parameterType="java.lang.Long">
+        select * from t_spark_app where project_id=#{projectId}
+    </select>
+
+    <select id="selectPage" resultType="org.apache.streampark.console.core.entity.SparkApplication" parameterType="org.apache.streampark.console.core.entity.SparkApplication">
+        select
+            t.*,
+            p.name as projectName,
+            u.username,
+           case
+               when trim(u.nick_name) = ''
+                   then u.username
+               else u.nick_name
+            end as nickname,
+            v.version as sparkVersion
+        from t_spark_app t
+        inner join t_user u
+        on t.user_id = u.user_id
+        left join t_spark_env v
+        on t.version_id = v.id
+        <where>
+            t.team_id = #{app.teamId}
+            <if test="app.jobType != null and app.jobType != ''">
+                and t.job_type = #{app.jobType}
+            </if>
+            <if test="app.jobTypeArray != null and app.jobTypeArray.length>0">
+                and t.job_type in
+                <foreach item="item" index="index" collection="app.jobTypeArray" open="("  close=")" separator=",">
+                    #{item}
+                </foreach>
+            </if>
+            <if test="app.executionMode != null and app.executionMode != ''">
+                and t.execution_mode = #{app.executionMode}
+            </if>
+            <if test="app.jobName != null and app.jobName != ''">
+                and t.job_name like concat('%', '${app.jobName}', '%')
+            </if>
+            <if test="app.projectName != null and app.projectName != ''">
+                and p.name like concat('%', '${app.projectName}', '%')
+            </if>
+            <if test="app.appId != null and app.appId != ''">
+                and t.app_id = #{app.appId}
+            </if>
+            <if test="app.state != null and app.state != ''">
+                and t.state = #{app.state}
+            </if>
+
+            <if test="app.userId != null and app.userId != ''">
+                and t.user_id = #{app.userId}
+            </if>
+            <if test="app.stateArray != null and app.stateArray.length>0">
+                and t.state in
+                <foreach item="item" index="index" collection="app.stateArray" open="("  close=")" separator=",">
+                    #{item}
+                </foreach>
+            </if>
+            <if test="app.tags != null and app.tags != ''">
+                and t.tags like concat('%', '${app.tags}', '%')
+            </if>
+        </where>
+    </select>
+
+    <select id="selectAppsByTeamId" resultType="org.apache.streampark.console.core.entity.SparkApplication" parameterType="java.lang.Long">
+        select
+            t.*,
+            u.username,
+            case
+                when trim(u.nick_name) = ''
+                    then u.username
+                else u.nick_name
+            end as nick_name
+        from t_spark_app t
+        inner join t_user u
+        on t.user_id = u.user_id
+        where t.team_id=#{teamId}
+    </select>
+
+    <select id="selectProbeApps" resultType="org.apache.streampark.console.core.entity.SparkApplication">
+        select
+            t.*,
+            u.username,
+            case
+                when trim(u.nick_name) = ''
+                    then u.username
+                else u.nick_name
+                end as nick_name
+        from t_spark_app t
+        inner join t_user u
+        on t.user_id = u.user_id
+        where t.probing = true or (t.tracking = 1 and t.state = 13)
+    </select>
+
+    <update id="mapping" parameterType="org.apache.streampark.console.core.entity.SparkApplication">
+        update t_spark_app
+        <set>
+            <if test="app.jobId != null">
+                job_id=#{app.jobId},
+            </if>
+            <if test="app.appId != null">
+                app_id=#{app.appId},
+            </if>
+            end_time=null,
+            state=14,
+            tracking=1
+        </set>
+        where id=#{app.id}
+    </update>
+
+    <select id="selectRecentK8sNamespaces" resultType="java.lang.String" parameterType="java.lang.Integer">
+        select k8s_namespace
+        from (
+            select k8s_namespace, max(create_time) as ct
+            from t_spark_app
+            where k8s_namespace is not null
+            group by k8s_namespace
+            order by ct desc
+        ) as ns
+        limit #{limitSize}
+    </select>
+
+    <select id="selectRecentK8sClusterIds" resultType="java.lang.String" parameterType="java.util.Map">
+        select cluster_id
+        from (
+            select cluster_id, max(create_time) as ct
+            from t_spark_app
+            where cluster_id is not null
+            and execution_mode = #{executionMode}
+            group by cluster_id
+            order by ct desc
+        ) as ci
+        limit #{limitSize}
+    </select>
+
+    <select id="selectRecentK8sPodTemplates" resultType="java.lang.String" parameterType="java.lang.Integer">
+        select k8s_pod_template
+        from (
+            select k8s_pod_template, max(create_time) as ct
+            from t_spark_app
+            where k8s_pod_template is not null
+            and k8s_pod_template !=''
+            and execution_mode = 6
+            group by k8s_pod_template
+            order by ct desc
+        ) as pt
+        limit #{limitSize}
+    </select>
+
+    <select id="selectRecentK8sJmPodTemplates" resultType="java.lang.String" parameterType="java.lang.Integer">
+        select k8s_jm_pod_template
+        from (
+            select k8s_jm_pod_template, max(create_time) as ct
+            from t_spark_app
+            where k8s_jm_pod_template is not null
+            and k8s_jm_pod_template != ''
+            and execution_mode = 6
+            group by k8s_jm_pod_template
+            order by ct desc
+        ) as pt
+        limit #{limitSize}
+    </select>
+
+    <select id="selectRecentK8sTmPodTemplates" resultType="java.lang.String" parameterType="java.lang.Integer">
+        select k8s_tm_pod_template
+        from (
+            select k8s_tm_pod_template, max(create_time) as ct
+            from t_spark_app
+            where k8s_tm_pod_template is not null
+            and k8s_tm_pod_template != ''
+            and execution_mode = 6
+            group by k8s_tm_pod_template
+            order by ct desc
+        ) as pt
+        limit #{limitSize}
+    </select>
+
+    <update id="persistMetrics" parameterType="org.apache.streampark.console.core.entity.SparkApplication">
+        update t_spark_app
+        <set>
+            <if test="app.jobId != null and app.jobId != ''">
+                job_id=#{app.jobId},
+            </if>
+            <if test="app.tracking != null">
+                tracking=#{app.tracking},
+            </if>
+            <if test="app.optionState != null">
+                option_state=#{app.optionState},
+            </if>
+            <if test="app.startTime != null">
+                start_time=#{app.startTime},
+            </if>
+            <if test="app.endTime != null">
+                end_time=#{app.endTime},
+            </if>
+            <if test="app.duration != null">
+                duration=#{app.duration},
+            </if>
+            <choose>
+                <when test="@org.apache.streampark.console.core.enums.SparkAppStateEnum@isEndState(app.state)">
+                    total_tm=null,
+                    total_slot=null,
+                    total_task=null,
+                    available_slot=null,
+                    jm_memory=null,
+                    tm_memory=null,
+                </when>
+                <otherwise>
+                    <if test="app.totalTM != null">
+                        total_tm=#{app.totalTM},
+                    </if>
+                    <if test="app.totalSlot != null">
+                        total_slot=#{app.totalSlot},
+                    </if>
+                    <if test="app.totalTask != null">
+                        total_task=#{app.totalTask},
+                    </if>
+                    <if test="app.availableSlot != null">
+                        available_slot=#{app.availableSlot},
+                    </if>
+                    <if test="app.jmMemory != null">
+                        jm_memory=#{app.jmMemory},
+                    </if>
+                    <if test="app.tmMemory != null">
+                        tm_memory=#{app.tmMemory},
+                    </if>
+                </otherwise>
+            </choose>
+            state=#{app.state}
+        </set>
+        where id=#{app.id}
+    </update>
+
+</mapper>
diff --git a/streampark-console/streampark-console-service/src/main/resources/mapper/core/SparkEnvMapper.xml b/streampark-console/streampark-console-service/src/main/resources/mapper/core/SparkEnvMapper.xml
new file mode 100644
index 0000000..6fce103
--- /dev/null
+++ b/streampark-console/streampark-console-service/src/main/resources/mapper/core/SparkEnvMapper.xml
@@ -0,0 +1,47 @@
+<?xml version="1.0" encoding="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.
+  -->
+<!DOCTYPE mapper PUBLIC "-//mybatis.org//DTD Mapper 3.0//EN" "http://mybatis.org/dtd/mybatis-3-mapper.dtd">
+<mapper namespace="org.apache.streampark.console.core.mapper.SparkEnvMapper">
+    <resultMap id="BaseResultMap" type="org.apache.streampark.console.core.entity.SparkEnv">
+        <id column="id" jdbcType="BIGINT" property="id"/>
+        <result column="spark_name" jdbcType="VARCHAR" property="sparkName"/>
+        <result column="spark_home" jdbcType="VARCHAR" property="sparkHome"/>
+        <result column="spark_conf" jdbcType="VARCHAR" property="sparkConf"/>
+        <result column="description" jdbcType="VARCHAR" property="description"/>
+        <result column="version" jdbcType="VARCHAR" property="version"/>
+        <result column="scala_version" jdbcType="VARCHAR" property="scalaVersion"/>
+        <result column="is_default" jdbcType="BOOLEAN" property="isDefault"/>
+        <result column="create_time" jdbcType="DATE" property="createTime"/>
+    </resultMap>
+
+    <select id="selectByAppId" resultType="org.apache.streampark.console.core.entity.SparkEnv" parameterType="java.lang.Long">
+        select v.*
+        from t_spark_env v
+        inner join (
+            select version_id
+            from t_spark_app
+            where id = #{appId}
+        ) as t
+        on v.id = t.version_id
+    </select>
+
+    <update id="setDefault" parameterType="java.lang.Long">
+        update t_spark_env
+        set is_default = case id when #{id} then true else false end
+    </update>
+</mapper>
diff --git a/streampark-console/streampark-console-webapp/src/views/resource/upload/useUploadRender.tsx b/streampark-console/streampark-console-webapp/src/views/resource/upload/useUploadRender.tsx
index fc9e56f..084e1c9 100644
--- a/streampark-console/streampark-console-webapp/src/views/resource/upload/useUploadRender.tsx
+++ b/streampark-console/streampark-console-webapp/src/views/resource/upload/useUploadRender.tsx
@@ -68,7 +68,7 @@
   const renderOptions = () => {
     const options = [
       { label: 'Apache Flink', value: EngineTypeEnum.FLINK, disabled: false, src: flinkAppSvg },
-      { label: 'Apache Spark', value: EngineTypeEnum.SPARK, disabled: true, src: sparkSvg },
+      { label: 'Apache Spark', value: EngineTypeEnum.SPARK, disabled: false, src: sparkSvg },
     ];
     return options.map(({ label, value, disabled, src }) => {
       return (
diff --git a/streampark-flink/streampark-flink-packer/src/main/java/org/apache/streampark/flink/packer/pipeline/PipelineTypeEnum.java b/streampark-flink/streampark-flink-packer/src/main/java/org/apache/streampark/flink/packer/pipeline/PipelineTypeEnum.java
index bf460db..9411668 100644
--- a/streampark-flink/streampark-flink-packer/src/main/java/org/apache/streampark/flink/packer/pipeline/PipelineTypeEnum.java
+++ b/streampark-flink/streampark-flink-packer/src/main/java/org/apache/streampark/flink/packer/pipeline/PipelineTypeEnum.java
@@ -79,7 +79,17 @@
           .put(1, "Create building workspace")
           .put(2, "Build shaded flink app jar")
           .build(),
-      K8sAppModeBuildResponse.class);
+      K8sAppModeBuildResponse.class),
+
+  SPARK_YARN_APPLICATION(
+      6,
+      "spark yarn application mode task building pipeline",
+      ImmutableMap.<Integer, String>builder()
+          .put(1, "Prepare hadoop yarn environment and building workspace")
+          .put(2, "Resolve maven dependencies")
+          .put(3, "upload jar to yarn.provided.lib.dirs")
+          .build(),
+      SimpleBuildResponse.class);
 
   private final Integer code;
   /** short description of pipeline type. */
diff --git a/streampark-flink/streampark-flink-packer/src/main/scala/org/apache/streampark/flink/packer/pipeline/BuildRequest.scala b/streampark-flink/streampark-flink-packer/src/main/scala/org/apache/streampark/flink/packer/pipeline/BuildRequest.scala
index abca419..d8ccd11 100644
--- a/streampark-flink/streampark-flink-packer/src/main/scala/org/apache/streampark/flink/packer/pipeline/BuildRequest.scala
+++ b/streampark-flink/streampark-flink-packer/src/main/scala/org/apache/streampark/flink/packer/pipeline/BuildRequest.scala
@@ -122,3 +122,12 @@
     developmentMode: FlinkDevelopmentMode,
     dependencyInfo: DependencyInfo)
   extends BuildParam
+
+case class SparkYarnApplicationBuildRequest(
+    appName: String,
+    mainClass: String,
+    localWorkspace: String,
+    yarnProvidedPath: String,
+    developmentMode: FlinkDevelopmentMode,
+    dependencyInfo: DependencyInfo)
+  extends BuildParam
diff --git a/streampark-flink/streampark-flink-packer/src/main/scala/org/apache/streampark/flink/packer/pipeline/impl/SparkYarnApplicationBuildPipeline.scala b/streampark-flink/streampark-flink-packer/src/main/scala/org/apache/streampark/flink/packer/pipeline/impl/SparkYarnApplicationBuildPipeline.scala
new file mode 100644
index 0000000..dcc940d
--- /dev/null
+++ b/streampark-flink/streampark-flink-packer/src/main/scala/org/apache/streampark/flink/packer/pipeline/impl/SparkYarnApplicationBuildPipeline.scala
@@ -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.streampark.flink.packer.pipeline.impl
+
+import org.apache.streampark.common.conf.Workspace
+import org.apache.streampark.common.enums.FlinkDevelopmentMode
+import org.apache.streampark.common.fs.{FsOperator, HdfsOperator, LfsOperator}
+import org.apache.streampark.common.util.ImplicitsUtils._
+import org.apache.streampark.flink.packer.maven.MavenTool
+import org.apache.streampark.flink.packer.pipeline._
+
+import org.apache.commons.codec.digest.DigestUtils
+
+import java.io.{File, FileInputStream, IOException}
+
+import scala.collection.convert.ImplicitConversions._
+
+/** Building pipeline for spark yarn application mode */
+class SparkYarnApplicationBuildPipeline(request: SparkYarnApplicationBuildRequest)
+  extends BuildPipeline {
+
+  /** the type of pipeline */
+  override def pipeType: PipelineTypeEnum = PipelineTypeEnum.SPARK_YARN_APPLICATION
+
+  override def offerBuildParam: SparkYarnApplicationBuildRequest = request
+
+  /**
+   * the actual build process. the effective steps progress should be implemented in multiple
+   * BuildPipeline.execStep() functions.
+   */
+  @throws[Throwable]
+  override protected def buildProcess(): SimpleBuildResponse = {
+    execStep(1) {
+      request.developmentMode match {
+        case FlinkDevelopmentMode.FLINK_SQL =>
+          LfsOperator.mkCleanDirs(request.localWorkspace)
+          HdfsOperator.mkCleanDirs(request.yarnProvidedPath)
+        case _ =>
+      }
+      logInfo(s"Recreate building workspace: ${request.yarnProvidedPath}")
+    }.getOrElse(throw getError.exception)
+
+    val mavenJars =
+      execStep(2) {
+        request.developmentMode match {
+          case FlinkDevelopmentMode.FLINK_SQL =>
+            val mavenArts = MavenTool.resolveArtifacts(request.dependencyInfo.mavenArts)
+            mavenArts.map(_.getAbsolutePath) ++ request.dependencyInfo.extJarLibs
+          case _ => List[String]()
+        }
+      }.getOrElse(throw getError.exception)
+
+    execStep(3) {
+      mavenJars.foreach(
+        jar => {
+          uploadJarToHdfsOrLfs(FsOperator.lfs, jar, request.localWorkspace)
+          uploadJarToHdfsOrLfs(FsOperator.hdfs, jar, request.yarnProvidedPath)
+        })
+    }.getOrElse(throw getError.exception)
+
+    SimpleBuildResponse()
+  }
+
+  @throws[IOException]
+  private[this] def uploadJarToHdfsOrLfs(
+      fsOperator: FsOperator,
+      origin: String,
+      target: String): Unit = {
+    val originFile = new File(origin)
+    if (!fsOperator.exists(target)) {
+      fsOperator.mkdirs(target)
+    }
+    if (originFile.isFile) {
+      // check file in upload dir
+      fsOperator match {
+        case FsOperator.lfs =>
+          fsOperator.copy(originFile.getAbsolutePath, target)
+        case FsOperator.hdfs =>
+          val uploadFile = s"${Workspace.remote.APP_UPLOADS}/${originFile.getName}"
+          if (fsOperator.exists(uploadFile)) {
+            new FileInputStream(originFile).autoClose(
+              inputStream => {
+                if (DigestUtils.md5Hex(inputStream) != fsOperator.fileMd5(uploadFile)) {
+                  fsOperator.upload(originFile.getAbsolutePath, uploadFile)
+                }
+              })
+          } else {
+            fsOperator.upload(originFile.getAbsolutePath, uploadFile)
+          }
+          // copy jar from upload dir to target dir
+          fsOperator.copy(uploadFile, target)
+      }
+    } else {
+      fsOperator match {
+        case FsOperator.hdfs => fsOperator.upload(originFile.getAbsolutePath, target)
+        case _ =>
+      }
+    }
+  }
+
+}
+
+object SparkYarnApplicationBuildPipeline {
+  def of(request: SparkYarnApplicationBuildRequest): SparkYarnApplicationBuildPipeline =
+    new SparkYarnApplicationBuildPipeline(request)
+}
diff --git a/streampark-spark/pom.xml b/streampark-spark/pom.xml
index 6bd6c7c..e6d26d5 100644
--- a/streampark-spark/pom.xml
+++ b/streampark-spark/pom.xml
@@ -30,6 +30,8 @@
     <modules>
         <module>streampark-spark-core</module>
         <module>streampark-spark-connector</module>
+        <module>streampark-spark-client</module>
+        <module>streampark-spark-sqlclient</module>
     </modules>
 
     <dependencies>
@@ -50,14 +52,12 @@
                     <artifactId>slf4j-log4j12</artifactId>
                 </exclusion>
             </exclusions>
-            <scope>provided</scope>
         </dependency>
 
         <dependency>
             <groupId>org.apache.spark</groupId>
             <artifactId>spark-sql_2.12</artifactId>
             <version>${spark.version}</version>
-            <scope>provided</scope>
         </dependency>
 
         <dependency>
@@ -67,6 +67,27 @@
         </dependency>
 
         <dependency>
+            <groupId>com.esotericsoftware</groupId>
+            <artifactId>kryo-shaded</artifactId>
+            <version>4.0.2</version>
+            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.json4s</groupId>
+            <artifactId>json4s-ast_2.12</artifactId>
+            <version>3.7.0-M5</version>
+            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.json4s</groupId>
+            <artifactId>json4s-core_2.12</artifactId>
+            <version>3.7.0-M5</version>
+            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-client-api</artifactId>
             <scope>provided</scope>
diff --git a/streampark-spark/streampark-spark-client/pom.xml b/streampark-spark/streampark-spark-client/pom.xml
new file mode 100644
index 0000000..6f05f6a
--- /dev/null
+++ b/streampark-spark/streampark-spark-client/pom.xml
@@ -0,0 +1,44 @@
+<?xml version="1.0" encoding="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.
+  -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.streampark</groupId>
+        <artifactId>streampark-spark</artifactId>
+        <version>2.2.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>streampark-spark-client</artifactId>
+    <name>StreamPark : Spark Client</name>
+    <packaging>pom</packaging>
+
+    <modules>
+        <module>streampark-spark-client-api</module>
+        <module>streampark-spark-client-core</module>
+    </modules>
+
+    <profiles>
+        <profile>
+            <id>apache-release</id>
+            <properties>
+                <maven.deploy.skip>true</maven.deploy.skip>
+            </properties>
+        </profile>
+    </profiles>
+
+</project>
diff --git a/streampark-spark/streampark-spark-client/streampark-spark-client-api/pom.xml b/streampark-spark/streampark-spark-client/streampark-spark-client-api/pom.xml
new file mode 100644
index 0000000..a8be5c8
--- /dev/null
+++ b/streampark-spark/streampark-spark-client/streampark-spark-client-api/pom.xml
@@ -0,0 +1,103 @@
+<?xml version="1.0" encoding="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.
+  -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.streampark</groupId>
+        <artifactId>streampark-spark-client</artifactId>
+        <version>2.2.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>streampark-spark-client-api_${scala.binary.version}</artifactId>
+    <name>StreamPark : Spark Client Api</name>
+
+    <dependencies>
+
+        <dependency>
+            <groupId>org.apache.streampark</groupId>
+            <artifactId>streampark-common_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-client-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-client-runtime</artifactId>
+            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.streampark</groupId>
+            <artifactId>streampark-flink-packer_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-shade-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>shade</goal>
+                        </goals>
+                        <configuration>
+                            <createDependencyReducedPom>true</createDependencyReducedPom>
+                            <dependencyReducedPomLocation>${project.basedir}/target/dependency-reduced-pom.xml</dependencyReducedPomLocation>
+                            <artifactSet>
+                                <includes>
+                                    <include>org.json4s:*</include>
+                                </includes>
+                            </artifactSet>
+                            <filters>
+                                <filter>
+                                    <artifact>*:*</artifact>
+                                    <excludes>
+                                        <exclude>META-INF/*.SF</exclude>
+                                        <exclude>META-INF/*.DSA</exclude>
+                                        <exclude>META-INF/*.RSA</exclude>
+                                    </excludes>
+                                </filter>
+                            </filters>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+
+    <profiles>
+        <profile>
+            <id>apache-release</id>
+            <properties>
+                <maven.deploy.skip>true</maven.deploy.skip>
+            </properties>
+        </profile>
+    </profiles>
+
+</project>
diff --git a/streampark-spark/streampark-spark-client/streampark-spark-client-api/src/main/scala/org/apache/streampark/spark/client/SparkClient.scala b/streampark-spark/streampark-spark-client/streampark-spark-client-api/src/main/scala/org/apache/streampark/spark/client/SparkClient.scala
new file mode 100644
index 0000000..a452241
--- /dev/null
+++ b/streampark-spark/streampark-spark-client/streampark-spark-client-api/src/main/scala/org/apache/streampark/spark/client/SparkClient.scala
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.streampark.spark.client
+
+import org.apache.streampark.common.conf.SparkVersion
+import org.apache.streampark.common.util.Logger
+import org.apache.streampark.spark.client.bean._
+import org.apache.streampark.spark.client.proxy.SparkShimsProxy
+
+import scala.language.{implicitConversions, reflectiveCalls}
+import scala.reflect.ClassTag
+
+object SparkClient extends Logger {
+
+  private[this] val SPARK_CLIENT_ENDPOINT_CLASS =
+    "org.apache.streampark.spark.client.SparkClientEndpoint"
+
+  private[this] val SUBMIT_REQUEST =
+    "org.apache.streampark.spark.client.bean.SubmitRequest" -> "submit"
+
+  private[this] val CANCEL_REQUEST =
+    "org.apache.streampark.spark.client.bean.CancelRequest" -> "cancel"
+
+  def submit(submitRequest: SubmitRequest): SubmitResponse = {
+    proxy[SubmitResponse](submitRequest, submitRequest.sparkVersion, SUBMIT_REQUEST)
+  }
+
+  def cancel(stopRequest: CancelRequest): CancelResponse = {
+    proxy[CancelResponse](stopRequest, stopRequest.sparkVersion, CANCEL_REQUEST)
+  }
+
+  private[this] def proxy[T: ClassTag](
+      request: Object,
+      sparkVersion: SparkVersion,
+      requestBody: (String, String)): T = {
+    sparkVersion.checkVersion()
+    SparkShimsProxy.proxy(
+      sparkVersion,
+      (classLoader: ClassLoader) => {
+        val submitClass = classLoader.loadClass(SPARK_CLIENT_ENDPOINT_CLASS)
+        val requestClass = classLoader.loadClass(requestBody._1)
+        val method = submitClass.getDeclaredMethod(requestBody._2, requestClass)
+        method.setAccessible(true)
+        val obj = method.invoke(null, SparkShimsProxy.getObject(classLoader, request))
+        if (obj == null) null.asInstanceOf[T]
+        else {
+          SparkShimsProxy.getObject[T](this.getClass.getClassLoader, obj)
+        }
+      }
+    )
+  }
+
+}
diff --git a/streampark-spark/streampark-spark-client/streampark-spark-client-api/src/main/scala/org/apache/streampark/spark/client/bean/CancelRequest.scala b/streampark-spark/streampark-spark-client/streampark-spark-client-api/src/main/scala/org/apache/streampark/spark/client/bean/CancelRequest.scala
new file mode 100644
index 0000000..6b8e1bf
--- /dev/null
+++ b/streampark-spark/streampark-spark-client/streampark-spark-client-api/src/main/scala/org/apache/streampark/spark/client/bean/CancelRequest.scala
@@ -0,0 +1,35 @@
+/*
+ * 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.streampark.spark.client.bean
+
+import org.apache.streampark.common.conf.SparkVersion
+import org.apache.streampark.common.enums.SparkExecutionMode
+
+import javax.annotation.Nullable
+
+import java.util.{Map => JavaMap}
+
+case class CancelRequest(
+    id: Long,
+    sparkVersion: SparkVersion,
+    executionMode: SparkExecutionMode,
+    @Nullable properties: JavaMap[String, Any],
+    clusterId: String,
+    jobId: String,
+    withDrain: Boolean,
+    nativeFormat: Boolean)
diff --git a/streampark-spark/streampark-spark-client/streampark-spark-client-api/src/main/scala/org/apache/streampark/spark/client/bean/CancelResponse.scala b/streampark-spark/streampark-spark-client/streampark-spark-client-api/src/main/scala/org/apache/streampark/spark/client/bean/CancelResponse.scala
new file mode 100644
index 0000000..d293947
--- /dev/null
+++ b/streampark-spark/streampark-spark-client/streampark-spark-client-api/src/main/scala/org/apache/streampark/spark/client/bean/CancelResponse.scala
@@ -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.
+ */
+
+package org.apache.streampark.spark.client.bean
+
+case class CancelResponse(savePointDir: String)
diff --git a/streampark-spark/streampark-spark-client/streampark-spark-client-api/src/main/scala/org/apache/streampark/spark/client/bean/SubmitRequest.scala b/streampark-spark/streampark-spark-client/streampark-spark-client-api/src/main/scala/org/apache/streampark/spark/client/bean/SubmitRequest.scala
new file mode 100644
index 0000000..4fe1fac
--- /dev/null
+++ b/streampark-spark/streampark-spark-client/streampark-spark-client-api/src/main/scala/org/apache/streampark/spark/client/bean/SubmitRequest.scala
@@ -0,0 +1,186 @@
+/*
+ * 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.streampark.spark.client.bean
+
+import org.apache.streampark.common.Constant
+import org.apache.streampark.common.conf.{SparkVersion, Workspace}
+import org.apache.streampark.common.conf.ConfigKeys._
+import org.apache.streampark.common.enums._
+import org.apache.streampark.common.util.{DeflaterUtils, HdfsUtils, PropertiesUtils}
+import org.apache.streampark.flink.packer.pipeline.{BuildResult, ShadedBuildResponse}
+
+import com.fasterxml.jackson.databind.ObjectMapper
+
+import javax.annotation.Nullable
+
+import java.io.{File, IOException}
+import java.net.URL
+import java.nio.file.Files
+import java.util.{Map => JavaMap}
+
+import scala.collection.convert.ImplicitConversions._
+import scala.util.Try
+
+case class SubmitRequest(
+    sparkVersion: SparkVersion,
+    executionMode: SparkExecutionMode,
+    properties: JavaMap[String, Any],
+    sparkYaml: String,
+    developmentMode: FlinkDevelopmentMode,
+    id: Long,
+    jobId: String,
+    appName: String,
+    appConf: String,
+    applicationType: ApplicationType,
+    args: String,
+    @Nullable hadoopUser: String,
+    @Nullable buildResult: BuildResult,
+    @Nullable extraParameter: JavaMap[String, Any]) {
+
+  private[this] lazy val appProperties: Map[String, String] = getParameterMap(
+    KEY_SPARK_PROPERTY_PREFIX)
+
+  lazy val appMain: String = this.developmentMode match {
+    case FlinkDevelopmentMode.FLINK_SQL => Constant.STREAMPARK_SPARKSQL_CLIENT_CLASS
+    case _ => appProperties(KEY_FLINK_APPLICATION_MAIN_CLASS)
+  }
+
+  lazy val effectiveAppName: String =
+    if (this.appName == null) appProperties(KEY_FLINK_APP_NAME) else this.appName
+
+  lazy val libs: List[URL] = {
+    val path = s"${Workspace.local.APP_WORKSPACE}/$id/lib"
+    Try(new File(path).listFiles().map(_.toURI.toURL).toList).getOrElse(List.empty[URL])
+  }
+
+  lazy val classPaths: List[URL] = sparkVersion.sparkLibs ++ libs
+
+  lazy val flinkSQL: String = extraParameter.get(KEY_FLINK_SQL()).toString
+
+  lazy val userJarFile: File = {
+    executionMode match {
+      case _ =>
+        checkBuildResult()
+        new File(buildResult.asInstanceOf[ShadedBuildResponse].shadedJarPath)
+    }
+  }
+
+  lazy val safePackageProgram: Boolean = {
+    sparkVersion.version.split("\\.").map(_.trim.toInt) match {
+      case Array(a, b, c) if a >= 3 => b > 1
+      case _ => false
+    }
+  }
+
+  private[this] def getParameterMap(prefix: String = ""): Map[String, String] = {
+    if (this.appConf == null) {
+      return Map.empty[String, String]
+    }
+    val format = this.appConf.substring(0, 7)
+    if (format == "json://") {
+      val json = this.appConf.drop(7)
+      new ObjectMapper()
+        .readValue[JavaMap[String, String]](json, classOf[JavaMap[String, String]])
+        .toMap
+        .filter(_._2 != null)
+    } else {
+      lazy val content = DeflaterUtils.unzipString(this.appConf.trim.drop(7))
+      val map = format match {
+        case "yaml://" => PropertiesUtils.fromYamlText(content)
+        case "conf://" => PropertiesUtils.fromHoconText(content)
+        case "prop://" => PropertiesUtils.fromPropertiesText(content)
+        case "hdfs://" =>
+          /**
+           * If the configuration file is HDFS mode, you need to copy the HDFS related configuration
+           * file to resources.
+           */
+          val text = HdfsUtils.read(this.appConf)
+          val extension = this.appConf.split("\\.").last.toLowerCase
+          extension match {
+            case "yml" | "yaml" => PropertiesUtils.fromYamlText(text)
+            case "conf" => PropertiesUtils.fromHoconText(text)
+            case "properties" => PropertiesUtils.fromPropertiesText(text)
+            case _ =>
+              throw new IllegalArgumentException(
+                "[StreamPark] Usage: application config format error,must be [yaml|conf|properties]")
+          }
+        case _ =>
+          throw new IllegalArgumentException("[StreamPark] application config format error.")
+      }
+      map
+        .filter(_._1.startsWith(prefix))
+        .filter(_._2.nonEmpty)
+        .map(x => x._1.drop(prefix.length) -> x._2)
+    }
+  }
+
+  @throws[IOException]
+  def isSymlink(file: File): Boolean = {
+    if (file == null) throw new NullPointerException("File must not be null")
+    Files.isSymbolicLink(file.toPath)
+  }
+
+  private[client] lazy val hdfsWorkspace = {
+
+    /**
+     * The spark version and configuration in the native spark and hdfs must be kept exactly the
+     * same.
+     */
+    val workspace = Workspace.remote
+    val sparkHome = sparkVersion.sparkHome
+    val sparkHomeDir = new File(sparkHome)
+    val sparkName = if (isSymlink(sparkHomeDir)) {
+      sparkHomeDir.getCanonicalFile.getName
+    } else {
+      sparkHomeDir.getName
+    }
+    val sparkHdfsHome = s"${workspace.APP_SPARK}/$sparkName"
+    HdfsWorkspace(
+      sparkName,
+      sparkHome,
+      sparkLib = s"$sparkHdfsHome/jars",
+      sparkPlugins = s"$sparkHdfsHome/plugins",
+      appJars = workspace.APP_JARS,
+      appPlugins = workspace.APP_PLUGINS
+    )
+  }
+
+  @throws[Exception]
+  def checkBuildResult(): Unit = {
+    executionMode match {
+      case _ =>
+        if (this.buildResult == null) {
+          throw new Exception(
+            s"[spark-submit] current job: ${this.effectiveAppName} was not yet built, buildResult is empty")
+        }
+        if (!this.buildResult.pass) {
+          throw new Exception(
+            s"[spark-submit] current job ${this.effectiveAppName} build failed, please check")
+        }
+    }
+  }
+
+}
+
+case class HdfsWorkspace(
+    sparkName: String,
+    sparkHome: String,
+    sparkLib: String,
+    sparkPlugins: String,
+    appJars: String,
+    appPlugins: String)
diff --git a/streampark-spark/streampark-spark-client/streampark-spark-client-api/src/main/scala/org/apache/streampark/spark/client/bean/SubmitResponse.scala b/streampark-spark/streampark-spark-client/streampark-spark-client-api/src/main/scala/org/apache/streampark/spark/client/bean/SubmitResponse.scala
new file mode 100644
index 0000000..d2582dd
--- /dev/null
+++ b/streampark-spark/streampark-spark-client/streampark-spark-client-api/src/main/scala/org/apache/streampark/spark/client/bean/SubmitResponse.scala
@@ -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.streampark.spark.client.bean
+
+import javax.annotation.Nullable
+
+import java.util.{Map => JavaMap}
+
+case class SubmitResponse(
+    clusterId: String,
+    sparkConfig: JavaMap[String, String],
+    @Nullable jobId: String = "",
+    @Nullable jobManagerUrl: String = "")
diff --git a/streampark-spark/streampark-spark-client/streampark-spark-client-api/src/main/scala/org/apache/streampark/spark/client/proxy/ChildFirstClassLoader.scala b/streampark-spark/streampark-spark-client/streampark-spark-client-api/src/main/scala/org/apache/streampark/spark/client/proxy/ChildFirstClassLoader.scala
new file mode 100644
index 0000000..865bcda
--- /dev/null
+++ b/streampark-spark/streampark-spark-client/streampark-spark-client-api/src/main/scala/org/apache/streampark/spark/client/proxy/ChildFirstClassLoader.scala
@@ -0,0 +1,153 @@
+/*
+ * 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.streampark.spark.client.proxy
+
+import java.io.{File, IOException}
+import java.net.{URL, URLClassLoader}
+import java.util
+import java.util.function.Consumer
+import java.util.regex.Pattern
+
+import scala.util.Try
+
+/**
+ * A variant of the URLClassLoader that first loads from the URLs and only after that from the
+ * parent.
+ *
+ * <p>{@link # getResourceAsStream ( String )} uses {@link # getResource ( String )} internally so
+ * we don't override that.
+ */
+
+class ChildFirstClassLoader(
+    urls: Array[URL],
+    parent: ClassLoader,
+    sparkResourcePattern: Pattern,
+    classLoadingExceptionHandler: Consumer[Throwable])
+  extends URLClassLoader(urls, parent) {
+
+  ClassLoader.registerAsParallelCapable()
+
+  def this(urls: Array[URL], parent: ClassLoader, sparkResourcePattern: Pattern) {
+    this(
+      urls,
+      parent,
+      sparkResourcePattern,
+      new Consumer[Throwable] {
+        override def accept(t: Throwable): Unit = {}
+      })
+  }
+
+  private val SPARK_PATTERN =
+    Pattern.compile("spark-(.*).jar", Pattern.CASE_INSENSITIVE | Pattern.DOTALL)
+
+  private val JAR_PROTOCOL = "jar"
+
+  private val PARENT_FIRST_PATTERNS = List(
+    "java.",
+    "javax.xml",
+    "org.slf4j",
+    "org.apache.log4j",
+    "org.apache.logging",
+    "org.apache.commons.logging",
+    "ch.qos.logback",
+    "org.xml",
+    "org.w3c",
+    "org.apache.hadoop"
+  )
+
+  @throws[ClassNotFoundException]
+  override def loadClass(name: String, resolve: Boolean): Class[_] = {
+    try {
+      this.synchronized(this.loadClassWithoutExceptionHandling(name, resolve))
+    } catch {
+      case e: Throwable =>
+        classLoadingExceptionHandler.accept(e)
+        throw e
+    }
+  }
+
+  override def getResource(name: String): URL = {
+    // first, try and find it via the URLClassloader
+    val urlClassLoaderResource = findResource(name)
+    if (urlClassLoaderResource != null) return urlClassLoaderResource
+    // delegate to super
+    super.getResource(name)
+  }
+
+  private def filterSparkShimsResource(urlClassLoaderResource: URL): URL = {
+    if (urlClassLoaderResource != null && JAR_PROTOCOL == urlClassLoaderResource.getProtocol) {
+      val spec = urlClassLoaderResource.getFile
+      val filename = new File(spec.substring(0, spec.indexOf("!/"))).getName
+      if (
+        SPARK_PATTERN.matcher(filename).matches && !sparkResourcePattern.matcher(filename).matches
+      ) {
+        return null
+      }
+    }
+    urlClassLoaderResource
+  }
+
+  private def addResources(result: util.List[URL], resources: util.Enumeration[URL]) = {
+    while (resources.hasMoreElements) {
+      val urlClassLoaderResource = filterSparkShimsResource(resources.nextElement)
+      if (urlClassLoaderResource != null) {
+        result.add(urlClassLoaderResource)
+      }
+    }
+    result
+  }
+
+  @throws[IOException]
+  override def getResources(name: String): util.Enumeration[URL] = {
+    // first get resources from URLClassloader
+    val result = addResources(new util.ArrayList[URL], findResources(name))
+    val parent = getParent
+    if (parent != null) {
+      // get parent urls
+      addResources(result, parent.getResources(name))
+    }
+    new util.Enumeration[URL]() {
+      final private[proxy] val iter = result.iterator
+
+      override def hasMoreElements: Boolean = iter.hasNext
+
+      override def nextElement: URL = iter.next
+    }
+  }
+
+  @throws[ClassNotFoundException]
+  private def loadClassWithoutExceptionHandling(name: String, resolve: Boolean): Class[_] = {
+    // First, check if the class has already been loaded
+    super.findLoadedClass(name) match {
+      case null =>
+        // check whether the class should go parent-first
+        for (parentFirstPattern <- PARENT_FIRST_PATTERNS) {
+          if (name.startsWith(parentFirstPattern)) {
+            return super.loadClass(name, resolve)
+          }
+        }
+        Try(findClass(name)).getOrElse(super.loadClass(name, resolve))
+      case c =>
+        if (resolve) {
+          resolveClass(c)
+        }
+        c
+    }
+  }
+
+}
diff --git a/streampark-spark/streampark-spark-client/streampark-spark-client-api/src/main/scala/org/apache/streampark/spark/client/proxy/ClassLoaderObjectInputStream.scala b/streampark-spark/streampark-spark-client/streampark-spark-client-api/src/main/scala/org/apache/streampark/spark/client/proxy/ClassLoaderObjectInputStream.scala
new file mode 100644
index 0000000..a03c970
--- /dev/null
+++ b/streampark-spark/streampark-spark-client/streampark-spark-client-api/src/main/scala/org/apache/streampark/spark/client/proxy/ClassLoaderObjectInputStream.scala
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.streampark.spark.client.proxy
+
+import java.io.{InputStream, IOException, ObjectInputStream, ObjectStreamClass}
+import java.lang.reflect.Proxy
+
+import scala.util.Try
+
+/**
+ * A special ObjectInputStream that loads a class based on a specified <code>ClassLoader</code>
+ * rather than the system default. <p> This is useful in dynamic container environments.
+ *
+ * @since 1.1
+ */
+class ClassLoaderObjectInputStream(classLoader: ClassLoader, inputStream: InputStream)
+  extends ObjectInputStream(inputStream) {
+
+  /**
+   * Resolve a class specified by the descriptor using the specified ClassLoader or the super
+   * ClassLoader.
+   *
+   * @param objectStreamClass
+   *   descriptor of the class
+   * @return
+   *   the Class object described by the ObjectStreamClass
+   * @throws IOException
+   *   in case of an I/O error
+   * @throws ClassNotFoundException
+   *   if the Class cannot be found
+   */
+  @throws[IOException]
+  @throws[ClassNotFoundException]
+  override protected def resolveClass(objectStreamClass: ObjectStreamClass): Class[_] = {
+    // delegate to super class loader which can resolve primitives
+    Try(Class.forName(objectStreamClass.getName, false, classLoader))
+      .getOrElse(super.resolveClass(objectStreamClass))
+  }
+
+  /**
+   * Create a proxy class that implements the specified interfaces using the specified ClassLoader
+   * or the super ClassLoader.
+   *
+   * @param interfaces
+   *   the interfaces to implement
+   * @return
+   *   a proxy class implementing the interfaces
+   * @throws IOException
+   *   in case of an I/O error
+   * @throws ClassNotFoundException
+   *   if the Class cannot be found
+   * @see
+   *   ObjectInputStream#resolveProxyClass(String[])
+   * @since 2.1
+   */
+  @throws[IOException]
+  @throws[ClassNotFoundException]
+  override protected def resolveProxyClass(interfaces: Array[String]): Class[_] = {
+    val interfaceClasses = new Array[Class[_]](interfaces.length)
+    for (i <- interfaces.indices) {
+      interfaceClasses(i) = Class.forName(interfaces(i), false, classLoader)
+    }
+    Try(Proxy.getProxyClass(classLoader, interfaceClasses: _*))
+      .getOrElse(super.resolveProxyClass(interfaces))
+  }
+
+}
diff --git a/streampark-spark/streampark-spark-client/streampark-spark-client-api/src/main/scala/org/apache/streampark/spark/client/proxy/SparkShimsProxy.scala b/streampark-spark/streampark-spark-client/streampark-spark-client-api/src/main/scala/org/apache/streampark/spark/client/proxy/SparkShimsProxy.scala
new file mode 100644
index 0000000..40a51d8
--- /dev/null
+++ b/streampark-spark/streampark-spark-client/streampark-spark-client-api/src/main/scala/org/apache/streampark/spark/client/proxy/SparkShimsProxy.scala
@@ -0,0 +1,181 @@
+/*
+ * 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.streampark.spark.client.proxy
+
+import org.apache.streampark.common.Constant
+import org.apache.streampark.common.conf.{ConfigKeys, SparkVersion}
+import org.apache.streampark.common.util.{ClassLoaderUtils, Logger}
+import org.apache.streampark.common.util.ImplicitsUtils._
+
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream, File, ObjectOutputStream}
+import java.net.URL
+import java.util.function.{Function => JavaFunc}
+import java.util.regex.Pattern
+
+import scala.collection.mutable.{ListBuffer, Map => MutableMap}
+
+object SparkShimsProxy extends Logger {
+
+  private[this] val SHIMS_CLASS_LOADER_CACHE = MutableMap[String, ClassLoader]()
+
+  private[this] val VERIFY_SQL_CLASS_LOADER_CACHE = MutableMap[String, ClassLoader]()
+
+  private[this] val INCLUDE_PATTERN: Pattern =
+    Pattern.compile(
+      "(streampark-shaded-jackson-)(.*).jar",
+      Pattern.CASE_INSENSITIVE | Pattern.DOTALL)
+
+  private[this] def getSparkShimsResourcePattern(sparkLargeVersion: String) =
+    Pattern.compile(
+      s"spark-(.*)-$sparkLargeVersion(.*).jar",
+      Pattern.CASE_INSENSITIVE | Pattern.DOTALL)
+
+  private[this] lazy val SPARK_SHIMS_PREFIX = "streampark-spark-shims_spark"
+
+  def proxy[T](sparkVersion: SparkVersion, func: ClassLoader => T): T = {
+    val shimsClassLoader = getSParkShimsClassLoader(sparkVersion)
+    ClassLoaderUtils.runAsClassLoader[T](shimsClassLoader, () => func(shimsClassLoader))
+  }
+
+  def proxy[T](sparkVersion: SparkVersion, func: JavaFunc[ClassLoader, T]): T = {
+    val shimsClassLoader = getSParkShimsClassLoader(sparkVersion)
+    ClassLoaderUtils.runAsClassLoader[T](shimsClassLoader, () => func(shimsClassLoader))
+  }
+
+  // need to load all spark-table dependencies compatible with different versions
+  def getVerifySqlLibClassLoader(sparkVersion: SparkVersion): ClassLoader = {
+    logInfo(s"Add verify sql lib,spark version: $sparkVersion")
+    VERIFY_SQL_CLASS_LOADER_CACHE.getOrElseUpdate(
+      s"${sparkVersion.fullVersion}", {
+        val getSparkTable: File => Boolean = _.getName.startsWith("spark-table")
+        // 1) spark/lib/spark-table*
+        val libTableURL = getSparkHomeLib(sparkVersion.sparkHome, "lib", getSparkTable)
+
+        // 2) After version 1.15 need add spark/opt/spark-table*
+        val optTableURL = getSparkHomeLib(sparkVersion.sparkHome, "opt", getSparkTable)
+        val shimsUrls = ListBuffer[URL](libTableURL ++ optTableURL: _*)
+
+        // 3) add only streampark shims jar
+        addShimsUrls(
+          sparkVersion,
+          file => {
+            if (file.getName.startsWith("streampark-spark-shims")) {
+              shimsUrls += file.toURI.toURL
+            }
+          })
+
+        new ChildFirstClassLoader(
+          shimsUrls.toArray,
+          Thread.currentThread().getContextClassLoader,
+          getSparkShimsResourcePattern(sparkVersion.majorVersion))
+      }
+    )
+  }
+
+  def addShimsUrls(sparkVersion: SparkVersion, addShimUrl: File => Unit): Unit = {
+    val appHome = System.getProperty(ConfigKeys.KEY_APP_HOME)
+    require(
+      appHome != null,
+      String.format("%s is not found on System env.", ConfigKeys.KEY_APP_HOME))
+
+    val libPath = new File(s"$appHome/lib")
+    require(libPath.exists())
+
+    val majorVersion = sparkVersion.majorVersion
+    val scalaVersion = sparkVersion.scalaVersion
+
+    libPath
+      .listFiles()
+      .foreach(
+        (jar: File) => {
+          val jarName = jar.getName
+          if (jarName.endsWith(Constant.JAR_SUFFIX)) {
+            if (jarName.startsWith(SPARK_SHIMS_PREFIX)) {
+              val prefixVer = s"$SPARK_SHIMS_PREFIX-${majorVersion}_$scalaVersion"
+              if (jarName.startsWith(prefixVer)) {
+                addShimUrl(jar)
+                logInfo(s"Include spark shims jar lib: $jarName")
+              }
+            } else {
+              if (INCLUDE_PATTERN.matcher(jarName).matches()) {
+                addShimUrl(jar)
+                logInfo(s"Include jar lib: $jarName")
+              } else if (jarName.matches(s"^streampark-.*_$scalaVersion.*$$")) {
+                addShimUrl(jar)
+                logInfo(s"Include streampark lib: $jarName")
+              }
+            }
+          }
+        })
+  }
+
+  def proxyVerifySql[T](sparkVersion: SparkVersion, func: JavaFunc[ClassLoader, T]): T = {
+    val shimsClassLoader = getVerifySqlLibClassLoader(sparkVersion)
+    ClassLoaderUtils.runAsClassLoader[T](shimsClassLoader, () => func(shimsClassLoader))
+  }
+
+  private[this] def getSParkShimsClassLoader(sparkVersion: SparkVersion): ClassLoader = {
+    logInfo(s"add spark shims urls classloader,spark version: $sparkVersion")
+
+    SHIMS_CLASS_LOADER_CACHE.getOrElseUpdate(
+      s"${sparkVersion.fullVersion}", {
+        // 1) spark/lib
+        val libURL = getSparkHomeLib(sparkVersion.sparkHome, "jars", !_.getName.startsWith("log4j"))
+        val shimsUrls = ListBuffer[URL](libURL: _*)
+
+        // 2) add all shims jar
+        addShimsUrls(
+          sparkVersion,
+          file => {
+            if (file != null) {
+              shimsUrls += file.toURI.toURL
+            }
+          })
+
+        new ChildFirstClassLoader(
+          shimsUrls.toArray,
+          Thread.currentThread().getContextClassLoader,
+          getSparkShimsResourcePattern(sparkVersion.majorVersion)
+        )
+      }
+    )
+  }
+
+  private[this] def getSparkHomeLib(
+      sparkHome: String,
+      childDir: String,
+      filterFun: File => Boolean): List[URL] = {
+    val file = new File(sparkHome, childDir)
+    require(file.isDirectory, s"SPARK_HOME $file does not exist")
+    file.listFiles.filter(filterFun).map(_.toURI.toURL).toList
+  }
+
+  @throws[Exception]
+  def getObject[T](loader: ClassLoader, obj: Object): T = {
+    val arrayOutputStream = new ByteArrayOutputStream
+    new ObjectOutputStream(arrayOutputStream)
+      .autoClose(
+        objectOutputStream => {
+          objectOutputStream.writeObject(obj)
+          val byteArrayInputStream = new ByteArrayInputStream(arrayOutputStream.toByteArray)
+          new ClassLoaderObjectInputStream(loader, byteArrayInputStream).autoClose(_.readObject())
+        })
+      .asInstanceOf[T]
+  }
+
+}
diff --git a/streampark-spark/streampark-spark-client/streampark-spark-client-core/pom.xml b/streampark-spark/streampark-spark-client/streampark-spark-client-core/pom.xml
new file mode 100644
index 0000000..4862618
--- /dev/null
+++ b/streampark-spark/streampark-spark-client/streampark-spark-client-core/pom.xml
@@ -0,0 +1,113 @@
+<?xml version="1.0" encoding="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.
+  -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.streampark</groupId>
+        <artifactId>streampark-spark-client</artifactId>
+        <version>2.2.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>streampark-spark-client-core_${scala.binary.version}</artifactId>
+    <name>StreamPark : Spark Client Core</name>
+
+    <dependencies>
+
+        <dependency>
+            <groupId>org.apache.streampark</groupId>
+            <artifactId>streampark-spark-client-api_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-client-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-client-runtime</artifactId>
+            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.json4s</groupId>
+            <artifactId>json4s-jackson_${scala.binary.version}</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.streampark</groupId>
+            <artifactId>streampark-flink-packer_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-shade-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>shade</goal>
+                        </goals>
+                        <configuration>
+                            <createDependencyReducedPom>true</createDependencyReducedPom>
+                            <dependencyReducedPomLocation>${project.basedir}/target/dependency-reduced-pom.xml</dependencyReducedPomLocation>
+                            <artifactSet>
+                                <includes>
+                                    <include>org.json4s:*</include>
+                                </includes>
+                            </artifactSet>
+                            <filters>
+                                <filter>
+                                    <artifact>*:*</artifact>
+                                    <excludes>
+                                        <exclude>META-INF/*.SF</exclude>
+                                        <exclude>META-INF/*.DSA</exclude>
+                                        <exclude>META-INF/*.RSA</exclude>
+                                    </excludes>
+                                </filter>
+                            </filters>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+    <profiles>
+        <profile>
+            <id>apache-release</id>
+            <properties>
+                <maven.deploy.skip>true</maven.deploy.skip>
+            </properties>
+        </profile>
+    </profiles>
+
+</project>
diff --git a/streampark-spark/streampark-spark-client/streampark-spark-client-core/src/main/scala/org/apache/streampark/spark/client/SparkClientEndpoint.scala b/streampark-spark/streampark-spark-client/streampark-spark-client-core/src/main/scala/org/apache/streampark/spark/client/SparkClientEndpoint.scala
new file mode 100644
index 0000000..f9adf80
--- /dev/null
+++ b/streampark-spark/streampark-spark-client/streampark-spark-client-core/src/main/scala/org/apache/streampark/spark/client/SparkClientEndpoint.scala
@@ -0,0 +1,51 @@
+/*
+ * 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.streampark.spark.client
+
+import org.apache.streampark.common.enums.SparkExecutionMode
+import org.apache.streampark.common.enums.SparkExecutionMode._
+import org.apache.streampark.spark.client.`trait`.SparkClientTrait
+import org.apache.streampark.spark.client.bean._
+import org.apache.streampark.spark.client.impl._
+
+object SparkClientEndpoint {
+
+  private[this] val clients: Map[SparkExecutionMode, SparkClientTrait] = Map(
+    YARN_CLUSTER -> YarnApplicationClient,
+    YARN_CLIENT -> YarnApplicationClient
+  )
+
+  def submit(submitRequest: SubmitRequest): SubmitResponse = {
+    clients.get(submitRequest.executionMode) match {
+      case Some(client) => client.submit(submitRequest)
+      case _ =>
+        throw new UnsupportedOperationException(
+          s"Unsupported ${submitRequest.executionMode} submit ")
+    }
+  }
+
+  def cancel(cancelRequest: CancelRequest): CancelResponse = {
+    clients.get(cancelRequest.executionMode) match {
+      case Some(client) => client.cancel(cancelRequest)
+      case _ =>
+        throw new UnsupportedOperationException(
+          s"Unsupported ${cancelRequest.executionMode} cancel ")
+    }
+  }
+
+}
diff --git a/streampark-spark/streampark-spark-client/streampark-spark-client-core/src/main/scala/org/apache/streampark/spark/client/impl/YarnApplicationClient.scala b/streampark-spark/streampark-spark-client/streampark-spark-client-core/src/main/scala/org/apache/streampark/spark/client/impl/YarnApplicationClient.scala
new file mode 100644
index 0000000..1d9845b
--- /dev/null
+++ b/streampark-spark/streampark-spark-client/streampark-spark-client-core/src/main/scala/org/apache/streampark/spark/client/impl/YarnApplicationClient.scala
@@ -0,0 +1,129 @@
+/*
+ * 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.streampark.spark.client.impl
+
+import org.apache.streampark.common.conf.Workspace
+import org.apache.streampark.flink.packer.pipeline.ShadedBuildResponse
+import org.apache.streampark.spark.client.`trait`.SparkClientTrait
+import org.apache.streampark.spark.client.bean._
+
+import org.apache.commons.collections.MapUtils
+import org.apache.spark.launcher.{SparkAppHandle, SparkLauncher}
+
+import java.util.concurrent.{CountDownLatch, Executors, ExecutorService}
+
+import scala.util.control.Breaks.break
+
+/** yarn application mode submit */
+object YarnApplicationClient extends SparkClientTrait {
+
+  private val threadPool: ExecutorService = Executors.newFixedThreadPool(1)
+
+  private[this] lazy val workspace = Workspace.remote
+
+  override def doCancel(cancelRequest: CancelRequest): CancelResponse = {
+    null
+  }
+
+  override def setConfig(submitRequest: SubmitRequest): Unit = {}
+
+  override def doSubmit(submitRequest: SubmitRequest): SubmitResponse = {
+    launch(submitRequest)
+    null
+
+  }
+
+  private def launch(submitRequest: SubmitRequest): Unit = {
+    val launcher: SparkLauncher = new SparkLauncher()
+      .setSparkHome(submitRequest.sparkVersion.sparkHome)
+      .setAppResource(submitRequest.buildResult.asInstanceOf[ShadedBuildResponse].shadedJarPath)
+      .setMainClass(submitRequest.appMain)
+      .setMaster("yarn")
+      .setDeployMode("cluster")
+      .setAppName(submitRequest.appName)
+      .setConf("spark.executor.memory", "5g")
+      .setConf("spark.executor.cores", "4")
+      .setConf("spark.num.executors", "1")
+      .setConf(
+        "spark.yarn.jars",
+        submitRequest.asInstanceOf[SubmitRequest].hdfsWorkspace.sparkLib + "/*.jar")
+      .setVerbose(true)
+
+    if (
+      MapUtils.isNotEmpty(submitRequest.extraParameter) && submitRequest.extraParameter.containsKey(
+        "sql")
+    ) {
+      launcher.addAppArgs("--sql", submitRequest.extraParameter.get("sql").toString)
+    }
+
+    logger.info("The spark task start")
+
+    threadPool.execute(new Runnable {
+      override def run(): Unit = {
+        try {
+          val countDownLatch: CountDownLatch = new CountDownLatch(1)
+          val sparkAppHandle: SparkAppHandle =
+            launcher.startApplication(new SparkAppHandle.Listener() {
+              override def stateChanged(handle: SparkAppHandle): Unit = {
+                if (handle.getAppId != null) {
+                  logInfo(
+                    String.format("%s stateChanged :%s", handle.getAppId, handle.getState.toString))
+                } else logger.info("stateChanged :{}", handle.getState.toString)
+
+                if (SparkAppHandle.State.FAILED.toString == handle.getState.toString) {
+                  logger.error("Task run failure stateChanged :{}", handle.getState.toString)
+                }
+
+                if (handle.getState.isFinal) countDownLatch.countDown()
+              }
+
+              override def infoChanged(handle: SparkAppHandle): Unit = {}
+            })
+          logger.info(
+            "The task is executing, current is get application id before,please wait ........")
+          var applicationId: String = null
+          while ({
+            !(SparkAppHandle.State.RUNNING == sparkAppHandle.getState)
+          }) {
+            applicationId = sparkAppHandle.getAppId
+            if (applicationId != null) {
+              logInfo(
+                String.format(
+                  "handle current state is %s, appid is %s",
+                  sparkAppHandle.getState.toString,
+                  applicationId))
+              break // todo: break is not supported
+
+            }
+          }
+          logInfo(
+            String.format(
+              "handle current state is %s, appid is %s",
+              sparkAppHandle.getState.toString,
+              applicationId))
+          countDownLatch.await()
+        } catch {
+          case e: Exception =>
+            logger.error(e.getMessage, e)
+        }
+      }
+    })
+
+  }
+
+}
diff --git a/streampark-spark/streampark-spark-client/streampark-spark-client-core/src/main/scala/org/apache/streampark/spark/client/trait/SparkClientTrait.scala b/streampark-spark/streampark-spark-client/streampark-spark-client-core/src/main/scala/org/apache/streampark/spark/client/trait/SparkClientTrait.scala
new file mode 100644
index 0000000..4787510
--- /dev/null
+++ b/streampark-spark/streampark-spark-client/streampark-spark-client-core/src/main/scala/org/apache/streampark/spark/client/trait/SparkClientTrait.scala
@@ -0,0 +1,84 @@
+/*
+ * 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.streampark.spark.client.`trait`
+
+import org.apache.streampark.common.util._
+import org.apache.streampark.spark.client.bean._
+
+import scala.collection.convert.ImplicitConversions._
+import scala.language.postfixOps
+
+trait SparkClientTrait extends Logger {
+
+  @throws[Exception]
+  def submit(submitRequest: SubmitRequest): SubmitResponse = {
+    logInfo(
+      s"""
+         |--------------------------------------- spark job start ---------------------------------------
+         |    userSparkHome    : ${submitRequest.sparkVersion.sparkHome}
+         |    sparkVersion     : ${submitRequest.sparkVersion.version}
+         |    appName          : ${submitRequest.appName}
+         |    devMode          : ${submitRequest.developmentMode.name()}
+         |    execMode         : ${submitRequest.executionMode.name()}
+         |    applicationType  : ${submitRequest.applicationType.getName}
+         |    properties       : ${submitRequest.properties.mkString(" ")}
+         |    args             : ${submitRequest.args}
+         |    appConf          : ${submitRequest.appConf}
+         |-------------------------------------------------------------------------------------------
+         |""".stripMargin)
+
+    submitRequest.developmentMode match {
+      case _ =>
+        if (submitRequest.userJarFile != null) {
+          val uri = submitRequest.userJarFile.getAbsolutePath
+        }
+    }
+
+    setConfig(submitRequest)
+
+    doSubmit(submitRequest)
+
+  }
+
+  def setConfig(submitRequest: SubmitRequest): Unit
+
+  @throws[Exception]
+  def cancel(cancelRequest: CancelRequest): CancelResponse = {
+    logInfo(
+      s"""
+         |----------------------------------------- spark job cancel --------------------------------
+         |     userSparkHome     : ${cancelRequest.sparkVersion.sparkHome}
+         |     sparkVersion      : ${cancelRequest.sparkVersion.version}
+         |     clusterId         : ${cancelRequest.clusterId}
+         |     withDrain         : ${cancelRequest.withDrain}
+         |     nativeFormat      : ${cancelRequest.nativeFormat}
+         |     appId             : ${cancelRequest.clusterId}
+         |     jobId             : ${cancelRequest.jobId}
+         |-------------------------------------------------------------------------------------------
+         |""".stripMargin)
+
+    doCancel(cancelRequest)
+  }
+
+  @throws[Exception]
+  def doSubmit(submitRequest: SubmitRequest): SubmitResponse
+
+  @throws[Exception]
+  def doCancel(cancelRequest: CancelRequest): CancelResponse
+
+}
diff --git a/streampark-spark/streampark-spark-core/src/main/scala/org/apache/streampark/spark/core/Spark.scala b/streampark-spark/streampark-spark-core/src/main/scala/org/apache/streampark/spark/core/Spark.scala
index 83aec63..b004620 100644
--- a/streampark-spark/streampark-spark-core/src/main/scala/org/apache/streampark/spark/core/Spark.scala
+++ b/streampark-spark/streampark-spark-core/src/main/scala/org/apache/streampark/spark/core/Spark.scala
@@ -17,14 +17,19 @@
 
 package org.apache.streampark.spark.core
 
-import org.apache.streampark.common.conf.ConfigConst._
-import org.apache.streampark.common.util.{Logger, PropertiesUtils}
+import org.apache.streampark.common.conf.ConfigKeys._
+import org.apache.streampark.common.util.{DeflaterUtils, Logger, PropertiesUtils}
+import org.apache.streampark.spark.core.util.{ParameterTool, SqlCommandParser}
 
+import org.apache.commons.lang3.StringUtils
 import org.apache.spark.SparkConf
-import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.{DataFrame, SparkSession}
+
+import java.util.concurrent.locks.ReentrantReadWriteLock
 
 import scala.annotation.meta.getter
 import scala.collection.mutable.ArrayBuffer
+import scala.util.{Failure, Success, Try}
 
 /** <b><code>Spark</code></b> <p/> Spark Basic Traits <p/> */
 trait Spark extends Logger {
@@ -44,6 +49,8 @@
   // If recovery from checkpoint fails, recreate
   final protected var createOnError: Boolean = true
 
+  private[this] val lock = new ReentrantReadWriteLock().writeLock
+
   /** Entrance */
   def main(args: Array[String]): Unit = {
 
@@ -78,7 +85,40 @@
     }
 
     ready()
-    handle()
+
+    val parameterTool = ParameterTool.fromArgs(args)
+
+    val sparkSqls = {
+      val sql = parameterTool.get(KEY_FLINK_SQL())
+      require(StringUtils.isNotBlank(sql), "Usage: spark sql cannot be null")
+      Try(DeflaterUtils.unzipString(sql)) match {
+        case Success(value) => value
+        case Failure(_) =>
+          throw new IllegalArgumentException("Usage: spark sql is invalid or null, please check")
+      }
+    }
+
+    SqlCommandParser
+      .parseSQL(sparkSqls)
+      .foreach(
+        x => {
+          val args = if (x.operands.isEmpty) null else x.operands.head
+          val command = x.command.name
+          x.command match {
+            case _ =>
+              try {
+                lock.lock()
+                val dataFrame: DataFrame = handle(x.originSql)
+                logInfo(s"$command:$args")
+              } finally {
+                if (lock.isHeldByCurrentThread) {
+                  lock.unlock()
+                }
+              }
+          }
+        })
+
+//    handle(sparkSqls)
     start()
     destroy()
   }
@@ -110,18 +150,22 @@
       }
     }
 
-    val localConf = conf.split("\\.").last match {
-      case "conf" => PropertiesUtils.fromHoconFile(conf)
-      case "properties" => PropertiesUtils.fromPropertiesFile(conf)
-      case "yaml" | "yml" => PropertiesUtils.fromYamlFile(conf)
-      case _ =>
-        throw new IllegalArgumentException(
-          "[StreamPark] Usage: config file error,must be [properties|yaml|conf]")
+    if (conf != null) {
+      val localConf = conf.split("\\.").last match {
+        case "conf" => PropertiesUtils.fromHoconFile(conf)
+        case "properties" => PropertiesUtils.fromPropertiesFile(conf)
+        case "yaml" | "yml" => PropertiesUtils.fromYamlFile(conf)
+        case _ =>
+          throw new IllegalArgumentException(
+            "[StreamPark] Usage: config file error,must be [properties|yaml|conf]")
+      }
+
+      sparkConf.setAll(localConf)
     }
 
-    sparkConf.setAll(localConf).setAll(userArgs)
+    sparkConf.setAll(userArgs)
 
-    val appMain = sparkConf.get(KEY_SPARK_MAIN_CLASS, null)
+    val appMain = sparkConf.get(KEY_SPARK_MAIN_CLASS, "org.apache.streampark.spark.cli.SqlClient")
     if (appMain == null) {
       logError(s"[StreamPark] parameter: $KEY_SPARK_MAIN_CLASS must not be empty!")
       System.exit(1)
@@ -166,7 +210,7 @@
    * The handle phase is the entry point to the code written by the developer and is the most
    * important phase.
    */
-  def handle(): Unit
+  def handle(sql: String = null): DataFrame = sparkSession.sql(sql)
 
   /** The start phase starts the task, which is executed automatically by the framework. */
   def start(): Unit = {}
diff --git a/streampark-spark/streampark-spark-core/src/main/scala/org/apache/streampark/spark/core/SparkStreaming.scala b/streampark-spark/streampark-spark-core/src/main/scala/org/apache/streampark/spark/core/SparkStreaming.scala
index 849cd3e..99a0540 100644
--- a/streampark-spark/streampark-spark-core/src/main/scala/org/apache/streampark/spark/core/SparkStreaming.scala
+++ b/streampark-spark/streampark-spark-core/src/main/scala/org/apache/streampark/spark/core/SparkStreaming.scala
@@ -17,7 +17,7 @@
 
 package org.apache.streampark.spark.core
 
-import org.apache.streampark.common.conf.ConfigConst._
+import org.apache.streampark.common.conf.ConfigKeys._
 
 import org.apache.spark.SparkConf
 import org.apache.spark.streaming.{Seconds, StreamingContext}
diff --git a/streampark-spark/streampark-spark-core/src/main/scala/org/apache/streampark/spark/core/util/ParameterTool.java b/streampark-spark/streampark-spark-core/src/main/scala/org/apache/streampark/spark/core/util/ParameterTool.java
new file mode 100644
index 0000000..0a2bf27
--- /dev/null
+++ b/streampark-spark/streampark-spark-core/src/main/scala/org/apache/streampark/spark/core/util/ParameterTool.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.streampark.spark.core.util;
+
+import org.apache.commons.lang3.math.NumberUtils;
+import java.util.*;
+
+
+public class ParameterTool  {
+    private static final long serialVersionUID = 15345346345L;
+
+    protected static final String NO_VALUE_KEY = "__NO_VALUE_KEY";
+
+    public static String getKeyFromArgs(String[] args, int index) {
+        String key;
+        if (args[index].startsWith("--")) {
+            key = args[index].substring(2);
+        } else if (args[index].startsWith("-")) {
+            key = args[index].substring(1);
+        } else {
+            throw new IllegalArgumentException(
+                    String.format(
+                            "Error parsing arguments '%s' on '%s'. Please prefix keys with -- or -.",
+                            Arrays.toString(args), args[index]));
+        }
+
+        if (key.isEmpty()) {
+            throw new IllegalArgumentException(
+                    "The input " + Arrays.toString(args) + " contains an empty argument");
+        }
+
+        return key;
+    }
+
+    public static Map<String, String> fromArgs(String[] args) {
+        final Map<String, String> map = new HashMap<>(args.length / 2);
+
+        int i = 0;
+        while (i < args.length) {
+            final String key = getKeyFromArgs(args, i);
+
+            if (key.isEmpty()) {
+                throw new IllegalArgumentException(
+                        "The input " + Arrays.toString(args) + " contains an empty argument");
+            }
+
+            i += 1; // try to find the value
+
+            if (i >= args.length) {
+                map.put(key, NO_VALUE_KEY);
+            } else if (NumberUtils.isNumber(args[i])) {
+                map.put(key, args[i]);
+                i += 1;
+            } else if (args[i].startsWith("--") || args[i].startsWith("-")) {
+                // the argument cannot be a negative number because we checked earlier
+                // -> the next argument is a parameter name
+                map.put(key, NO_VALUE_KEY);
+            } else {
+                map.put(key, args[i]);
+                i += 1;
+            }
+        }
+
+        return map;
+    }
+}
diff --git a/streampark-spark/streampark-spark-core/src/main/scala/org/apache/streampark/spark/core/util/SqlCommandParser.scala b/streampark-spark/streampark-spark-core/src/main/scala/org/apache/streampark/spark/core/util/SqlCommandParser.scala
new file mode 100644
index 0000000..df4e2d2
--- /dev/null
+++ b/streampark-spark/streampark-spark-core/src/main/scala/org/apache/streampark/spark/core/util/SqlCommandParser.scala
@@ -0,0 +1,614 @@
+/*
+ * 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.streampark.spark.core.util
+
+import org.apache.streampark.common.conf.ConfigKeys.PARAM_PREFIX
+import org.apache.streampark.common.enums.FlinkSqlValidationFailedType
+import org.apache.streampark.common.util.Logger
+
+import enumeratum.EnumEntry
+import org.apache.commons.lang3.StringUtils
+
+import java.lang.{Boolean => JavaBool}
+import java.util.Scanner
+import java.util.regex.{Matcher, Pattern}
+
+import scala.annotation.tailrec
+import scala.collection.{immutable, mutable}
+import scala.collection.mutable.ListBuffer
+import scala.util.control.Breaks.{break, breakable}
+
+object SqlCommandParser extends Logger {
+
+  def parseSQL(
+      sql: String,
+      validationCallback: SparkSqlValidationResult => Unit = null): List[SqlCommandCall] = {
+    val sqlEmptyError = "verify failed: spark sql cannot be empty."
+    require(StringUtils.isNotBlank(sql), sqlEmptyError)
+    val sqlSegments = SqlSplitter.splitSql(sql)
+    sqlSegments match {
+      case s if s.isEmpty =>
+        if (validationCallback != null) {
+          validationCallback(
+            SparkSqlValidationResult(
+              success = false,
+              failedType = FlinkSqlValidationFailedType.VERIFY_FAILED,
+              exception = sqlEmptyError))
+          null
+        } else {
+          throw new IllegalArgumentException(sqlEmptyError)
+        }
+      case segments =>
+        val calls = new ListBuffer[SqlCommandCall]
+        for (segment <- segments) {
+          parseLine(segment) match {
+            case Some(x) => calls += x
+            case _ =>
+              if (validationCallback != null) {
+                validationCallback(
+                  SparkSqlValidationResult(
+                    success = false,
+                    failedType = FlinkSqlValidationFailedType.UNSUPPORTED_SQL,
+                    lineStart = segment.start,
+                    lineEnd = segment.end,
+                    exception = s"unsupported sql",
+                    sql = segment.sql
+                  ))
+              } else {
+                throw new UnsupportedOperationException(s"unsupported sql: ${segment.sql}")
+              }
+          }
+        }
+
+        calls.toList match {
+          case c if c.isEmpty =>
+            if (validationCallback != null) {
+              validationCallback(
+                SparkSqlValidationResult(
+                  success = false,
+                  failedType = FlinkSqlValidationFailedType.VERIFY_FAILED,
+                  exception = "spark sql syntax error, no executable sql"))
+              null
+            } else {
+              throw new UnsupportedOperationException("spark sql syntax error, no executable sql")
+            }
+          case r => r
+        }
+    }
+  }
+
+  private[this] def parseLine(sqlSegment: SqlSegment): Option[SqlCommandCall] = {
+    val sqlCommand = SqlCommand.get(sqlSegment.sql.trim)
+    if (sqlCommand == null) None
+    else {
+      val matcher = sqlCommand.matcher
+      val groups = new Array[String](matcher.groupCount)
+      for (i <- groups.indices) {
+        groups(i) = matcher.group(i + 1)
+      }
+      sqlCommand
+        .converter(groups)
+        .map(
+          x => SqlCommandCall(sqlSegment.start, sqlSegment.end, sqlCommand, x, sqlSegment.sql.trim))
+    }
+  }
+
+}
+
+object Converters {
+  val NO_OPERANDS = (_: Array[String]) => Some(Array.empty[String])
+}
+
+sealed abstract class SqlCommand(
+    val name: String,
+    private val regex: String,
+    val converter: Array[String] => Option[Array[String]] = (x: Array[String]) =>
+      Some(Array[String](x.head)))
+  extends EnumEntry {
+  var matcher: Matcher = _
+
+  def matches(input: String): Boolean = {
+    if (StringUtils.isBlank(regex)) false
+    else {
+      val pattern = Pattern.compile(regex, Pattern.CASE_INSENSITIVE | Pattern.DOTALL)
+      matcher = pattern.matcher(input)
+      matcher.matches()
+    }
+  }
+}
+
+object SqlCommand extends enumeratum.Enum[SqlCommand] {
+
+  def get(stmt: String): SqlCommand = {
+    var cmd: SqlCommand = null
+    breakable {
+      this.values.foreach(
+        x => {
+          if (x.matches(stmt)) {
+            cmd = x
+            break()
+          }
+        })
+    }
+    cmd
+  }
+
+  val values: immutable.IndexedSeq[SqlCommand] = findValues
+
+  // ---- SELECT Statements--------------------------------------------------------------------------------------------------------------------------------
+  case object SELECT extends SqlCommand("select", "(SELECT\\s+.+)")
+
+  // ----CREATE Statements--------------------------------------------------------------------------------------------------------------------------------
+
+  /**
+   * <pre> CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [catalog_name.][db_name.]table_name ( {
+   * <physical_column_definition> | <metadata_column_definition> | <computed_column_definition> }[ ,
+   * ...n] [ <watermark_definition> ] [ <table_constraint> ][ , ...n] ) [COMMENT table_comment]
+   * [PARTITIONED BY (partition_column_name1, partition_column_name2, ...)] WITH (key1=val1,
+   * key2=val2, ...) [ LIKE source_table [( <like_options> )] ] </pre
+   */
+  case object CREATE_TABLE
+    extends SqlCommand("create table", "(CREATE\\s+(TEMPORARY\\s+|)TABLE\\s+.+)")
+
+  /** <pre> CREATE CATALOG catalog_name WITH (key1=val1, key2=val2, ...) </pre> */
+  case object CREATE_CATALOG extends SqlCommand("create catalog", "(CREATE\\s+CATALOG\\s+.+)")
+
+  /**
+   * <pre> CREATE DATABASE [IF NOT EXISTS] [catalog_name.]db_name<br> [COMMENT database_comment]<br>
+   * WITH (key1=val1, key2=val2, ...)<br> </pre>
+   */
+  case object CREATE_DATABASE extends SqlCommand("create database", "(CREATE\\s+DATABASE\\s+.+)")
+
+  /**
+   * <pre> CREATE [TEMPORARY] VIEW [IF NOT EXISTS] [catalog_name.][db_name.]view_name [( columnName
+   * [, columnName ]* )] [COMMENT view_comment] AS query_expression< </pre
+   */
+  case object CREATE_VIEW
+    extends SqlCommand(
+      "create view",
+      "(CREATE\\s+(TEMPORARY\\s+|)VIEW\\s+(IF\\s+NOT\\s+EXISTS\\s+|)(\\S+)\\s+AS\\s+SELECT\\s+.+)")
+
+  /**
+   * <pre> CREATE [TEMPORARY|TEMPORARY SYSTEM] FUNCTION [IF NOT EXISTS]
+   * [catalog_name.][db_name.]function_name AS identifier [LANGUAGE JAVA|SCALA|PYTHON] </pre
+   */
+  case object CREATE_FUNCTION
+    extends SqlCommand(
+      "create function",
+      "(CREATE\\s+(TEMPORARY\\s+|TEMPORARY\\s+SYSTEM\\s+|)FUNCTION\\s+(IF\\s+NOT\\s+EXISTS\\s+|)(\\S+)\\s+AS\\s+.*)")
+
+  // ----DROP Statements--------------------------------------------------------------------------------------------------------------------------------
+
+  /**
+   * <pre> DROP statements are used to remove a catalog with the given catalog name or to remove a
+   * registered table/view/function from the current or specified Catalog.
+   *
+   * Spark SQL supports the following DROP statements for now: * DROP CATALOG * DROP TABLE * DROP
+   * DATABASE * DROP VIEW * DROP FUNCTION </pre>
+   */
+
+  /** <strong>DROP CATALOG [IF EXISTS] catalog_name</strong> */
+  case object DROP_CATALOG extends SqlCommand("drop catalog", "(DROP\\s+CATALOG\\s+.+)")
+
+  /** <strong>DROP [TEMPORARY] TABLE [IF EXISTS] [catalog_name.][db_name.]table_name</strong> */
+  case object DROP_TABLE extends SqlCommand("drop table", "(DROP\\s+(TEMPORARY\\s+|)TABLE\\s+.+)")
+
+  /** <strong>DROP DATABASE [IF EXISTS] [catalog_name.]db_name [ (RESTRICT | CASCADE) ]</strong> */
+  case object DROP_DATABASE extends SqlCommand("drop database", "(DROP\\s+DATABASE\\s+.+)")
+
+  /** <strong>DROP [TEMPORARY] VIEW  [IF EXISTS] [catalog_name.][db_name.]view_name</strong> */
+  case object DROP_VIEW extends SqlCommand("drop view", "(DROP\\s+(TEMPORARY\\s+|)VIEW\\s+.+)")
+
+  /**
+   * <strong>DROP [TEMPORARY|TEMPORARY SYSTEM] FUNCTION [IF EXISTS]
+   * [catalog_name.][db_name.]function_name</strong>
+   */
+  case object DROP_FUNCTION
+    extends SqlCommand(
+      "drop function",
+      "(DROP\\s+(TEMPORARY\\s+|TEMPORARY\\s+SYSTEM\\s+|)FUNCTION\\s+.+)")
+
+  // ----ALTER Statements--------------------------------------------------------------------------------------------------------------------------------
+
+  /**
+   * <strong>ALTER TABLE [catalog_name.][db_name.]table_name RENAME TO new_table_name</strong>
+   *
+   * <strong>ALTER TABLE [catalog_name.][db_name.]table_name SET (key1=val1, key2=val2,
+   * ...)</strong>
+   */
+  case object ALTER_TABLE extends SqlCommand("alter table", "(ALTER\\s+TABLE\\s+.+)")
+
+  /**
+   * <strong>ALTER VIEW [catalog_name.][db_name.]view_name RENAME TO new_view_name</strong>
+   *
+   * <strong>ALTER VIEW [catalog_name.][db_name.]view_name AS new_query_expression</strong>
+   */
+  case object ALTER_VIEW extends SqlCommand("alter view", "(ALTER\\s+VIEW\\s+.+)")
+
+  /** <strong>ALTER DATABASE [catalog_name.]db_name SET (key1=val1, key2=val2, ...)</strong> */
+  case object ALTER_DATABASE extends SqlCommand("alter database", "(ALTER\\s+DATABASE\\s+.+)")
+
+  /**
+   * <strong> ALTER [TEMPORARY|TEMPORARY SYSTEM] FUNCTION [IF EXISTS]
+   * [catalog_name.][db_name.]function_name AS identifier [LANGUAGE JAVA|SCALA|PYTHON] </strong>
+   */
+  case object ALTER_FUNCTION
+    extends SqlCommand(
+      "alter function",
+      "(ALTER\\s+(TEMPORARY\\s+|TEMPORARY\\s+SYSTEM\\s+|)FUNCTION\\s+.+)")
+
+  // ---- INSERT Statement--------------------------------------------------------------------------------------------------------------------------------
+
+  /**
+   * INSERT { INTO | OVERWRITE } [catalog_name.][db_name.]table_name [PARTITION part_spec]
+   * [column_list] select_statement INSERT { INTO | OVERWRITE } [catalog_name.][db_name.]table_name
+   * VALUES values_row [, values_row ...]
+   */
+  case object INSERT extends SqlCommand("insert", "(INSERT\\s+(INTO|OVERWRITE)\\s+.+)")
+
+  // ---- DESCRIBE Statement--------------------------------------------------------------------------------------------------------------------------------
+
+  /** { DESCRIBE | DESC } [catalog_name.][db_name.]table_name */
+  case object DESC extends SqlCommand("desc", "(DESC\\s+.+)")
+
+  /** { DESCRIBE | DESC } [catalog_name.][db_name.]table_name */
+  case object DESCRIBE extends SqlCommand("describe", "(DESCRIBE\\s+.+)")
+
+  // ---- EXPLAIN Statement--------------------------------------------------------------------------------------------------------------------------------
+
+  case object EXPLAIN extends SqlCommand("explain", "(EXPLAIN\\s+.+)")
+
+  // ---- USE Statements--------------------------------------------------------------------------------------------------------------------------------
+
+  /** USE CATALOG catalog_name */
+  case object USE_CATALOG extends SqlCommand("use catalog", "(USE\\s+CATALOG\\s+.+)")
+
+  /** USE MODULES module_name1[, module_name2, ...] */
+  case object USE_MODULES extends SqlCommand("use modules", "(USE\\s+MODULES\\s+.+)")
+
+  /** USE [catalog_name.]database_name */
+  case object USE_DATABASE extends SqlCommand("use database", "(USE\\s+(?!(CATALOG|MODULES)).+)")
+
+  // ----SHOW Statements--------------------------------------------------------------------------------------------------------------------------------
+
+  /** SHOW CATALOGS */
+  case object SHOW_CATALOGS extends SqlCommand("show catalogs", "(SHOW\\s+CATALOGS\\s*)")
+
+  /** SHOW CURRENT CATALOG */
+  case object SHOW_CURRENT_CATALOG
+    extends SqlCommand("show current catalog", "(SHOW\\s+CURRENT\\s+CATALOG\\s*)")
+
+  /** SHOW DATABASES */
+  case object SHOW_DATABASES extends SqlCommand("show databases", "(SHOW\\s+DATABASES\\s*)")
+
+  /** SHOW CURRENT DATABASE */
+  case object SHOW_CURRENT_DATABASE
+    extends SqlCommand("show current database", "(SHOW\\s+CURRENT\\s+DATABASE\\s*)")
+
+  case object SHOW_TABLES extends SqlCommand("show tables", "(SHOW\\s+TABLES.*)")
+
+  case object SHOW_CREATE_TABLE
+    extends SqlCommand("show create table", "(SHOW\\s+CREATE\\s+TABLE\\s+.+)")
+
+  case object SHOW_COLUMNS extends SqlCommand("show columns", "(SHOW\\s+COLUMNS\\s+.+)")
+
+  /** SHOW VIEWS */
+  case object SHOW_VIEWS extends SqlCommand("show views", "(SHOW\\s+VIEWS\\s*)")
+
+  /** SHOW CREATE VIEW */
+  case object SHOW_CREATE_VIEW
+    extends SqlCommand("show create view", "(SHOW\\s+CREATE\\s+VIEW\\s+.+)")
+
+  /** SHOW [USER] FUNCTIONS */
+  case object SHOW_FUNCTIONS
+    extends SqlCommand("show functions", "(SHOW\\s+(USER\\s+|)FUNCTIONS\\s*)")
+
+  /** SHOW [FULL] MODULES */
+  case object SHOW_MODULES extends SqlCommand("show modules", "(SHOW\\s+(FULL\\s+|)MODULES\\s*)")
+
+  // ----LOAD Statements--------------------------------------------------------------------------------------------------------------------------------
+
+  /** LOAD MODULE module_name [WITH ('key1' = 'val1', 'key2' = 'val2', ...)] */
+  case object LOAD_MODULE extends SqlCommand("load module", "(LOAD\\s+MODULE\\s+.+)")
+
+  // ----UNLOAD Statements--------------------------------------------------------------------------------------------------------------------------------
+
+  /** UNLOAD MODULE module_name */
+  case object UNLOAD_MODULE extends SqlCommand("unload module", "(UNLOAD\\s+MODULE\\s+.+)")
+
+  // ----SET Statements--------------------------------------------------------------------------------------------------------------------------------
+
+  /** SET ('key' = 'value') */
+  case object SET
+    extends SqlCommand(
+      "set",
+      "SET(\\s+(\\S+)\\s*=(.*))?",
+      {
+        case a if a.length < 3 => None
+        case a if a.head == null => Some(Array[String](cleanUp(a.head)))
+        case a => Some(Array[String](cleanUp(a(1)), cleanUp(a(2))))
+      })
+
+  // ----RESET Statements--------------------------------------------------------------------------------------------------------------------------------
+
+  /** RESET ('key') */
+  case object RESET extends SqlCommand("reset", "RESET\\s+'(.*)'")
+
+  /** RESET */
+  case object RESET_ALL extends SqlCommand("reset all", "RESET", _ => Some(Array[String]("ALL")))
+
+  case object DELETE extends SqlCommand("delete", "(DELETE\\s+FROM\\s+.+)")
+
+  case object UPDATE extends SqlCommand("update", "(UPDATE\\s+.+)")
+
+  private[this] def cleanUp(sql: String): String = sql.trim.replaceAll("^(['\"])|(['\"])$", "")
+
+}
+
+/** Call of SQL command with operands and command type. */
+case class SqlCommandCall(
+    lineStart: Int,
+    lineEnd: Int,
+    command: SqlCommand,
+    operands: Array[String],
+    originSql: String) {}
+
+case class SparkSqlValidationResult(
+    success: JavaBool = true,
+    failedType: FlinkSqlValidationFailedType = null,
+    lineStart: Int = 0,
+    lineEnd: Int = 0,
+    errorLine: Int = 0,
+    errorColumn: Int = 0,
+    sql: String = null,
+    exception: String = null)
+
+case class SqlSegment(start: Int, end: Int, sql: String)
+
+object SqlSplitter {
+
+  private lazy val singleLineCommentPrefixList = Set[String](PARAM_PREFIX)
+
+  /**
+   * Split whole text into multiple sql statements. Two Steps: Step 1, split the whole text into
+   * multiple sql statements. Step 2, refine the results. Replace the preceding sql statements with
+   * empty lines, so that we can get the correct line number in the parsing error message. e.g:
+   * select a from table_1; select a from table_2; select a from table_3; The above text will be
+   * splitted into: sql_1: select a from table_1 sql_2: \nselect a from table_2 sql_3: \n\nselect a
+   * from table_3
+   *
+   * @param sql
+   * @return
+   */
+  def splitSql(sql: String): List[SqlSegment] = {
+    val queries = ListBuffer[String]()
+    val lastIndex = if (StringUtils.isNotBlank(sql)) sql.length - 1 else 0
+    var query = new mutable.StringBuilder
+
+    var multiLineComment = false
+    var singleLineComment = false
+    var singleQuoteString = false
+    var doubleQuoteString = false
+    var lineNum: Int = 0
+    val lineNumMap = new collection.mutable.HashMap[Int, (Int, Int)]()
+
+    // Whether each line of the record is empty. If it is empty, it is false. If it is not empty, it is true
+    val lineDescriptor = {
+      val scanner = new Scanner(sql)
+      val descriptor = new collection.mutable.HashMap[Int, Boolean]
+      var lineNumber = 0
+      var startComment = false
+      var hasComment = false
+
+      while (scanner.hasNextLine) {
+        lineNumber += 1
+        val line = scanner.nextLine().trim
+        val nonEmpty = StringUtils.isNotBlank(line) && !line.startsWith(PARAM_PREFIX)
+        if (line.startsWith("/*")) {
+          startComment = true
+          hasComment = true
+        }
+
+        descriptor += lineNumber -> (nonEmpty && !hasComment)
+
+        if (startComment && line.endsWith("*/")) {
+          startComment = false
+          hasComment = false
+        }
+      }
+      descriptor
+    }
+
+    @tailrec
+    def findStartLine(num: Int): Int =
+      if (num >= lineDescriptor.size || lineDescriptor(num)) num else findStartLine(num + 1)
+
+    def markLineNumber(): Unit = {
+      val line = lineNum + 1
+      if (lineNumMap.isEmpty) {
+        lineNumMap += (0 -> (findStartLine(1) -> line))
+      } else {
+        val index = lineNumMap.size
+        val start = lineNumMap(lineNumMap.size - 1)._2 + 1
+        lineNumMap += (index -> (findStartLine(start) -> line))
+      }
+    }
+
+    for (idx <- 0 until sql.length) {
+
+      if (sql.charAt(idx) == '\n') lineNum += 1
+
+      breakable {
+        val ch = sql.charAt(idx)
+
+        // end of single line comment
+        if (singleLineComment && (ch == '\n')) {
+          singleLineComment = false
+          query += ch
+          if (idx == lastIndex && query.toString.trim.nonEmpty) {
+            // add query when it is the end of sql.
+            queries += query.toString
+          }
+          break()
+        }
+
+        // end of multiple line comment
+        if (
+          multiLineComment && (idx - 1) >= 0 && sql.charAt(idx - 1) == '/'
+          && (idx - 2) >= 0 && sql.charAt(idx - 2) == '*'
+        ) {
+          multiLineComment = false
+        }
+
+        // single quote start or end mark
+        if (ch == '\'' && !(singleLineComment || multiLineComment)) {
+          if (singleQuoteString) {
+            singleQuoteString = false
+          } else if (!doubleQuoteString) {
+            singleQuoteString = true
+          }
+        }
+
+        // double quote start or end mark
+        if (ch == '"' && !(singleLineComment || multiLineComment)) {
+          if (doubleQuoteString && idx > 0) {
+            doubleQuoteString = false
+          } else if (!singleQuoteString) {
+            doubleQuoteString = true
+          }
+        }
+
+        // single line comment or multiple line comment start mark
+        if (
+          !singleQuoteString && !doubleQuoteString && !multiLineComment && !singleLineComment && idx < lastIndex
+        ) {
+          if (isSingleLineComment(sql.charAt(idx), sql.charAt(idx + 1))) {
+            singleLineComment = true
+          } else if (
+            sql.charAt(idx) == '/' && sql.length > (idx + 2)
+            && sql.charAt(idx + 1) == '*' && sql.charAt(idx + 2) != '+'
+          ) {
+            multiLineComment = true
+          }
+        }
+
+        if (
+          ch == ';' && !singleQuoteString && !doubleQuoteString && !multiLineComment && !singleLineComment
+        ) {
+          markLineNumber()
+          // meet the end of semicolon
+          if (query.toString.trim.nonEmpty) {
+            queries += query.toString
+            query = new mutable.StringBuilder
+          }
+        } else if (idx == lastIndex) {
+          markLineNumber()
+
+          // meet the last character
+          if (!singleLineComment && !multiLineComment) {
+            query += ch
+          }
+
+          if (query.toString.trim.nonEmpty) {
+            queries += query.toString
+            query = new mutable.StringBuilder
+          }
+        } else if (!singleLineComment && !multiLineComment) {
+          // normal case, not in single line comment and not in multiple line comment
+          query += ch
+        } else if (ch == '\n') {
+          query += ch
+        }
+      }
+    }
+
+    val refinedQueries = new collection.mutable.HashMap[Int, String]()
+    for (i <- queries.indices) {
+      val currStatement = queries(i)
+      if (isSingleLineComment(currStatement) || isMultipleLineComment(currStatement)) {
+        // transform comment line as blank lines
+        if (refinedQueries.nonEmpty) {
+          val lastRefinedQuery = refinedQueries.last
+          refinedQueries(refinedQueries.size - 1) =
+            lastRefinedQuery + extractLineBreaks(currStatement)
+        }
+      } else {
+        var linesPlaceholder = ""
+        if (i > 0) {
+          linesPlaceholder = extractLineBreaks(refinedQueries(i - 1))
+        }
+        // add some blank lines before the statement to keep the original line number
+        val refinedQuery = linesPlaceholder + currStatement
+        refinedQueries += refinedQueries.size -> refinedQuery
+      }
+    }
+
+    val set = new ListBuffer[SqlSegment]
+    refinedQueries.foreach(
+      x => {
+        val line = lineNumMap(x._1)
+        set += SqlSegment(line._1, line._2, x._2)
+      })
+    set.toList.sortWith((a, b) => a.start < b.start)
+  }
+
+  /**
+   * extract line breaks
+   *
+   * @param text
+   * @return
+   */
+  private[this] def extractLineBreaks(text: String): String = {
+    val builder = new mutable.StringBuilder
+    for (i <- 0 until text.length) {
+      if (text.charAt(i) == '\n') {
+        builder.append('\n')
+      }
+    }
+    builder.toString
+  }
+
+  private[this] def isSingleLineComment(text: String) = text.trim.startsWith(PARAM_PREFIX)
+
+  private[this] def isMultipleLineComment(text: String) =
+    text.trim.startsWith("/*") && text.trim.endsWith("*/")
+
+  /**
+   * check single-line comment
+   *
+   * @param curChar
+   * @param nextChar
+   * @return
+   */
+  private[this] def isSingleLineComment(curChar: Char, nextChar: Char): Boolean = {
+    var flag = false
+    for (singleCommentPrefix <- singleLineCommentPrefixList) {
+      if (singleCommentPrefix.length == 1) {
+        if (curChar == singleCommentPrefix.charAt(0)) {
+          flag = true
+        }
+      }
+      if (singleCommentPrefix.length == 2) {
+        if (curChar == singleCommentPrefix.charAt(0) && nextChar == singleCommentPrefix.charAt(1)) {
+          flag = true
+        }
+      }
+    }
+    flag
+  }
+
+}
diff --git a/streampark-spark/streampark-spark-sqlclient/pom.xml b/streampark-spark/streampark-spark-sqlclient/pom.xml
new file mode 100644
index 0000000..d7e951e
--- /dev/null
+++ b/streampark-spark/streampark-spark-sqlclient/pom.xml
@@ -0,0 +1,107 @@
+<?xml version="1.0" encoding="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.
+  -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.streampark</groupId>
+        <artifactId>streampark-spark</artifactId>
+        <version>2.2.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>streampark-spark-sqlclient_${scala.binary.version}</artifactId>
+    <name>StreamPark : Spark SQL Client</name>
+
+    <dependencies>
+
+        <dependency>
+            <groupId>com.beachape</groupId>
+            <artifactId>enumeratum_${scala.binary.version}</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.streampark</groupId>
+            <artifactId>streampark-spark-core_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+    </dependencies>
+
+    <build>
+        <plugins>
+
+            <plugin>
+                <groupId>net.alchim31.maven</groupId>
+                <artifactId>scala-maven-plugin</artifactId>
+            </plugin>
+
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-shade-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>shade</goal>
+                        </goals>
+                    </execution>
+                </executions>
+                <configuration>
+                    <createDependencyReducedPom>true</createDependencyReducedPom>
+                    <dependencyReducedPomLocation>${project.basedir}/target/dependency-reduced-pom.xml</dependencyReducedPomLocation>
+                    <transformers>
+                        <transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
+                            <mainClass>org.apache.streampark.spark.cli.SqlClient</mainClass>
+                        </transformer>
+                    </transformers>
+                    <artifactSet>
+                        <includes>
+                            <include>*:*</include>
+                        </includes>
+                        <excludes>
+                            <exclude>org.apache.spark:force-shading</exclude>
+                            <exclude>com.google.code.findbugs:jsr305</exclude>
+                            <exclude>org.slf4j:*</exclude>
+                            <exclude>log4j:*</exclude>
+                        </excludes>
+                    </artifactSet>
+                    <filters>
+                        <filter>
+                            <artifact>*:*</artifact>
+                            <excludes>
+                                <exclude>META-INF/*.SF</exclude>
+                                <exclude>META-INF/*.DSA</exclude>
+                                <exclude>META-INF/*.RSA</exclude>
+                                <exclude>Log4j-*.*</exclude>
+                            </excludes>
+                        </filter>
+                    </filters>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+
+    <profiles>
+        <profile>
+            <id>apache-release</id>
+            <properties>
+                <maven.deploy.skip>true</maven.deploy.skip>
+            </properties>
+        </profile>
+    </profiles>
+
+</project>
diff --git a/streampark-spark/streampark-spark-sqlclient/src/main/scala/org/apache/streampark/spark/cli/SqlClient.scala b/streampark-spark/streampark-spark-sqlclient/src/main/scala/org/apache/streampark/spark/cli/SqlClient.scala
new file mode 100644
index 0000000..c481126
--- /dev/null
+++ b/streampark-spark/streampark-spark-sqlclient/src/main/scala/org/apache/streampark/spark/cli/SqlClient.scala
@@ -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.streampark.spark.cli
+
+import org.apache.streampark.common.conf.ConfigKeys.KEY_FLINK_SQL
+import org.apache.streampark.common.util.DeflaterUtils
+import org.apache.streampark.spark.core.{SparkBatch, SparkStreaming}
+import org.apache.streampark.spark.core.util.{ParameterTool, SqlCommand, SqlCommandParser}
+
+import org.apache.commons.lang3.StringUtils
+import org.apache.spark.sql.DataFrame
+
+import scala.collection.mutable.ArrayBuffer
+import scala.language.implicitConversions
+import scala.util.{Failure, Success, Try}
+
+object SqlClient extends App {
+
+  val arguments = ArrayBuffer(args: _*)
+
+  private[this] val parameterTool = ParameterTool.fromArgs(args)
+
+  private[this] val sparkSql = {
+    val sql = parameterTool.get(KEY_FLINK_SQL())
+    require(StringUtils.isNotBlank(sql), "Usage: spark sql cannot be null")
+    Try(DeflaterUtils.unzipString(sql)) match {
+      case Success(value) => value
+      case Failure(_) =>
+        throw new IllegalArgumentException("Usage: spark sql is invalid or null, please check")
+    }
+  }
+
+  private[this] val sets = SqlCommandParser.parseSQL(sparkSql).filter(_.command == SqlCommand.SET)
+
+  private[this] val defaultMode = "BATCH"
+
+  private[this] val mode = defaultMode
+
+  mode match {
+    case "STREAMING" | "AUTOMATIC" => StreamSqlApp.main(arguments.toArray)
+    case "BATCH" => BatchSqlApp.main(arguments.toArray)
+    case _ =>
+      throw new IllegalArgumentException(
+        "Usage: runtime execution-mode invalid, optional [STREAMING|BATCH|AUTOMATIC]")
+  }
+
+  private[this] object BatchSqlApp extends SparkBatch {
+    override def handle(sql: String): DataFrame = super.handle(sql)
+  }
+
+  private[this] object StreamSqlApp extends SparkStreaming {
+    override def handle(sql: String): DataFrame = super.handle(sql)
+  }
+
+}