You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@seatunnel.apache.org by ki...@apache.org on 2022/07/22 10:24:48 UTC

[incubator-seatunnel] branch st-engine updated: Merge from dev to st-engine (#2243)

This is an automated email from the ASF dual-hosted git repository.

kirs pushed a commit to branch st-engine
in repository https://gitbox.apache.org/repos/asf/incubator-seatunnel.git


The following commit(s) were added to refs/heads/st-engine by this push:
     new 41e530afd Merge from dev to st-engine (#2243)
41e530afd is described below

commit 41e530afd59d3e72e0517c2f3c2287c86d935f59
Author: Kirs <ki...@apache.org>
AuthorDate: Fri Jul 22 18:24:42 2022 +0800

    Merge from dev to st-engine (#2243)
    
    * Delete a repeated dependency libary. (#2180)
    
    Signed-off-by: root <l-...@localhost.localdomain>
    
    Co-authored-by: root <l-...@localhost.localdomain>
    
    * update flinkCommand to sparkCommand in spark example (#2184)
    
    * update doc about module desc to keep consistent with the real module name (#2185)
    
    * [Connector-V2] Add Hive sink connector v2 (#2158)
    
    * tmp commit
    
    * add hadoop2 and hadoop3 shade jar
    
    * add hadoop2 and hadoop3 shade jar
    
    * add license head
    
    * change know denpendencies
    
    * tmp commit
    
    * tmp commit
    
    * change hadoop dependency scope to provide
    
    * back pom
    
    * fix checkstyle
    
    * add example
    
    * fix example bug
    
    * remove file connector from example and e2e because hadoop2 can not compile with jdk11
    
    * no need jdk8 and jdk11 profile because we don't use hadoop shade jar
    
    * change hadoop jar dependency scope to provided
    
    * back
    
    * file connector can not build in jdk11
    
    * drop hadoop shade
    
    * add gitignore item
    
    * add hadoop and local file sink
    
    * fix pom error
    
    * fix pom error
    
    * fix pom error
    
    * implement new interface
    
    * fix UT error
    
    * fix e2e error
    
    * update build timeout from 30min to 40min
    
    * fix e2e error
    
    * remove auto service
    
    * fix e2e error
    
    * fix e2e error
    
    * fix e2e error
    
    * found e2e error
    
    * fix e2e error
    
    * fix e2e error
    
    * fix e2e error
    
    * merge from upstream
    
    * merge from upstream
    
    * merge from upstream
    
    * merge from upstream
    
    * merge from upstream
    
    * add mvn jvm option
    
    * add mvn jvm option
    
    * add license
    
    * add licnese
    
    * add licnese
    
    * fix dependency
    
    * fix build jvm oom
    
    * fix build jvm oom
    
    * fix build jvm oom
    
    * fix dependency
    
    * fix dependency
    
    * fix e2e error
    
    * add codeql check timeout from 30min to 60min
    
    * merge from dev
    
    * merge from dev
    
    * fix ci error
    
    * fix checkstyle
    
    * fix ci
    
    * fix ci
    
    * aa
    
    * aa
    
    * aa
    
    * add .idea
    
    * del .idea
    
    * del .idea
    
    * del .idea
    
    * del .idea
    
    * remove no use license
    
    * remove no use before and after method in test
    
    * fix license; remove dependency
    
    * fix review
    
    * fix build order
    
    * fix license
    
    * fix license
    
    * fix review
    
    * fix review
    
    * fix review
    
    * fix review
    
    * fix review
    
    * fix review
    
    * fix review
    
    * fix review
    
    * fix review
    
    * add code-analysys timeout to 120
    
    * retry ci
    
    * update license and remove no use jar from LICENSE file
    
    * retry ci
    
    * add hive sink
    
    * add hive sink connector doc
    
    * add hive sink connector doc
    
    * fix checkstyle error.
    
    * fix bug
    
    * tmp
    
    * fix hive shade error
    
    * fix hive shade error
    
    * fix commit bug
    
    * optimaze doc
    
    * optimaze doc
    
    * optimize doc
    
    * optimize code
    
    * [Feat][UI] Add login page. (#2183)
    
    * [bug]fix  commandArgs  -t(--check)  conflict  with flink deployment t… (#2174)
    
    * [bug]fix  commandArgs  -t(--check)  conflict  with flink deployment target
    
    * [bug]fix  commandArgs  -t(--check)  conflict  with flink deployment target
    
    * [Bug][spark-connector-v2-example] fix the bug of no class found. (#2191) (#2192)
    
    * [Bug][spark-connector-v2-example] fix the bug of no class found. (#2191)
    
    * add the janino dependency in pom
    
    * [Bug][spark-connector-v2-example] remove janino dependency in main pom and add it to connector[v2]-hive (#2191)
    
    * [Bug][spark-connector-v2-example] add janino-3.0.9.jar in known-dependencies.txt to fix dependency license error (#2191)
    
    * update the condition to 1 = 0 about get table operation (#2186)
    
    * [Docs] Add connectors-v2 to docs item (#2187)
    
    * [Feat][UI] Add dashboard layout. (#2198)
    
    * [checkstyle] Improved validation scope of MagicNumber (#2194)
    
    * [Bug][Connector]Hudi Source loads the data twice
    
    * add unknown exception message (#2204)
    
    * [Bug] [seatunnel-api-flink] Connectors dependencies repeat additions (#2207)
    
    * [Bug] [connector-v2] When outputting data to clickhouse, a ClassCastException was encountered
    
    * [Bug] [seatunnel-api-flink] Connectors dependencies repeat additions
    
    * [Bug][Script]Fix the problem that the help command is invalid
    
    * [Fix][CI] Add remove jar from /tmp/seatunnel-dependencies before run
    
    * [Feat][UI] Add dashboard default router. (#2216)
    
    * [Feat][UI] Add the header component in the dashboard layout. (#2218)
    
    * [Core][Starter] Change jar connector load logic (#2193)
    
    * [Docs]Fix Flink engine version requirements (#2220)
    
    Flink 1.13.6 version is compatible with 1.12, but not applicable to below 1.12
    
    * [Feat][UI] Add the setting dropdown in the dashboard layout. (#2225)
    
    * [Feat][UI] Add the user dropdown in the dashboard layout. (#2228)
    
    * [Bug][hive-connector-v2] Resolve the schema inconsistency bug (#2229) (#2230)
    
    * [doc] Correct v2 connector avoid duplicate slug (#2231)
    
    Currently, url https://seatunnel.apache.org/docs/category/source
    will expand two parent sidebar with both source and source-v2.
    This is because we're using same slug in our sidebars.js.
    
    * [Build]Optimize license check (#2232)
    
    * [Core][Starter] Fix connector v2 can't deserialize on spark (#2221)
    
    * [Core][Starter] Fix connector v2 can't deserialize on spark
    
    * [Core][Starter] Add SerializationUtils Unit Test
    
    * [Core][Starter] Add SerializationUtils Unit Test
    
    * [Core][Flink] Fixed FlinkEnvironment registerPlugin logic both old and new api
    
    * [Bug][connector-hive] filter '_SUCCESS' file in file list (#2235) (#2236)
    
    * StateT of SeaTunnelSource should extend `Serializable` (#2214)
    
    * [Improvement][core] StateT of SeaTunnelSource should extend `Serializable`
    ,so that `org.apache.seatunnel.api.source.SeaTunnelSource.getEnumeratorStateSerializer` can support a default implementation.
    This will be useful to each SeaTunnelSource subclass implementation.
    
    * repetitive dependency
    
    repetitive dependency
    
    * [Improvement][connector-v2] postgre jar should be contained in container like mysql-java, so it should be  provided, not compile
    
    * [Improvement][connector-v2] remove the code block in the implementation class to keep code clean.
    
    * [Improvement][connector-v2] remove unused import
    
    * [Improvement][connector-v2] modify import order
    
    Co-authored-by: bjyflihongyu <li...@jd.com>
    
    * [Feat][UI] Add the table in the user manage. (#2234)
    
    * Merge dev to st-engine branch
    
    Co-authored-by: l-shen <li...@cmss.chinamobile.com>
    Co-authored-by: root <l-...@localhost.localdomain>
    Co-authored-by: Xiao Zhao <49...@users.noreply.github.com>
    Co-authored-by: Eric <ga...@gmail.com>
    Co-authored-by: songjianet <17...@qq.com>
    Co-authored-by: sandyfog <15...@qq.com>
    Co-authored-by: TyrantLucifer <Ty...@gmail.com>
    Co-authored-by: Zongwen Li <zo...@gmail.com>
    Co-authored-by: superzhang0929 <45...@users.noreply.github.com>
    Co-authored-by: Kerwin <37...@users.noreply.github.com>
    Co-authored-by: gaara <85...@users.noreply.github.com>
    Co-authored-by: lvlv <40...@users.noreply.github.com>
    Co-authored-by: Hisoka <fa...@qq.com>
    Co-authored-by: Jiajie Zhong <zh...@gmail.com>
    Co-authored-by: Jared Li <lh...@gmail.com>
    Co-authored-by: bjyflihongyu <li...@jd.com>
---
 .github/workflows/backend.yml                      |   1 -
 docs/en/connector-v2/sink/File.mdx                 |   2 +
 docs/en/connector-v2/sink/Hive.md                  |  62 +++++
 docs/en/connector-v2/source/Hudi.md                |   2 +
 docs/en/start/local.mdx                            |   2 +-
 docs/sidebars.js                                   |  45 ++++
 pom.xml                                            |  11 -
 .../seatunnel/api/source/SeaTunnelSource.java      |   6 +-
 .../seatunnel/api/table/connector/TableSource.java |   4 +-
 .../seatunnel/api/table/factory/FactoryUtil.java   |   3 +-
 .../api/table/factory/TableSourceFactory.java      |   4 +-
 .../seatunnel/api/table/type/SeaTunnelRow.java     |   1 -
 .../seatunnel/api/table/type/SeaTunnelRowType.java |   1 -
 .../apache/seatunnel/flink/FlinkEnvironment.java   |  40 +--
 .../seatunnel/common/utils/ReflectionUtils.java    |  13 +-
 .../common/utils/SerializationException.java       |  68 +++++
 .../seatunnel/common/utils/SerializationUtils.java |  38 ++-
 .../common/utils/ReflectionUtilsTest.java          |  22 +-
 .../common/utils/SerializationUtilsTest.java       |  74 ++++++
 .../common/utils/VariablesSubstituteTest.java      |   1 -
 seatunnel-connectors-v2/README.md                  |   2 +-
 seatunnel-connectors-v2/README.zh.md               |   2 +-
 .../flink/assertion/AssertExecutorTest.java        |   1 -
 .../flink/assertion/rule/AssertRuleParserTest.java |   1 -
 .../clickhouse/source/ClickhouseSource.java        |   7 -
 .../common/source/AbstractSingleSplitSource.java   |   5 -
 ...TextFileConfig.java => BaseTextFileConfig.java} |  10 +-
 .../seatunnel/file/sink/AbstractFileSink.java      |   3 +-
 .../file/sink/FileAggregatedCommitInfo.java        |   3 +
 .../seatunnel/file/sink/FileCommitInfo.java        |   3 +
 .../file/sink/FileSinkAggregatedCommitter.java     |   9 +-
 .../file/sink/FileSinkWriterWithTransaction.java   | 160 ++++++++++++
 .../file/sink/config/TextFileSinkConfig.java       |  10 +-
 .../file/sink/transaction/Transaction.java         |   5 +-
 .../writer/AbstractTransactionStateFileWriter.java |  14 +-
 .../writer/FileSinkPartitionDirNameGenerator.java  |  18 +-
 .../sink/writer/PartitionDirNameGenerator.java     |   4 +-
 .../TestFileSinkPartitionDirNameGenerator.java     |   6 +-
 .../sink/hdfs/FileSinkAggregatedCommitterTest.java |  36 ++-
 .../TestHdfsTxtTransactionStateFileWriter.java     |  10 +
 .../local/FileSinkAggregatedCommitterTest.java     |  39 ++-
 .../TestLocalTxtTransactionStateFileWriter.java    |  10 +
 seatunnel-connectors-v2/connector-hive/pom.xml     |  28 +-
 .../connectors/seatunnel/hive/config/Constant.java |  10 +-
 .../hive/sink/HiveAggregatedCommitInfo.java        |  14 +-
 .../seatunnel/hive/sink/HiveCommitInfo.java        |  16 +-
 .../connectors/seatunnel/hive/sink/HiveSink.java   |  56 +++-
 .../hive/sink/HiveSinkAggregatedCommitter.java     |  93 +++++--
 .../seatunnel/hive/sink/HiveSinkConfig.java        | 148 +++++++----
 .../seatunnel/hive/sink/HiveSinkState.java         |   3 +-
 .../seatunnel/hive/sink/HiveSinkWriter.java        | 137 +++++++---
 .../hive/sink/file/writer/AbstractFileWriter.java  | 155 -----------
 .../hive/sink/file/writer/FileWriter.java          |  49 ----
 .../hive/sink/file/writer/HdfsTxtFileWriter.java   | 151 -----------
 .../seatunnel/hive/sink/file/writer/HdfsUtils.java |  96 -------
 .../seatunnel/hive/source/HiveSource.java          |   6 -
 .../file/reader/format/AbstractReadStrategy.java   |   6 +-
 .../file/reader/format/TextReadStrategy.java       |   2 +-
 .../seatunnel/hive/utils/HiveMetaStoreProxy.java   |  52 ++++
 .../seatunnel/hive/sink/TestHiveSinkConfig.java    |  53 ++++
 .../src/test/resources/fakesource_to_hive.conf     |  23 +-
 seatunnel-connectors-v2/connector-hudi/pom.xml     |   6 +-
 .../seatunnel/hudi/source/HudiSource.java          |   6 -
 seatunnel-connectors-v2/connector-jdbc/pom.xml     |   1 +
 .../seatunnel/jdbc/catalog/MySqlCatalog.java       |   2 +-
 .../seatunnel/jdbc/source/JdbcSource.java          |   6 -
 .../seatunnel/kafka/source/KafkaSource.java        |   7 -
 .../seatunnel/pulsar/source/PulsarSource.java      |   6 -
 seatunnel-connectors-v2/pom.xml                    |  11 +
 .../flink/assertion/AssertExecutorTest.java        |   1 -
 .../flink/assertion/rule/AssertRuleParserTest.java |   1 -
 .../seatunnel/flink/clickhouse/pojo/Shard.java     |   1 -
 .../flink/clickhouse/pojo/ShardMetadata.java       |   1 -
 .../apache/seatunnel/spark/hudi/source/Hudi.scala  |   2 +-
 .../apache/seatunnel/spark/jdbc/source/Jdbc.scala  |   4 +-
 .../util/{HiveDialet.scala => HiveDialect.scala}   |   2 +-
 .../core/base/command/AbstractCommandArgs.java     |   2 +-
 .../core/flink/args/FlinkCommandArgsTest.java      |   2 +-
 .../core/flink/utils/CommandLineUtilsTest.java     |  10 +-
 .../bin/start-seatunnel-flink-new-connector.sh     |   4 +-
 .../core/starter/flink/env/FlinkEnvironment.java   | 286 ---------------------
 .../execution/AbstractPluginExecuteProcessor.java  |  15 ++
 .../flink/execution/SinkExecuteProcessor.java      |   2 +-
 .../flink/execution/SourceExecuteProcessor.java    |   2 +-
 .../bin/start-seatunnel-spark-new-connector.sh     |   8 +-
 .../seatunnel/core/starter/spark/SparkStarter.java |  21 +-
 seatunnel-dependency-shade/pom.xml                 |  39 ---
 .../seatunnel-hive-shade/pom.xml                   |  69 -----
 seatunnel-e2e/pom.xml                              |  11 +
 .../e2e/flink/v2/fake/FakeSourceToConsoleIT.java   |   1 -
 .../flink/clickhouse/FakeSourceToClickhouseIT.java |   1 -
 .../e2e/spark/v2/fake/FakeSourceToConsoleIT.java   |   1 -
 .../e2e/spark/v2/file/FakeSourceToFileIT.java      |   1 -
 .../seatunnel-flink-connector-v2-example/pom.xml   |   7 +-
 .../resources/examples/fakesource_to_file.conf     |   6 +-
 .../seatunnel-spark-connector-v2-example/pom.xml   |   2 +-
 .../example/spark/v2/SeaTunnelApiExample.java      |   4 +-
 .../plugin/discovery/AbstractPluginDiscovery.java  |  67 ++++-
 .../plugin/discovery/PluginIdentifier.java         |   1 -
 ...very.java => FlinkAbstractPluginDiscovery.java} |  31 +--
 .../discovery/flink/FlinkSinkPluginDiscovery.java  |   3 +-
 .../flink/FlinkSourcePluginDiscovery.java          |   3 +-
 .../flink/FlinkTransformPluginDiscovery.java       |   3 +-
 .../seatunnel/SeaTunnelSinkPluginDiscovery.java    |   7 +
 .../seatunnel/SeaTunnelSourcePluginDiscovery.java  |   8 +
 .../seatunnel/app/common/SeatunnelErrorEnum.java   |   2 +-
 .../seatunnel/app/util/GlobalExceptionHandler.java |   4 +-
 .../translation/source/CoordinatedSource.java      |   3 +-
 .../translation/source/ParallelSource.java         |   3 +-
 .../flink/utils/TypeConverterUtilsTest.java        |   1 -
 .../batch/CoordinatedBatchPartitionReader.java     |   3 +-
 .../source/batch/ParallelBatchPartitionReader.java |   3 +-
 .../CoordinatedMicroBatchPartitionReader.java      |   3 +-
 seatunnel-ui/package.json                          |   3 +
 seatunnel-ui/pnpm-lock.yaml                        | 174 +++++++++++++
 .../store/theme/types.ts => postcss.config.js}     |  12 +-
 seatunnel-ui/src/App.tsx                           |   5 +-
 seatunnel-ui/src/components/chart/index.ts         |   2 +-
 seatunnel-ui/src/env.d.ts                          |   2 +-
 .../src/{store/theme/types.ts => index.css}        |  12 +-
 .../login => layouts/dashboard/header}/index.tsx   |  26 +-
 .../dashboard/header/logo}/index.tsx               |  15 +-
 .../dashboard/header/menu}/index.tsx               |  29 ++-
 .../dashboard/header/menu/use-menu.ts}             |  36 +--
 .../dashboard/header/setting}/index.tsx            |  30 ++-
 .../header/setting/use-setting-dropdown.ts         |  42 ++-
 .../dashboard/header/user}/index.tsx               |  31 ++-
 .../dashboard/header/user/use-user-dropdown.ts     |  35 ++-
 .../{views/login => layouts/dashboard}/index.tsx   |  17 +-
 seatunnel-ui/src/locales/en_US/index.ts            |  12 +-
 .../src/locales/{zh_CN/index.ts => en_US/login.ts} |  13 +-
 .../src/locales/{zh_CN/index.ts => en_US/menu.ts}  |  12 +-
 .../{zh_CN/index.ts => en_US/user-manage.ts}       |  19 +-
 seatunnel-ui/src/locales/index.ts                  |   2 +-
 seatunnel-ui/src/locales/zh_CN/index.ts            |   8 +-
 seatunnel-ui/src/main.ts                           |   5 +-
 .../src/router/{routes.ts => data-pipes.ts}        |  36 +--
 seatunnel-ui/src/router/index.ts                   |  11 +-
 seatunnel-ui/src/router/routes.ts                  |   8 +-
 .../src/router/{routes.ts => user-manage.ts}       |  36 +--
 seatunnel-ui/src/store/locale/index.ts             |   2 +-
 seatunnel-ui/src/store/locale/types.ts             |   2 +-
 seatunnel-ui/src/store/theme/index.ts              |   2 +-
 seatunnel-ui/src/store/theme/types.ts              |   3 +-
 seatunnel-ui/src/themes/index.ts                   |   3 +-
 seatunnel-ui/src/themes/modules/dark.ts            |   2 +-
 seatunnel-ui/src/themes/modules/light.ts           |   2 +-
 seatunnel-ui/src/utils/index.ts                    |   6 +-
 seatunnel-ui/src/utils/mapping.ts                  |   6 +-
 .../src/{store/theme/types.ts => utils/trim.ts}    |   7 +-
 .../src/views/{login => data-pipes/list}/index.tsx |  14 +-
 seatunnel-ui/src/views/login/index.tsx             |  63 ++++-
 .../{utils/mapping.ts => views/login/use-form.ts}  |  54 ++--
 seatunnel-ui/src/views/user-manage/list/index.tsx  |  68 +++++
 .../src/views/user-manage/list/use-table.ts        |  72 ++++++
 .../store/theme/types.ts => tailwind.config.js}    |  17 +-
 tools/checkstyle/checkStyle.xml                    |   9 +-
 tools/dependencies/checkLicense.sh                 |   6 +-
 tools/dependencies/known-dependencies.txt          |   1 +
 159 files changed, 2036 insertions(+), 1488 deletions(-)

diff --git a/.github/workflows/backend.yml b/.github/workflows/backend.yml
index 0a75a4950..ee6620ec7 100644
--- a/.github/workflows/backend.yml
+++ b/.github/workflows/backend.yml
@@ -133,7 +133,6 @@ jobs:
           -D"scalastyle.skip"=true
           -D"checkstyle.skip"=true
           -D"license.skipAddThirdParty"
-          -P release
       - name: Check Dependencies Licenses
         run: tools/dependencies/checkLicense.sh
 
diff --git a/docs/en/connector-v2/sink/File.mdx b/docs/en/connector-v2/sink/File.mdx
index cf92d188b..7e2e3efd7 100644
--- a/docs/en/connector-v2/sink/File.mdx
+++ b/docs/en/connector-v2/sink/File.mdx
@@ -111,6 +111,8 @@ Streaming Job not support `overwrite`.
 </TabItem>
 <TabItem value="HdfsFile">
 
+In order to use this connector, You must ensure your spark/flink cluster already integrated hadoop. The tested hadoop version is 2.x.
+
 | name                              | type   | required | default value                                                 |
 | --------------------------------- | ------ | -------- | ------------------------------------------------------------- |
 | path                              | string | yes      | -                                                             |
diff --git a/docs/en/connector-v2/sink/Hive.md b/docs/en/connector-v2/sink/Hive.md
new file mode 100644
index 000000000..1794633f1
--- /dev/null
+++ b/docs/en/connector-v2/sink/Hive.md
@@ -0,0 +1,62 @@
+# Hive
+
+## Description
+
+Write data to Hive.
+
+In order to use this connector, You must ensure your spark/flink cluster already integrated hive. The tested hive version is 2.3.9.
+
+## Options
+
+| name                              | type   | required | default value                                                 |
+| --------------------------------- | ------ | -------- | ------------------------------------------------------------- |
+| hive_table_name                   | string | yes      | -                                                             |
+| hive_metastore_uris               | string | yes      | -                                                             |
+| partition_by                      | array  | no       | -                                                             |
+| sink_columns                      | array  | no       | When this parameter is empty, all fields are sink columns     |
+| is_enable_transaction             | boolean| no       | true                                                          |
+| save_mode                         | string | no       | "append"                                                      |
+
+### hive_table_name [string]
+
+Target Hive table name eg: db1.table1
+
+### hive_metastore_uris [string]
+
+Hive metastore uris
+
+### partition_by [array]
+
+Partition data based on selected fields
+
+### sink_columns [array]
+
+Which columns need be write to hive, default value is all of the columns get from `Transform` or `Source`.
+The order of the fields determines the order in which the file is actually written.
+
+### is_enable_transaction [boolean]
+
+If `is_enable_transaction` is true, we will ensure that data will not be lost or duplicated when it is written to the target directory.
+
+Only support `true` now.
+
+### save_mode [string]
+
+Storage mode, we need support `overwrite` and `append`. `append` is now supported.
+
+Streaming Job not support `overwrite`.
+
+## Example
+
+```bash
+
+Hive {
+    hive_table_name="db1.table1"
+    hive_metastore_uris="thrift://localhost:9083"
+    partition_by=["age"]
+    sink_columns=["name","age"]
+    is_enable_transaction=true
+    save_mode="append"
+}
+
+```
diff --git a/docs/en/connector-v2/source/Hudi.md b/docs/en/connector-v2/source/Hudi.md
index 4803b1f44..4851d61ef 100644
--- a/docs/en/connector-v2/source/Hudi.md
+++ b/docs/en/connector-v2/source/Hudi.md
@@ -4,6 +4,8 @@
 
 Used to read data from Hudi. Currently, only supports hudi cow table and Snapshot Query with Batch Mode.
 
+In order to use this connector, You must ensure your spark/flink cluster already integrated hive. The tested hive version is 2.3.9.
+
 ## Options
 
 | name                     | type    | required | default value |
diff --git a/docs/en/start/local.mdx b/docs/en/start/local.mdx
index 13e61b885..8f4499390 100644
--- a/docs/en/start/local.mdx
+++ b/docs/en/start/local.mdx
@@ -15,7 +15,7 @@ Before you getting start the local run, you need to make sure you already have i
 * Download the engine, you can choose and download one of them from below as your favour, you could see more information about [why we need engine in SeaTunnel](../faq.md#why-i-should-install-computing-engine-like-spark-or-flink)
   * Spark: Please [download Spark](https://spark.apache.org/downloads.html) first(**required version >= 2 and version < 3.x **). For more information you could
   see [Getting Started: standalone](https://spark.apache.org/docs/latest/spark-standalone.html#installing-spark-standalone-to-a-cluster)
-  * Flink: Please [download Flink](https://flink.apache.org/downloads.html) first(**required version >= 1.9.0 and version < 1.14.x **). For more information you could see [Getting Started: standalone](https://nightlies.apache.org/flink/flink-docs-release-1.14/docs/deployment/resource-providers/standalone/overview/)
+  * Flink: Please [download Flink](https://flink.apache.org/downloads.html) first(**required version >= 1.12.0 and version < 1.14.x **). For more information you could see [Getting Started: standalone](https://nightlies.apache.org/flink/flink-docs-release-1.14/docs/deployment/resource-providers/standalone/overview/)
 
 ## Installation
 
diff --git a/docs/sidebars.js b/docs/sidebars.js
index ef8c73158..ad3cb667f 100644
--- a/docs/sidebars.js
+++ b/docs/sidebars.js
@@ -78,6 +78,51 @@ const sidebars = {
         'concept/config',
       ],
     },
+      {
+          type: 'category',
+          label: 'Connector-V2',
+          items: [
+              {
+                  type: 'category',
+                  label: 'Sink',
+                  link: {
+                      type: 'generated-index',
+                      title: 'Source-V2 of SeaTunnel',
+                      description: 'List all Sink supported Apache SeaTunnel for now.',
+                      // Should remove the `v2` suffix when we migrate all sink to v2 and delete the old one
+                      slug: '/category/sink-v2',
+                      keywords: ['sink'],
+                      image: '/img/favicon.ico',
+                  },
+                  items: [
+                      {
+                          type: 'autogenerated',
+                          dirName: 'connector-v2/sink',
+                      },
+                  ],
+              },
+              {
+                  type: 'category',
+                  label: 'Source',
+                  link: {
+                      type: 'generated-index',
+                      title: 'Source-V2 of SeaTunnel',
+                      description: 'List all source supported Apache SeaTunnel for now.',
+                      // Should remove the `v2` suffix when we migrate all sink to v2 and delete the old one
+                      slug: '/category/source-v2',
+                      keywords: ['source'],
+                      image: '/img/favicon.ico',
+                  },
+                  items: [
+                      {
+                          type: 'autogenerated',
+                          dirName: 'connector-v2/source',
+                      },
+                  ],
+
+              },
+          ],
+      },
     {
       type: 'category',
       label: 'Connector',
diff --git a/pom.xml b/pom.xml
index d0e5e1eb3..4469c2d74 100644
--- a/pom.xml
+++ b/pom.xml
@@ -98,7 +98,6 @@
                 <activeByDefault>true</activeByDefault>
             </activation>
             <modules>
-                <module>seatunnel-dependency-shade</module>
                 <module>seatunnel-connectors-v2</module>
                 <module>seatunnel-connectors-v2-dist</module>
                 <module>seatunnel-examples</module>
@@ -765,11 +764,6 @@
                 <artifactId>jackson-jaxrs</artifactId>
                 <version>${codehaus.jackson.version}</version>
             </dependency>
-            <dependency>
-                <groupId>org.codehaus.janino</groupId>
-                <artifactId>janino</artifactId>
-                <version>${janino.version}</version>
-            </dependency>
             <dependency>
                 <groupId>com.sun.jersey</groupId>
                 <artifactId>jersey-json</artifactId>
@@ -866,11 +860,6 @@
                 <artifactId>curator-recipes</artifactId>
                 <version>${curator.version}</version>
             </dependency>
-            <dependency>
-                <groupId>org.apache.curator</groupId>
-                <artifactId>curator-recipes</artifactId>
-                <version>${curator.version}</version>
-            </dependency>
             <dependency>
                 <groupId>com.sun.jersey</groupId>
                 <artifactId>jersey-core</artifactId>
diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/source/SeaTunnelSource.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/source/SeaTunnelSource.java
index ca3a9f900..f93f4d3bf 100644
--- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/source/SeaTunnelSource.java
+++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/source/SeaTunnelSource.java
@@ -33,7 +33,7 @@ import java.io.Serializable;
  * @param <SplitT> The type of splits handled by the source.
  * @param <StateT> The type of checkpoint states.
  */
-public interface SeaTunnelSource<T, SplitT extends SourceSplit, StateT>
+public interface SeaTunnelSource<T, SplitT extends SourceSplit, StateT extends Serializable>
     extends Serializable, PluginIdentifierInterface, SeaTunnelPluginLifeCycle, SeaTunnelContextAware {
 
     /**
@@ -94,6 +94,8 @@ public interface SeaTunnelSource<T, SplitT extends SourceSplit, StateT>
      *
      * @return enumerator state serializer.
      */
-    Serializer<StateT> getEnumeratorStateSerializer();
+    default Serializer<StateT> getEnumeratorStateSerializer(){
+        return new DefaultSerializer<>();
+    }
 
 }
diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/connector/TableSource.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/connector/TableSource.java
index edb8ee69d..d9b6294c4 100644
--- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/connector/TableSource.java
+++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/connector/TableSource.java
@@ -20,7 +20,9 @@ package org.apache.seatunnel.api.table.connector;
 import org.apache.seatunnel.api.source.SeaTunnelSource;
 import org.apache.seatunnel.api.source.SourceSplit;
 
-public interface TableSource<T, SplitT extends SourceSplit, StateT> {
+import java.io.Serializable;
+
+public interface TableSource<T, SplitT extends SourceSplit, StateT extends Serializable> {
 
     SeaTunnelSource<T, SplitT, StateT> createSource();
 }
diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/FactoryUtil.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/FactoryUtil.java
index c260c2640..288e35beb 100644
--- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/FactoryUtil.java
+++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/FactoryUtil.java
@@ -27,6 +27,7 @@ import org.apache.seatunnel.api.table.connector.TableSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.LinkedList;
 import java.util.List;
@@ -42,7 +43,7 @@ public final class FactoryUtil {
 
     private static final Logger LOG = LoggerFactory.getLogger(FactoryUtil.class);
 
-    public static <T, SplitT extends SourceSplit, StateT> List<SeaTunnelSource<T, SplitT, StateT>> createAndPrepareSource(
+    public static <T, SplitT extends SourceSplit, StateT extends Serializable> List<SeaTunnelSource<T, SplitT, StateT>> createAndPrepareSource(
         List<CatalogTable> multipleTables,
         Map<String, String> options,
         ClassLoader classLoader,
diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/TableSourceFactory.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/TableSourceFactory.java
index a75236e0c..d637cc970 100644
--- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/TableSourceFactory.java
+++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/TableSourceFactory.java
@@ -20,11 +20,13 @@ package org.apache.seatunnel.api.table.factory;
 import org.apache.seatunnel.api.source.SourceSplit;
 import org.apache.seatunnel.api.table.connector.TableSource;
 
+import java.io.Serializable;
+
 /**
  * This is an SPI interface, used to create {@link TableSource}. Each plugin need to have it own implementation.
  * todo: now we have not use this interface, we directly use {@link org.apache.seatunnel.api.source.SeaTunnelSource} as the SPI interface
  */
 public interface TableSourceFactory extends Factory {
 
-    <T, SplitT extends SourceSplit, StateT> TableSource<T, SplitT, StateT> createSource(TableFactoryContext context);
+    <T, SplitT extends SourceSplit, StateT extends Serializable> TableSource<T, SplitT, StateT> createSource(TableFactoryContext context);
 }
diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/type/SeaTunnelRow.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/type/SeaTunnelRow.java
index de82a639b..aa5923ec1 100644
--- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/type/SeaTunnelRow.java
+++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/type/SeaTunnelRow.java
@@ -87,7 +87,6 @@ public final class SeaTunnelRow implements Serializable {
         return tableId == that.tableId && kind == that.kind && Arrays.deepEquals(fields, that.fields);
     }
 
-    @SuppressWarnings("magicnumber")
     @Override
     public int hashCode() {
         int result = Objects.hash(tableId, kind);
diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/type/SeaTunnelRowType.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/type/SeaTunnelRowType.java
index 45164c246..cb94bba00 100644
--- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/type/SeaTunnelRowType.java
+++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/type/SeaTunnelRowType.java
@@ -98,7 +98,6 @@ public class SeaTunnelRowType implements CompositeType<SeaTunnelRow> {
     }
 
     @Override
-    @SuppressWarnings("MagicNumber")
     public int hashCode() {
         int result = Arrays.hashCode(fieldNames);
         result = 31 * result + Arrays.hashCode(fieldTypes);
diff --git a/seatunnel-apis/seatunnel-api-flink/src/main/java/org/apache/seatunnel/flink/FlinkEnvironment.java b/seatunnel-apis/seatunnel-api-flink/src/main/java/org/apache/seatunnel/flink/FlinkEnvironment.java
index eaf6780d8..49c97beb6 100644
--- a/seatunnel-apis/seatunnel-api-flink/src/main/java/org/apache/seatunnel/flink/FlinkEnvironment.java
+++ b/seatunnel-apis/seatunnel-api-flink/src/main/java/org/apache/seatunnel/flink/FlinkEnvironment.java
@@ -123,31 +123,31 @@ public class FlinkEnvironment implements RuntimeEnv {
     @Override
     public void registerPlugin(List<URL> pluginPaths) {
         pluginPaths.forEach(url -> LOGGER.info("register plugins : {}", url));
-        Configuration configuration;
+        List<Configuration> configurations = new ArrayList<>();
         try {
-            if (isStreaming()) {
-                configuration =
-                        (Configuration) Objects.requireNonNull(ReflectionUtils.getDeclaredMethod(StreamExecutionEnvironment.class,
-                                "getConfiguration")).orElseThrow(() -> new RuntimeException("can't find " +
-                                "method: getConfiguration")).invoke(this.environment);
-            } else {
-                configuration = batchEnvironment.getConfiguration();
+            configurations.add((Configuration) Objects.requireNonNull(ReflectionUtils.getDeclaredMethod(StreamExecutionEnvironment.class,
+                    "getConfiguration")).orElseThrow(() -> new RuntimeException("can't find " +
+                    "method: getConfiguration")).invoke(this.environment));
+            if (!isStreaming()) {
+                configurations.add(batchEnvironment.getConfiguration());
             }
         } catch (Exception e) {
             throw new RuntimeException(e);
         }
-        List<String> jars = configuration.get(PipelineOptions.JARS);
-        if (jars == null) {
-            jars = new ArrayList<>();
-        }
-        jars.addAll(pluginPaths.stream().map(URL::toString).collect(Collectors.toList()));
-        configuration.set(PipelineOptions.JARS, jars);
-        List<String> classpath = configuration.get(PipelineOptions.CLASSPATHS);
-        if (classpath == null) {
-            classpath = new ArrayList<>();
-        }
-        classpath.addAll(pluginPaths.stream().map(URL::toString).collect(Collectors.toList()));
-        configuration.set(PipelineOptions.CLASSPATHS, classpath);
+        configurations.forEach(configuration -> {
+            List<String> jars = configuration.get(PipelineOptions.JARS);
+            if (jars == null) {
+                jars = new ArrayList<>();
+            }
+            jars.addAll(pluginPaths.stream().map(URL::toString).collect(Collectors.toList()));
+            configuration.set(PipelineOptions.JARS, jars.stream().distinct().collect(Collectors.toList()));
+            List<String> classpath = configuration.get(PipelineOptions.CLASSPATHS);
+            if (classpath == null) {
+                classpath = new ArrayList<>();
+            }
+            classpath.addAll(pluginPaths.stream().map(URL::toString).collect(Collectors.toList()));
+            configuration.set(PipelineOptions.CLASSPATHS, classpath.stream().distinct().collect(Collectors.toList()));
+        });
     }
 
     public StreamExecutionEnvironment getStreamExecutionEnvironment() {
diff --git a/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/ReflectionUtils.java b/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/ReflectionUtils.java
index 3edd9cccf..f186f0486 100644
--- a/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/ReflectionUtils.java
+++ b/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/ReflectionUtils.java
@@ -28,7 +28,7 @@ public class ReflectionUtils {
 
         Optional<Method> method = Optional.empty();
         Method m;
-        for (; clazz != Object.class; clazz = clazz.getSuperclass()) {
+        for (; clazz != null; clazz = clazz.getSuperclass()) {
             try {
                 m = clazz.getDeclaredMethod(methodName, parameterTypes);
                 m.setAccessible(true);
@@ -80,9 +80,14 @@ public class ReflectionUtils {
     public static Object invoke(
             Object object, String methodName, Class<?>[] argTypes, Object[] args) {
         try {
-            Method method = object.getClass().getDeclaredMethod(methodName, argTypes);
-            method.setAccessible(true);
-            return method.invoke(object, args);
+            Optional<Method> method = getDeclaredMethod(object.getClass(), methodName, argTypes);
+            if (method.isPresent()) {
+                method.get().setAccessible(true);
+                return method.get().invoke(object, args);
+            } else {
+                throw new NoSuchMethodException(String.format("method invoke failed, no such method '%s' in '%s'",
+                        methodName, object.getClass()));
+            }
         } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) {
             throw new RuntimeException("method invoke failed", e);
         }
diff --git a/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/SerializationException.java b/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/SerializationException.java
new file mode 100644
index 000000000..33ca29a6f
--- /dev/null
+++ b/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/SerializationException.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.common.utils;
+
+public class SerializationException extends RuntimeException {
+
+    /**
+     * Required for serialization support.
+     *
+     * @see java.io.Serializable
+     */
+    private static final long serialVersionUID = 2263144814025689516L;
+
+    /**
+     * <p>Constructs a new {@code SerializationException} without specified
+     * detail message.</p>
+     */
+    public SerializationException() {
+    }
+
+    /**
+     * <p>Constructs a new {@code SerializationException} with specified
+     * detail message.</p>
+     *
+     * @param msg The error message.
+     */
+    public SerializationException(final String msg) {
+        super(msg);
+    }
+
+    /**
+     * <p>Constructs a new {@code SerializationException} with specified
+     * nested {@code Throwable}.</p>
+     *
+     * @param cause The {@code Exception} or {@code Error}
+     *              that caused this exception to be thrown.
+     */
+    public SerializationException(final Throwable cause) {
+        super(cause);
+    }
+
+    /**
+     * <p>Constructs a new {@code SerializationException} with specified
+     * detail message and nested {@code Throwable}.</p>
+     *
+     * @param msg   The error message.
+     * @param cause The {@code Exception} or {@code Error}
+     *              that caused this exception to be thrown.
+     */
+    public SerializationException(final String msg, final Throwable cause) {
+        super(msg, cause);
+    }
+}
diff --git a/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/SerializationUtils.java b/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/SerializationUtils.java
index 43dead167..7e721617e 100644
--- a/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/SerializationUtils.java
+++ b/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/SerializationUtils.java
@@ -20,30 +20,60 @@ package org.apache.seatunnel.common.utils;
 import org.apache.commons.codec.binary.Base64;
 import org.apache.commons.lang3.StringUtils;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamClass;
 import java.io.Serializable;
 
 public class SerializationUtils {
 
     public static String objectToString(Serializable obj) {
         if (obj != null) {
-            return Base64.encodeBase64String(org.apache.commons.lang3.SerializationUtils.serialize(obj));
+            return Base64.encodeBase64String(serialize(obj));
         }
         return null;
     }
 
     public static <T extends Serializable> T stringToObject(String str) {
         if (StringUtils.isNotEmpty(str)) {
-            return org.apache.commons.lang3.SerializationUtils.deserialize(Base64.decodeBase64(str));
+            return deserialize(Base64.decodeBase64(str));
         }
         return null;
     }
 
+    @SuppressWarnings("checkstyle:MagicNumber")
     public static <T extends Serializable> byte[] serialize(T obj) {
-        return org.apache.commons.lang3.SerializationUtils.serialize(obj);
+        try (ByteArrayOutputStream b = new ByteArrayOutputStream(512);
+             ObjectOutputStream out = new ObjectOutputStream(b)) {
+            out.writeObject(obj);
+            return b.toByteArray();
+        } catch (final IOException ex) {
+            throw new SerializationException(ex);
+        }
+
     }
 
     public static <T extends Serializable> T deserialize(byte[] bytes) {
-        return org.apache.commons.lang3.SerializationUtils.deserialize(bytes);
+        try (ByteArrayInputStream s = new ByteArrayInputStream(bytes);
+             ObjectInputStream in = new ObjectInputStream(s) {
+                 @Override
+                 protected Class<?> resolveClass(ObjectStreamClass desc) throws IOException, ClassNotFoundException {
+                     // make sure use current thread classloader
+                     ClassLoader cl = Thread.currentThread().getContextClassLoader();
+                     if (cl == null) {
+                         return super.resolveClass(desc);
+                     }
+                     return Class.forName(desc.getName(), false, cl);
+                 }
+             }) {
+            @SuppressWarnings("unchecked") final T obj = (T) in.readObject();
+            return obj;
+        } catch (final ClassNotFoundException | IOException ex) {
+            throw new SerializationException(ex);
+        }
     }
 
 }
diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/fake/FakeSourceToConsoleIT.java b/seatunnel-common/src/test/java/org/apache/seatunnel/common/utils/ReflectionUtilsTest.java
similarity index 60%
copy from seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/fake/FakeSourceToConsoleIT.java
copy to seatunnel-common/src/test/java/org/apache/seatunnel/common/utils/ReflectionUtilsTest.java
index f715c4e79..422c70eef 100644
--- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/fake/FakeSourceToConsoleIT.java
+++ b/seatunnel-common/src/test/java/org/apache/seatunnel/common/utils/ReflectionUtilsTest.java
@@ -15,22 +15,24 @@
  * limitations under the License.
  */
 
-package org.apache.seatunnel.e2e.flink.v2.fake;
-
-import org.apache.seatunnel.e2e.flink.FlinkContainer;
+package org.apache.seatunnel.common.utils;
 
 import org.junit.Assert;
 import org.junit.Test;
-import org.testcontainers.containers.Container;
 
-import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLClassLoader;
 
-public class FakeSourceToConsoleIT extends FlinkContainer {
+public class ReflectionUtilsTest {
 
     @Test
-    @SuppressWarnings("magicnumber")
-    public void testFakeSourceToConsoleSink() throws IOException, InterruptedException {
-        Container.ExecResult execResult = executeSeaTunnelFlinkJob("/fake/fakesource_to_console.conf");
-        Assert.assertEquals(0, execResult.getExitCode());
+    public void testInvoke() throws MalformedURLException {
+        ReflectionUtils.invoke(new String[]{}, "toString");
+
+        URLClassLoader classLoader = new URLClassLoader(new URL[]{}, Thread.currentThread().getContextClassLoader());
+        ReflectionUtils.invoke(classLoader, "addURL", new URL("file:///test"));
+        Assert.assertArrayEquals(classLoader.getURLs(), new URL[]{new URL("file:///test")});
     }
+
 }
diff --git a/seatunnel-common/src/test/java/org/apache/seatunnel/common/utils/SerializationUtilsTest.java b/seatunnel-common/src/test/java/org/apache/seatunnel/common/utils/SerializationUtilsTest.java
new file mode 100644
index 000000000..ad0e80cea
--- /dev/null
+++ b/seatunnel-common/src/test/java/org/apache/seatunnel/common/utils/SerializationUtilsTest.java
@@ -0,0 +1,74 @@
+/*
+ * 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.seatunnel.common.utils;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+
+@SuppressWarnings("checkstyle:RegexpSingleline")
+public class SerializationUtilsTest {
+
+    @Test
+    public void testObjectToString() {
+
+        HashMap<String, String> data = new HashMap<>();
+        data.put("key1", "value1");
+        data.put("seatunnelTest", "apache SeaTunnel");
+        data.put("中 文", "Apache Asia");
+        String configStr = SerializationUtils.objectToString(data);
+        Assert.assertNotNull(configStr);
+
+        HashMap<String, String> dataAfter = SerializationUtils.stringToObject(configStr);
+
+        Assert.assertEquals(dataAfter, data);
+
+        data.put("key2", "");
+        Assert.assertNotEquals(dataAfter, data);
+
+    }
+
+    @Test
+    public void testByteToObject() {
+
+        HashMap<String, String> data = new HashMap<>();
+        data.put("key1", "value1");
+        data.put("seatunnelTest", "apache SeaTunnel");
+        data.put("中 文", "Apache Asia");
+
+        ArrayList<HashMap<String, String>> array = new ArrayList<>();
+        array.add(data);
+        HashMap<String, String> data2 = new HashMap<>();
+        data2.put("Apache Asia", "中 文");
+        data2.put("value1", "key1");
+        data2.put("apache SeaTunnel", "seatunnelTest");
+        array.add(data2);
+
+        byte[] result = SerializationUtils.serialize(array);
+
+        ArrayList<HashMap<String, String>> array2 = SerializationUtils.deserialize(result);
+
+        Assert.assertEquals(array2, array);
+
+        Assert.assertThrows(SerializationException.class, () -> SerializationUtils.deserialize(new byte[]{1, 0, 1}));
+
+    }
+
+}
diff --git a/seatunnel-common/src/test/java/org/apache/seatunnel/common/utils/VariablesSubstituteTest.java b/seatunnel-common/src/test/java/org/apache/seatunnel/common/utils/VariablesSubstituteTest.java
index 33d721ae8..09d044a53 100644
--- a/seatunnel-common/src/test/java/org/apache/seatunnel/common/utils/VariablesSubstituteTest.java
+++ b/seatunnel-common/src/test/java/org/apache/seatunnel/common/utils/VariablesSubstituteTest.java
@@ -26,7 +26,6 @@ import java.util.HashMap;
 
 public class VariablesSubstituteTest {
 
-    @SuppressWarnings("checkstyle:MagicNumber")
     @Test
     public void testSubstitute() {
         String timeFormat = "yyyyMMddHHmmss";
diff --git a/seatunnel-connectors-v2/README.md b/seatunnel-connectors-v2/README.md
index 7522b846e..b24122d94 100644
--- a/seatunnel-connectors-v2/README.md
+++ b/seatunnel-connectors-v2/README.md
@@ -5,7 +5,7 @@ This article introduces the new interface and the new code structure on account
 In order to separate from the old code, we have defined new modules for execution flow. This facilitates parallel development at the current stage, and reduces the difficulty of merging. All the relevant code at this stage is kept on the ``api-draft`` branch.
 
 ### **Example**
-We have prepared a new version of the locally executable example program in ``seatunnel-examples``, which can be directly called using ``seatunnel-flink-new-connector-example`` or ``seatunnel-spark-new-connector-example`` in ``SeaTunnelApiExample``. This is also the debugging method that is often used in the local development of Connector. The corresponding configuration files are saved in the same module ``resources/examples`` folder as before.
+We have prepared a new version of the locally executable example program in ``seatunnel-examples``, which can be directly called using ``seatunnel-flink-connector-v2-example`` or ``seatunnel-spark-connector-v2-example`` in ``SeaTunnelApiExample``. This is also the debugging method that is often used in the local development of Connector. The corresponding configuration files are saved in the same module ``resources/examples`` folder as before.
 
 
 ### **Startup Class**
diff --git a/seatunnel-connectors-v2/README.zh.md b/seatunnel-connectors-v2/README.zh.md
index c0aa4d639..017b19907 100644
--- a/seatunnel-connectors-v2/README.zh.md
+++ b/seatunnel-connectors-v2/README.zh.md
@@ -4,7 +4,7 @@ Because SeaTunnel design new API for connectors, 所以通过这篇文章来介
 现阶段所有相关代码保存在`api-draft`分支上。
 为了和老的代码分开,方便现阶段的并行开发,以及降低merge的难度。我们为新的执行流程定义了新的模块
 ### Example
-我们已经在`seatunnel-examples`中准备好了新版本的可本地执行Example程序,直接调用`seatunnel-flink-new-connector-example`或`seatunnel-spark-new-connector-example`中的`SeaTunnelApiExample`即可。这也是本地开发Connector经常会用到的调试方式。
+我们已经在`seatunnel-examples`中准备好了新版本的可本地执行Example程序,直接调用`seatunnel-flink-connector-v2-example`或`seatunnel-spark-connector-v2-example`中的`SeaTunnelApiExample`即可。这也是本地开发Connector经常会用到的调试方式。
 对应的配置文件保存在同模块的`resources/examples`文件夹下,和以前一样。
 ### 启动类
 和老的启动类分开,我们创建了两个新的启动类工程,分别是`seatunnel-core/seatunnel-flink-starter`和`seatunnel-core/seatunnel-spark-starter`. 可以在这里找到如何将配置文件解析为可以执行的Flink/Spark流程。
diff --git a/seatunnel-connectors-v2/connector-assert/src/test/java/org/apache/seatunnel/flink/assertion/AssertExecutorTest.java b/seatunnel-connectors-v2/connector-assert/src/test/java/org/apache/seatunnel/flink/assertion/AssertExecutorTest.java
index c5d30f3b1..fa9fee209 100644
--- a/seatunnel-connectors-v2/connector-assert/src/test/java/org/apache/seatunnel/flink/assertion/AssertExecutorTest.java
+++ b/seatunnel-connectors-v2/connector-assert/src/test/java/org/apache/seatunnel/flink/assertion/AssertExecutorTest.java
@@ -29,7 +29,6 @@ import junit.framework.TestCase;
 
 import java.util.List;
 
-@SuppressWarnings("magicnumber")
 public class AssertExecutorTest extends TestCase {
     SeaTunnelRow row = new SeaTunnelRow(new Object[]{"jared", 17});
     SeaTunnelRowType rowType = new SeaTunnelRowType(new String[]{"name", "age"}, new SeaTunnelDataType[]{BasicType.STRING_TYPE, BasicType.INT_TYPE});
diff --git a/seatunnel-connectors-v2/connector-assert/src/test/java/org/apache/seatunnel/flink/assertion/rule/AssertRuleParserTest.java b/seatunnel-connectors-v2/connector-assert/src/test/java/org/apache/seatunnel/flink/assertion/rule/AssertRuleParserTest.java
index 9a9628ded..5c1bb650c 100644
--- a/seatunnel-connectors-v2/connector-assert/src/test/java/org/apache/seatunnel/flink/assertion/rule/AssertRuleParserTest.java
+++ b/seatunnel-connectors-v2/connector-assert/src/test/java/org/apache/seatunnel/flink/assertion/rule/AssertRuleParserTest.java
@@ -28,7 +28,6 @@ import junit.framework.TestCase;
 
 import java.util.List;
 
-@SuppressWarnings("magicnumber")
 public class AssertRuleParserTest extends TestCase {
     AssertRuleParser parser = new AssertRuleParser();
 
diff --git a/seatunnel-connectors-v2/connector-clickhouse/src/main/java/org/apache/seatunnel/connectors/seatunnel/clickhouse/source/ClickhouseSource.java b/seatunnel-connectors-v2/connector-clickhouse/src/main/java/org/apache/seatunnel/connectors/seatunnel/clickhouse/source/ClickhouseSource.java
index a94344e72..4a4b638d9 100644
--- a/seatunnel-connectors-v2/connector-clickhouse/src/main/java/org/apache/seatunnel/connectors/seatunnel/clickhouse/source/ClickhouseSource.java
+++ b/seatunnel-connectors-v2/connector-clickhouse/src/main/java/org/apache/seatunnel/connectors/seatunnel/clickhouse/source/ClickhouseSource.java
@@ -24,8 +24,6 @@ import static org.apache.seatunnel.connectors.seatunnel.clickhouse.config.Config
 import static org.apache.seatunnel.connectors.seatunnel.clickhouse.config.Config.USERNAME;
 
 import org.apache.seatunnel.api.common.PrepareFailException;
-import org.apache.seatunnel.api.serialization.DefaultSerializer;
-import org.apache.seatunnel.api.serialization.Serializer;
 import org.apache.seatunnel.api.source.Boundedness;
 import org.apache.seatunnel.api.source.SeaTunnelSource;
 import org.apache.seatunnel.api.source.SourceReader;
@@ -124,9 +122,4 @@ public class ClickhouseSource implements SeaTunnelSource<SeaTunnelRow, Clickhous
         return new ClickhouseSourceSplitEnumerator(enumeratorContext);
     }
 
-    @Override
-    public Serializer<ClickhouseSourceState> getEnumeratorStateSerializer() {
-        return new DefaultSerializer<>();
-    }
-
 }
diff --git a/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/source/AbstractSingleSplitSource.java b/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/source/AbstractSingleSplitSource.java
index 255150d9e..54410962b 100644
--- a/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/source/AbstractSingleSplitSource.java
+++ b/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/source/AbstractSingleSplitSource.java
@@ -45,11 +45,6 @@ public abstract class AbstractSingleSplitSource<T> implements SeaTunnelSource<T,
         return createEnumerator(enumeratorContext);
     }
 
-    @Override
-    public final Serializer<SingleSplitEnumeratorState> getEnumeratorStateSerializer() {
-        return new DefaultSerializer<>();
-    }
-
     @Override
     public final Serializer<SingleSplit> getSplitSerializer() {
         return new DefaultSerializer<>();
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/AbstractTextFileConfig.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/BaseTextFileConfig.java
similarity index 91%
rename from seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/AbstractTextFileConfig.java
rename to seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/BaseTextFileConfig.java
index 32672066d..d6fd26d1b 100644
--- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/AbstractTextFileConfig.java
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/BaseTextFileConfig.java
@@ -29,7 +29,7 @@ import java.io.Serializable;
 import java.util.Locale;
 
 @Data
-public class AbstractTextFileConfig implements DelimiterConfig, CompressConfig, Serializable {
+public class BaseTextFileConfig implements DelimiterConfig, CompressConfig, Serializable {
     private static final long serialVersionUID = 1L;
 
     protected String compressCodec;
@@ -42,9 +42,7 @@ public class AbstractTextFileConfig implements DelimiterConfig, CompressConfig,
     protected String fileNameExpression;
     protected FileFormat fileFormat = FileFormat.TEXT;
 
-    public AbstractTextFileConfig(@NonNull Config config) {
-        checkNotNull(config.getString(Constant.PATH));
-
+    public BaseTextFileConfig(@NonNull Config config) {
         if (config.hasPath(Constant.COMPRESS_CODEC)) {
             throw new RuntimeException("compress not support now");
         }
@@ -60,6 +58,7 @@ public class AbstractTextFileConfig implements DelimiterConfig, CompressConfig,
         if (config.hasPath(Constant.PATH) && !StringUtils.isBlank(config.getString(Constant.PATH))) {
             this.path = config.getString(Constant.PATH);
         }
+        checkNotNull(path);
 
         if (config.hasPath(Constant.FILE_NAME_EXPRESSION) && !StringUtils.isBlank(config.getString(Constant.FILE_NAME_EXPRESSION))) {
             this.fileNameExpression = config.getString(Constant.FILE_NAME_EXPRESSION);
@@ -70,6 +69,5 @@ public class AbstractTextFileConfig implements DelimiterConfig, CompressConfig,
         }
     }
 
-    protected AbstractTextFileConfig() {
-    }
+    public BaseTextFileConfig() {}
 }
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/AbstractFileSink.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/AbstractFileSink.java
index a296eea53..77b72f004 100644
--- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/AbstractFileSink.java
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/AbstractFileSink.java
@@ -41,7 +41,6 @@ import java.util.Optional;
 
 /**
  * Hive Sink implementation by using SeaTunnel sink API.
- * This class contains the method to create {@link TransactionStateFileSinkWriter} and {@link FileSinkAggregatedCommitter}.
  */
 public abstract class AbstractFileSink implements SeaTunnelSink<SeaTunnelRow, FileSinkState, FileCommitInfo, FileAggregatedCommitInfo> {
     private Config config;
@@ -92,7 +91,7 @@ public abstract class AbstractFileSink implements SeaTunnelSink<SeaTunnelRow, Fi
     @Override
     public SinkWriter<SeaTunnelRow, FileCommitInfo, FileSinkState> restoreWriter(SinkWriter.Context context, List<FileSinkState> states) throws IOException {
         if (this.getSinkConfig().isEnableTransaction()) {
-            return new TransactionStateFileSinkWriter(seaTunnelRowTypeInfo,
+            return new FileSinkWriterWithTransaction(seaTunnelRowTypeInfo,
                 config,
                 context,
                 textFileSinkConfig,
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/FileAggregatedCommitInfo.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/FileAggregatedCommitInfo.java
index 1036c3a59..c847ff659 100644
--- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/FileAggregatedCommitInfo.java
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/FileAggregatedCommitInfo.java
@@ -21,6 +21,7 @@ import lombok.AllArgsConstructor;
 import lombok.Data;
 
 import java.io.Serializable;
+import java.util.List;
 import java.util.Map;
 
 @Data
@@ -33,4 +34,6 @@ public class FileAggregatedCommitInfo implements Serializable {
      * V is the target file path of the data file.
      */
     private Map<String, Map<String, String>> transactionMap;
+
+    private Map<String, List<String>> partitionDirAndValsMap;
 }
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/FileCommitInfo.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/FileCommitInfo.java
index 689b85ebf..0fcb04a03 100644
--- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/FileCommitInfo.java
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/FileCommitInfo.java
@@ -21,6 +21,7 @@ import lombok.AllArgsConstructor;
 import lombok.Data;
 
 import java.io.Serializable;
+import java.util.List;
 import java.util.Map;
 
 @Data
@@ -34,5 +35,7 @@ public class FileCommitInfo implements Serializable {
      */
     private Map<String, String> needMoveFiles;
 
+    private Map<String, List<String>> partitionDirAndValsMap;
+
     private String transactionDir;
 }
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/FileSinkAggregatedCommitter.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/FileSinkAggregatedCommitter.java
index cc8ff2404..3c7c8cf9c 100644
--- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/FileSinkAggregatedCommitter.java
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/FileSinkAggregatedCommitter.java
@@ -21,6 +21,7 @@ import org.apache.seatunnel.api.sink.SinkAggregatedCommitter;
 import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.FileSystemCommitter;
 
 import lombok.NonNull;
+import org.apache.commons.collections4.CollectionUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -29,6 +30,7 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 public class FileSinkAggregatedCommitter implements SinkAggregatedCommitter<FileCommitInfo, FileAggregatedCommitInfo> {
     private static final Logger LOGGER = LoggerFactory.getLogger(FileSinkAggregatedCommitter.class);
@@ -63,6 +65,7 @@ public class FileSinkAggregatedCommitter implements SinkAggregatedCommitter<File
             return null;
         }
         Map<String, Map<String, String>> aggregateCommitInfo = new HashMap<>();
+        Map<String, List<String>> partitionDirAndValsMap = new HashMap<>();
         commitInfos.stream().forEach(commitInfo -> {
             Map<String, String> needMoveFileMap = aggregateCommitInfo.get(commitInfo.getTransactionDir());
             if (needMoveFileMap == null) {
@@ -70,8 +73,12 @@ public class FileSinkAggregatedCommitter implements SinkAggregatedCommitter<File
                 aggregateCommitInfo.put(commitInfo.getTransactionDir(), needMoveFileMap);
             }
             needMoveFileMap.putAll(commitInfo.getNeedMoveFiles());
+            Set<Map.Entry<String, List<String>>> entries = commitInfo.getPartitionDirAndValsMap().entrySet();
+            if (!CollectionUtils.isEmpty(entries)) {
+                partitionDirAndValsMap.putAll(commitInfo.getPartitionDirAndValsMap());
+            }
         });
-        return new FileAggregatedCommitInfo(aggregateCommitInfo);
+        return new FileAggregatedCommitInfo(aggregateCommitInfo, partitionDirAndValsMap);
     }
 
     @Override
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/FileSinkWriterWithTransaction.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/FileSinkWriterWithTransaction.java
new file mode 100644
index 000000000..83e51d1bc
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/FileSinkWriterWithTransaction.java
@@ -0,0 +1,160 @@
+/*
+ * 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.seatunnel.connectors.seatunnel.file.sink;
+
+import org.apache.seatunnel.api.sink.SinkWriter;
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.config.TextFileSinkConfig;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.SinkFileSystemPlugin;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.transaction.TransactionStateFileWriter;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.FileSinkPartitionDirNameGenerator;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.FileSinkTransactionFileNameGenerator;
+
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+
+import lombok.NonNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Optional;
+
+public class FileSinkWriterWithTransaction implements SinkWriter<SeaTunnelRow, FileCommitInfo, FileSinkState> {
+    private static final Logger LOGGER = LoggerFactory.getLogger(FileSinkWriterWithTransaction.class);
+
+    private SeaTunnelRowType seaTunnelRowTypeInfo;
+    private Config pluginConfig;
+    private Context context;
+    private String jobId;
+
+    private TransactionStateFileWriter fileWriter;
+
+    private TextFileSinkConfig textFileSinkConfig;
+
+    public FileSinkWriterWithTransaction(@NonNull SeaTunnelRowType seaTunnelRowTypeInfo,
+                                         @NonNull Config pluginConfig,
+                                         @NonNull SinkWriter.Context context,
+                                         @NonNull TextFileSinkConfig textFileSinkConfig,
+                                         @NonNull String jobId,
+                                         @NonNull SinkFileSystemPlugin sinkFileSystemPlugin) {
+        this.seaTunnelRowTypeInfo = seaTunnelRowTypeInfo;
+        this.pluginConfig = pluginConfig;
+        this.context = context;
+        this.jobId = jobId;
+        this.textFileSinkConfig = textFileSinkConfig;
+
+        Optional<TransactionStateFileWriter> transactionStateFileWriter = sinkFileSystemPlugin.getTransactionStateFileWriter(this.seaTunnelRowTypeInfo,
+            new FileSinkTransactionFileNameGenerator(
+                this.textFileSinkConfig.getFileFormat(),
+                this.textFileSinkConfig.getFileNameExpression(),
+                this.textFileSinkConfig.getFileNameTimeFormat()),
+            new FileSinkPartitionDirNameGenerator(
+                this.textFileSinkConfig.getPartitionFieldList(),
+                this.textFileSinkConfig.getPartitionFieldsIndexInRow(),
+                this.textFileSinkConfig.getPartitionDirExpression()),
+            this.textFileSinkConfig.getSinkColumnsIndexInRow(),
+            this.textFileSinkConfig.getTmpPath(),
+            this.textFileSinkConfig.getPath(),
+            this.jobId,
+            this.context.getIndexOfSubtask(),
+            this.textFileSinkConfig.getFieldDelimiter(),
+            this.textFileSinkConfig.getRowDelimiter(),
+            sinkFileSystemPlugin.getFileSystem().get());
+
+        if (!transactionStateFileWriter.isPresent()) {
+            throw new RuntimeException("A TransactionStateFileWriter is need");
+        }
+
+        this.fileWriter = transactionStateFileWriter.get();
+
+        fileWriter.beginTransaction(1L);
+    }
+
+    public FileSinkWriterWithTransaction(@NonNull SeaTunnelRowType seaTunnelRowTypeInfo,
+                                         @NonNull Config pluginConfig,
+                                         @NonNull SinkWriter.Context context,
+                                         @NonNull TextFileSinkConfig textFileSinkConfig,
+                                         @NonNull String jobId,
+                                         @NonNull List<FileSinkState> fileSinkStates,
+                                         @NonNull SinkFileSystemPlugin sinkFileSystemPlugin) {
+        this.seaTunnelRowTypeInfo = seaTunnelRowTypeInfo;
+        this.pluginConfig = pluginConfig;
+        this.context = context;
+        this.jobId = jobId;
+
+        Optional<TransactionStateFileWriter> transactionStateFileWriter = sinkFileSystemPlugin.getTransactionStateFileWriter(this.seaTunnelRowTypeInfo,
+            new FileSinkTransactionFileNameGenerator(
+                this.textFileSinkConfig.getFileFormat(),
+                this.textFileSinkConfig.getFileNameExpression(),
+                this.textFileSinkConfig.getFileNameTimeFormat()),
+            new FileSinkPartitionDirNameGenerator(
+                this.textFileSinkConfig.getPartitionFieldList(),
+                this.textFileSinkConfig.getPartitionFieldsIndexInRow(),
+                this.textFileSinkConfig.getPartitionDirExpression()),
+            this.textFileSinkConfig.getSinkColumnsIndexInRow(),
+            this.textFileSinkConfig.getTmpPath(),
+            this.textFileSinkConfig.getPath(),
+            this.jobId,
+            this.context.getIndexOfSubtask(),
+            this.textFileSinkConfig.getFieldDelimiter(),
+            this.textFileSinkConfig.getRowDelimiter(),
+            sinkFileSystemPlugin.getFileSystem().get());
+
+        if (!transactionStateFileWriter.isPresent()) {
+            throw new RuntimeException("A TransactionStateFileWriter is need");
+        }
+
+        this.fileWriter = transactionStateFileWriter.get();
+
+        // Rollback dirty transaction
+        if (fileSinkStates.size() > 0) {
+            List<String> transactionAfter = fileWriter.getTransactionAfter(fileSinkStates.get(0).getTransactionId());
+            fileWriter.abortTransactions(transactionAfter);
+        }
+        fileWriter.beginTransaction(fileSinkStates.get(0).getCheckpointId() + 1);
+    }
+
+    @Override
+    public void write(SeaTunnelRow element) throws IOException {
+        fileWriter.write(element);
+    }
+
+    @Override
+    public Optional<FileCommitInfo> prepareCommit() throws IOException {
+        return fileWriter.prepareCommit();
+    }
+
+    @Override
+    public void abortPrepare() {
+        fileWriter.abortTransaction();
+    }
+
+    @Override
+    public void close() throws IOException {
+        fileWriter.finishAndCloseWriteFile();
+    }
+
+    @Override
+    public List<FileSinkState> snapshotState(long checkpointId) throws IOException {
+        List<FileSinkState> fileSinkStates = fileWriter.snapshotState(checkpointId);
+        fileWriter.beginTransaction(checkpointId);
+        return fileSinkStates;
+    }
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/config/TextFileSinkConfig.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/config/TextFileSinkConfig.java
index 63b747bf2..ce94ff847 100644
--- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/config/TextFileSinkConfig.java
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/config/TextFileSinkConfig.java
@@ -19,10 +19,12 @@ package org.apache.seatunnel.connectors.seatunnel.file.sink.config;
 
 import static com.google.common.base.Preconditions.checkArgument;
 
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
 import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
-import org.apache.seatunnel.connectors.seatunnel.file.config.AbstractTextFileConfig;
+import org.apache.seatunnel.connectors.seatunnel.file.config.BaseTextFileConfig;
 import org.apache.seatunnel.connectors.seatunnel.file.config.Constant;
 import org.apache.seatunnel.connectors.seatunnel.file.config.PartitionConfig;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.FileSinkPartitionDirNameGenerator;
 
 import org.apache.seatunnel.shade.com.typesafe.config.Config;
 
@@ -38,14 +40,14 @@ import java.util.Map;
 import java.util.stream.Collectors;
 
 @Data
-public class TextFileSinkConfig extends AbstractTextFileConfig implements PartitionConfig {
+public class TextFileSinkConfig extends BaseTextFileConfig implements PartitionConfig {
 
     private List<String> sinkColumnList;
 
     private List<String> partitionFieldList;
 
     /**
-     * default is ${k1}=${v1}/${k2}=${v2}/...
+     * default is ${k0}=${v0}/${k1}=${v1}/... {@link FileSinkPartitionDirNameGenerator#generatorPartitionDir(SeaTunnelRow)} ()}
      */
     private String partitionDirExpression;
 
@@ -69,7 +71,7 @@ public class TextFileSinkConfig extends AbstractTextFileConfig implements Partit
         super(config);
         checkArgument(!CollectionUtils.isEmpty(Arrays.asList(seaTunnelRowTypeInfo.getFieldNames())));
 
-        if (config.hasPath(Constant.FILE_FORMAT) && !CollectionUtils.isEmpty(config.getStringList(Constant.SINK_COLUMNS))) {
+        if (config.hasPath(Constant.SINK_COLUMNS) && !CollectionUtils.isEmpty(config.getStringList(Constant.SINK_COLUMNS))) {
             this.sinkColumnList = config.getStringList(Constant.SINK_COLUMNS);
         }
 
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/transaction/Transaction.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/transaction/Transaction.java
index dd8d41bf6..d9a39c5df 100644
--- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/transaction/Transaction.java
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/transaction/Transaction.java
@@ -22,7 +22,6 @@ import org.apache.seatunnel.connectors.seatunnel.file.sink.AbstractFileSink;
 import org.apache.seatunnel.connectors.seatunnel.file.sink.FileCommitInfo;
 import org.apache.seatunnel.connectors.seatunnel.file.sink.FileSinkAggregatedCommitter;
 import org.apache.seatunnel.connectors.seatunnel.file.sink.FileSinkState;
-import org.apache.seatunnel.connectors.seatunnel.file.sink.TransactionStateFileSinkWriter;
 
 import lombok.NonNull;
 
@@ -40,7 +39,7 @@ public interface Transaction extends Serializable {
     String beginTransaction(@NonNull Long checkpointId);
 
     /**
-     * Abort current Transaction, called when {@link TransactionStateFileSinkWriter#prepareCommit()} or {@link TransactionStateFileSinkWriter#snapshotState(long)} failed
+     * Abort current Transaction, called when {@link org.apache.seatunnel.connectors.seatunnel.file.sink.TransactionStateFileSinkWriter#prepareCommit()} or {@link org.apache.seatunnel.connectors.seatunnel.file.sink.TransactionStateFileSinkWriter#snapshotState(long)} failed
      */
     void abortTransaction();
 
@@ -56,7 +55,7 @@ public interface Transaction extends Serializable {
     List<String> getTransactionAfter(@NonNull String transactionId);
 
     /**
-     * Called by {@link TransactionStateFileSinkWriter#prepareCommit()}
+     * Called by {@link org.apache.seatunnel.connectors.seatunnel.file.sink.TransactionStateFileSinkWriter#prepareCommit()}
      * We should end the transaction in this method. After this method is called, the transaction will no longer accept data writing
      *
      * @return Return the commit information that can be commit in {@link FileSinkAggregatedCommitter#commit(List)}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/AbstractTransactionStateFileWriter.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/AbstractTransactionStateFileWriter.java
index 784d87046..b14827b56 100644
--- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/AbstractTransactionStateFileWriter.java
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/AbstractTransactionStateFileWriter.java
@@ -66,6 +66,8 @@ public abstract class AbstractTransactionStateFileWriter implements TransactionS
 
     private FileSystem fileSystem;
 
+    private Map<String, List<String>> partitionDirAndValsMap;
+
     public AbstractTransactionStateFileWriter(@NonNull SeaTunnelRowType seaTunnelRowTypeInfo,
                                               @NonNull TransactionFileNameGenerator transactionFileNameGenerator,
                                               @NonNull PartitionDirNameGenerator partitionDirNameGenerator,
@@ -89,7 +91,8 @@ public abstract class AbstractTransactionStateFileWriter implements TransactionS
     }
 
     public String getOrCreateFilePathBeingWritten(@NonNull SeaTunnelRow seaTunnelRow) {
-        String beingWrittenFileKey = this.partitionDirNameGenerator.generatorPartitionDir(seaTunnelRow);
+        Map<String, List<String>> dataPartitionDirAndValsMap = this.partitionDirNameGenerator.generatorPartitionDir(seaTunnelRow);
+        String beingWrittenFileKey = dataPartitionDirAndValsMap.keySet().toArray()[0].toString();
         // get filePath from beingWrittenFile
         String beingWrittenFilePath = beingWrittenFile.get(beingWrittenFileKey);
         if (beingWrittenFilePath != null) {
@@ -99,6 +102,9 @@ public abstract class AbstractTransactionStateFileWriter implements TransactionS
             sbf.append("/").append(beingWrittenFileKey).append("/").append(transactionFileNameGenerator.generateFileName(this.transactionId));
             String newBeingWrittenFilePath = sbf.toString();
             beingWrittenFile.put(beingWrittenFileKey, newBeingWrittenFilePath);
+            if (!Constant.NON_PARTITION.equals(dataPartitionDirAndValsMap.keySet().toArray()[0].toString())){
+                partitionDirAndValsMap.putAll(dataPartitionDirAndValsMap);
+            }
             return newBeingWrittenFilePath;
         }
     }
@@ -114,6 +120,7 @@ public abstract class AbstractTransactionStateFileWriter implements TransactionS
         this.transactionId = "T" + Constant.TRANSACTION_ID_SPLIT + jobId + Constant.TRANSACTION_ID_SPLIT + subTaskIndex + Constant.TRANSACTION_ID_SPLIT + checkpointId;
         this.transactionDir = getTransactionDir(this.transactionId);
         this.needMoveFiles = new HashMap<>();
+        this.partitionDirAndValsMap = new HashMap<>();
         this.beingWrittenFile = new HashMap<>();
         this.beginTransaction(this.transactionId);
         this.checkpointId = checkpointId;
@@ -164,7 +171,10 @@ public abstract class AbstractTransactionStateFileWriter implements TransactionS
         // this.needMoveFiles will be clear when beginTransaction, so we need copy the needMoveFiles.
         Map<String, String> commitMap = new HashMap<>();
         commitMap.putAll(this.needMoveFiles);
-        return Optional.of(new FileCommitInfo(commitMap, this.transactionDir));
+
+        Map<String, List<String>> copyMap = this.partitionDirAndValsMap.entrySet().stream()
+            .collect(Collectors.toMap(e -> e.getKey(), e -> new ArrayList<String>(e.getValue())));
+        return Optional.of(new FileCommitInfo(commitMap, copyMap, this.transactionDir));
     }
 
     @Override
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/FileSinkPartitionDirNameGenerator.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/FileSinkPartitionDirNameGenerator.java
index aa8a3656b..a9175409f 100644
--- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/FileSinkPartitionDirNameGenerator.java
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/FileSinkPartitionDirNameGenerator.java
@@ -25,6 +25,7 @@ import lombok.Data;
 import org.apache.commons.collections4.CollectionUtils;
 import org.apache.commons.lang3.StringUtils;
 
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -59,11 +60,15 @@ public class FileSinkPartitionDirNameGenerator implements PartitionDirNameGenera
     }
 
     @Override
-    public String generatorPartitionDir(SeaTunnelRow seaTunnelRow) {
+    public Map<String, List<String>> generatorPartitionDir(SeaTunnelRow seaTunnelRow) {
+        Map<String, List<String>> partitionDirAndValsMap = new HashMap<>(1);
         if (CollectionUtils.isEmpty(this.partitionFieldsIndexInRow)) {
-            return Constant.NON_PARTITION;
+            partitionDirAndValsMap.put(Constant.NON_PARTITION, null);
+            return partitionDirAndValsMap;
         }
 
+        List<String> vals = new ArrayList<>(partitionFieldsIndexInRow.size());
+        String partitionDir;
         if (StringUtils.isBlank(partitionDirExpression)) {
             StringBuilder sbd = new StringBuilder();
             for (int i = 0; i < partitionFieldsIndexInRow.size(); i++) {
@@ -71,15 +76,20 @@ public class FileSinkPartitionDirNameGenerator implements PartitionDirNameGenera
                     .append("=")
                     .append(seaTunnelRow.getFields()[partitionFieldsIndexInRow.get(i)])
                     .append("/");
+                vals.add(seaTunnelRow.getFields()[partitionFieldsIndexInRow.get(i)].toString());
             }
-            return sbd.toString();
+            partitionDir = sbd.toString();
         } else {
             Map<String, String> valueMap = new HashMap<>(partitionFieldList.size() * 2);
             for (int i = 0; i < partitionFieldsIndexInRow.size(); i++) {
                 valueMap.put(keys[i], partitionFieldList.get(i));
                 valueMap.put(values[i], seaTunnelRow.getFields()[partitionFieldsIndexInRow.get(i)].toString());
+                vals.add(seaTunnelRow.getFields()[partitionFieldsIndexInRow.get(i)].toString());
             }
-            return VariablesSubstitute.substitute(partitionDirExpression, valueMap);
+            partitionDir = VariablesSubstitute.substitute(partitionDirExpression, valueMap);
         }
+
+        partitionDirAndValsMap.put(partitionDir, vals);
+        return partitionDirAndValsMap;
     }
 }
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/PartitionDirNameGenerator.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/PartitionDirNameGenerator.java
index 1145e847d..05c90256b 100644
--- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/PartitionDirNameGenerator.java
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/PartitionDirNameGenerator.java
@@ -20,7 +20,9 @@ package org.apache.seatunnel.connectors.seatunnel.file.sink.writer;
 import org.apache.seatunnel.api.table.type.SeaTunnelRow;
 
 import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
 
 public interface PartitionDirNameGenerator extends Serializable {
-    String generatorPartitionDir(SeaTunnelRow seaTunnelRow);
+    Map<String, List<String>> generatorPartitionDir(SeaTunnelRow seaTunnelRow);
 }
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/writer/TestFileSinkPartitionDirNameGenerator.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/writer/TestFileSinkPartitionDirNameGenerator.java
index 1989275f4..0867f104a 100644
--- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/writer/TestFileSinkPartitionDirNameGenerator.java
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/writer/TestFileSinkPartitionDirNameGenerator.java
@@ -55,15 +55,15 @@ public class TestFileSinkPartitionDirNameGenerator {
         partitionFieldsIndexInRow.add(3);
 
         PartitionDirNameGenerator partitionDirNameGenerator = new FileSinkPartitionDirNameGenerator(partitionFieldList, partitionFieldsIndexInRow, "${v0}/${v1}");
-        String partitionDir = partitionDirNameGenerator.generatorPartitionDir(seaTunnelRow);
+        String partitionDir = partitionDirNameGenerator.generatorPartitionDir(seaTunnelRow).keySet().toArray()[0].toString();
         Assert.assertEquals("test/3", partitionDir);
 
         partitionDirNameGenerator = new FileSinkPartitionDirNameGenerator(partitionFieldList, partitionFieldsIndexInRow, "${k0}=${v0}/${k1}=${v1}");
-        partitionDir = partitionDirNameGenerator.generatorPartitionDir(seaTunnelRow);
+        partitionDir = partitionDirNameGenerator.generatorPartitionDir(seaTunnelRow).keySet().toArray()[0].toString();
         Assert.assertEquals("c3=test/c4=3", partitionDir);
 
         partitionDirNameGenerator = new FileSinkPartitionDirNameGenerator(null, null, "${k0}=${v0}/${k1}=${v1}");
-        partitionDir = partitionDirNameGenerator.generatorPartitionDir(seaTunnelRow);
+        partitionDir = partitionDirNameGenerator.generatorPartitionDir(seaTunnelRow).keySet().toArray()[0].toString();
         Assert.assertEquals(Constant.NON_PARTITION, partitionDir);
     }
 }
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/FileSinkAggregatedCommitterTest.java b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/FileSinkAggregatedCommitterTest.java
index 79c54fcc9..3262ccd1c 100644
--- a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/FileSinkAggregatedCommitterTest.java
+++ b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/FileSinkAggregatedCommitterTest.java
@@ -25,12 +25,15 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
+import java.util.stream.Collectors;
 
 public class FileSinkAggregatedCommitterTest {
+    @SuppressWarnings("checkstyle:UnnecessaryParentheses")
     @Test
     public void testCommit() throws Exception {
         FileSinkAggregatedCommitter fileSinkAggregatedCommitter = new FileSinkAggregatedCommitter(new HdfsFileSystemCommitter());
@@ -46,7 +49,12 @@ public class FileSinkAggregatedCommitterTest {
         HdfsUtils.createFile(transactionDir + "/c3=4/c4=bbb/test1.txt");
 
         transactionFiles.put(transactionDir, needMoveFiles);
-        FileAggregatedCommitInfo fileAggregatedCommitInfo = new FileAggregatedCommitInfo(transactionFiles);
+
+        Map<String, List<String>> partitionDirAndVals = new HashMap<>();
+        partitionDirAndVals.put("/c3=4/c4=rrr", Arrays.stream((new String[]{"4", "rrr"})).collect(Collectors.toList()));
+        partitionDirAndVals.put("/c3=4/c4=bbb", Arrays.stream((new String[]{"4", "bbb"})).collect(Collectors.toList()));
+
+        FileAggregatedCommitInfo fileAggregatedCommitInfo = new FileAggregatedCommitInfo(transactionFiles, partitionDirAndVals);
         List<FileAggregatedCommitInfo> fileAggregatedCommitInfoList = new ArrayList<>();
         fileAggregatedCommitInfoList.add(fileAggregatedCommitInfo);
         fileSinkAggregatedCommitter.commit(fileAggregatedCommitInfoList);
@@ -56,7 +64,7 @@ public class FileSinkAggregatedCommitterTest {
         Assert.assertTrue(!HdfsUtils.fileExist(transactionDir));
     }
 
-    @SuppressWarnings("checkstyle:MagicNumber")
+    @SuppressWarnings("checkstyle:UnnecessaryParentheses")
     @Test
     public void testCombine() throws Exception {
         FileSinkAggregatedCommitter fileSinkAggregatedCommitter = new FileSinkAggregatedCommitter(new HdfsFileSystemCommitter());
@@ -66,28 +74,37 @@ public class FileSinkAggregatedCommitterTest {
         String transactionDir = String.format("/tmp/seatunnel/seatunnel/%s/T_%s_0_1", jobId, jobId);
         String targetDir = String.format("/tmp/hive/warehouse/%s", jobId);
         Map<String, String> needMoveFiles = new HashMap<>();
-        needMoveFiles.put(transactionDir + "/c3=4/c4=rrr/test1.txt", targetDir + "/c3=4/c4=rrr/test1.txt");
+        needMoveFiles.put(transactionDir + "/c3=3/c4=rrr/test1.txt", targetDir + "/c3=3/c4=rrr/test1.txt");
         needMoveFiles.put(transactionDir + "/c3=4/c4=bbb/test1.txt", targetDir + "/c3=4/c4=bbb/test1.txt");
-        HdfsUtils.createFile(transactionDir + "/c3=4/c4=rrr/test1.txt");
+        Map<String, List<String>> partitionDirAndVals = new HashMap<>();
+        partitionDirAndVals.put("/c3=3/c4=rrr", Arrays.stream((new String[]{"3", "rrr"})).collect(Collectors.toList()));
+        partitionDirAndVals.put("/c3=4/c4=bbb", Arrays.stream((new String[]{"4", "bbb"})).collect(Collectors.toList()));
+        FileCommitInfo fileCommitInfo = new FileCommitInfo(needMoveFiles, partitionDirAndVals, transactionDir);
+        HdfsUtils.createFile(transactionDir + "/c3=3/c4=rrr/test1.txt");
         HdfsUtils.createFile(transactionDir + "/c3=4/c4=bbb/test1.txt");
 
         Map<String, String> needMoveFiles1 = new HashMap<>();
         needMoveFiles1.put(transactionDir + "/c3=4/c4=rrr/test2.txt", targetDir + "/c3=4/c4=rrr/test2.txt");
         needMoveFiles1.put(transactionDir + "/c3=4/c4=bbb/test2.txt", targetDir + "/c3=4/c4=bbb/test2.txt");
-        FileCommitInfo fileCommitInfo = new FileCommitInfo(needMoveFiles, transactionDir);
-        FileCommitInfo fileCommitInfo1 = new FileCommitInfo(needMoveFiles1, transactionDir);
+        Map<String, List<String>> partitionDirAndVals1 = new HashMap<>();
+        partitionDirAndVals.put("/c3=4/c4=rrr", Arrays.stream((new String[]{"4", "rrr"})).collect(Collectors.toList()));
+        partitionDirAndVals.put("/c3=4/c4=bbb", Arrays.stream((new String[]{"4", "bbb"})).collect(Collectors.toList()));
+        FileCommitInfo fileCommitInfo1 = new FileCommitInfo(needMoveFiles1, partitionDirAndVals1, transactionDir);
         List<FileCommitInfo> fileCommitInfoList = new ArrayList<>();
         fileCommitInfoList.add(fileCommitInfo);
         fileCommitInfoList.add(fileCommitInfo1);
+
         FileAggregatedCommitInfo combine = fileSinkAggregatedCommitter.combine(fileCommitInfoList);
         Assert.assertEquals(1, combine.getTransactionMap().size());
         Assert.assertEquals(4, combine.getTransactionMap().get(transactionDir).size());
-        Assert.assertEquals(targetDir + "/c3=4/c4=rrr/test1.txt", combine.getTransactionMap().get(transactionDir).get(transactionDir + "/c3=4/c4=rrr/test1.txt"));
+        Assert.assertEquals(targetDir + "/c3=3/c4=rrr/test1.txt", combine.getTransactionMap().get(transactionDir).get(transactionDir + "/c3=3/c4=rrr/test1.txt"));
         Assert.assertEquals(targetDir + "/c3=4/c4=bbb/test1.txt", combine.getTransactionMap().get(transactionDir).get(transactionDir + "/c3=4/c4=bbb/test1.txt"));
         Assert.assertEquals(targetDir + "/c3=4/c4=rrr/test2.txt", combine.getTransactionMap().get(transactionDir).get(transactionDir + "/c3=4/c4=rrr/test2.txt"));
         Assert.assertEquals(targetDir + "/c3=4/c4=bbb/test2.txt", combine.getTransactionMap().get(transactionDir).get(transactionDir + "/c3=4/c4=bbb/test2.txt"));
+        Assert.assertEquals(3, combine.getPartitionDirAndValsMap().keySet().size());
     }
 
+    @SuppressWarnings("checkstyle:UnnecessaryParentheses")
     @Test
     public void testAbort() throws Exception {
         FileSinkAggregatedCommitter fileSinkAggregatedCommitter = new FileSinkAggregatedCommitter(new HdfsFileSystemCommitter());
@@ -99,11 +116,14 @@ public class FileSinkAggregatedCommitterTest {
         Map<String, String> needMoveFiles = new HashMap<>();
         needMoveFiles.put(transactionDir + "/c3=4/c4=rrr/test1.txt", targetDir + "/c3=4/c4=rrr/test1.txt");
         needMoveFiles.put(transactionDir + "/c3=4/c4=bbb/test1.txt", targetDir + "/c3=4/c4=bbb/test1.txt");
+        Map<String, List<String>> partitionDirAndVals = new HashMap<>();
+        partitionDirAndVals.put("/c3=4/c4=rrr", Arrays.stream((new String[]{"4", "rrr"})).collect(Collectors.toList()));
+        partitionDirAndVals.put("/c3=4/c4=bbb", Arrays.stream((new String[]{"4", "bbb"})).collect(Collectors.toList()));
         HdfsUtils.createFile(transactionDir + "/c3=4/c4=rrr/test1.txt");
         HdfsUtils.createFile(transactionDir + "/c3=4/c4=bbb/test1.txt");
 
         transactionFiles.put(transactionDir, needMoveFiles);
-        FileAggregatedCommitInfo fileAggregatedCommitInfo = new FileAggregatedCommitInfo(transactionFiles);
+        FileAggregatedCommitInfo fileAggregatedCommitInfo = new FileAggregatedCommitInfo(transactionFiles, partitionDirAndVals);
         List<FileAggregatedCommitInfo> fileAggregatedCommitInfoList = new ArrayList<>();
         fileAggregatedCommitInfoList.add(fileAggregatedCommitInfo);
         fileSinkAggregatedCommitter.commit(fileAggregatedCommitInfoList);
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/TestHdfsTxtTransactionStateFileWriter.java b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/TestHdfsTxtTransactionStateFileWriter.java
index f3e1847f4..d8633e86d 100644
--- a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/TestHdfsTxtTransactionStateFileWriter.java
+++ b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/TestHdfsTxtTransactionStateFileWriter.java
@@ -93,5 +93,15 @@ public class TestHdfsTxtTransactionStateFileWriter {
         Map<String, String> needMoveFiles = fileCommitInfo.getNeedMoveFiles();
         Assert.assertEquals(targetPath + "/c3=str1/c4=str2/" + transactionId + ".txt", needMoveFiles.get(transactionDir + "/c3=str1/c4=str2/" + transactionId + ".txt"));
         Assert.assertEquals(targetPath + "/c3=str1/c4=str3/" + transactionId + ".txt", needMoveFiles.get(transactionDir + "/c3=str1/c4=str3/" + transactionId + ".txt"));
+
+        Map<String, List<String>> partitionDirAndValsMap = fileCommitInfo.getPartitionDirAndValsMap();
+        Assert.assertEquals(2, partitionDirAndValsMap.size());
+        Assert.assertTrue(partitionDirAndValsMap.keySet().contains("c3=str1/c4=str2"));
+        Assert.assertTrue(partitionDirAndValsMap.keySet().contains("c3=str1/c4=str3"));
+        Assert.assertTrue(partitionDirAndValsMap.get("c3=str1/c4=str2").size() == 2);
+        Assert.assertEquals("str1", partitionDirAndValsMap.get("c3=str1/c4=str2").get(0));
+        Assert.assertEquals("str2", partitionDirAndValsMap.get("c3=str1/c4=str2").get(1));
+        Assert.assertEquals("str1", partitionDirAndValsMap.get("c3=str1/c4=str3").get(0));
+        Assert.assertEquals("str3", partitionDirAndValsMap.get("c3=str1/c4=str3").get(1));
     }
 }
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-local/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/sink/local/FileSinkAggregatedCommitterTest.java b/seatunnel-connectors-v2/connector-file/connector-file-local/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/sink/local/FileSinkAggregatedCommitterTest.java
index 89524aa5f..720ad3eba 100644
--- a/seatunnel-connectors-v2/connector-file/connector-file-local/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/sink/local/FileSinkAggregatedCommitterTest.java
+++ b/seatunnel-connectors-v2/connector-file/connector-file-local/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/sink/local/FileSinkAggregatedCommitterTest.java
@@ -25,13 +25,15 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
+import java.util.stream.Collectors;
 
 public class FileSinkAggregatedCommitterTest {
-    @Test
+    @SuppressWarnings("checkstyle:UnnecessaryParentheses")
     public void testCommit() throws Exception {
         FileSinkAggregatedCommitter fileSinkAggregatedCommitter = new FileSinkAggregatedCommitter(new LocalFileSystemCommitter());
         Map<String, Map<String, String>> transactionFiles = new HashMap<>();
@@ -47,7 +49,12 @@ public class FileSinkAggregatedCommitterTest {
         FileUtils.createFile(transactionDir + "/c3=4/c4=bbb/test1.txt");
 
         transactionFiles.put(transactionDir, needMoveFiles);
-        FileAggregatedCommitInfo fileAggregatedCommitInfo = new FileAggregatedCommitInfo(transactionFiles);
+
+        Map<String, List<String>> partitionDirAndVals = new HashMap<>();
+        partitionDirAndVals.put("/c3=4/c4=rrr", Arrays.stream((new String[]{"4", "rrr"})).collect(Collectors.toList()));
+        partitionDirAndVals.put("/c3=4/c4=bbb", Arrays.stream((new String[]{"4", "bbb"})).collect(Collectors.toList()));
+
+        FileAggregatedCommitInfo fileAggregatedCommitInfo = new FileAggregatedCommitInfo(transactionFiles, partitionDirAndVals);
         List<FileAggregatedCommitInfo> fileAggregatedCommitInfoList = new ArrayList<>();
         fileAggregatedCommitInfoList.add(fileAggregatedCommitInfo);
         fileSinkAggregatedCommitter.commit(fileAggregatedCommitInfoList);
@@ -57,7 +64,7 @@ public class FileSinkAggregatedCommitterTest {
         Assert.assertTrue(!FileUtils.fileExist(transactionDir));
     }
 
-    @SuppressWarnings("checkstyle:MagicNumber")
+    @SuppressWarnings("checkstyle:UnnecessaryParentheses")
     @Test
     public void testCombine() throws Exception {
         FileSinkAggregatedCommitter fileSinkAggregatedCommitter = new FileSinkAggregatedCommitter(new LocalFileSystemCommitter());
@@ -68,28 +75,37 @@ public class FileSinkAggregatedCommitterTest {
         String transactionDir = String.format("/tmp/seatunnel/seatunnel/%s/T_%s_0_1", jobId, jobId);
         String targetDir = String.format("/tmp/hive/warehouse/%s", jobId);
         Map<String, String> needMoveFiles = new HashMap<>();
-        needMoveFiles.put(transactionDir + "/c3=4/c4=rrr/test1.txt", targetDir + "/c3=4/c4=rrr/test1.txt");
+        needMoveFiles.put(transactionDir + "/c3=3/c4=rrr/test1.txt", targetDir + "/c3=3/c4=rrr/test1.txt");
         needMoveFiles.put(transactionDir + "/c3=4/c4=bbb/test1.txt", targetDir + "/c3=4/c4=bbb/test1.txt");
-        FileUtils.createFile(transactionDir + "/c3=4/c4=rrr/test1.txt");
+        Map<String, List<String>> partitionDirAndVals = new HashMap<>();
+        partitionDirAndVals.put("/c3=3/c4=rrr", Arrays.stream((new String[]{"3", "rrr"})).collect(Collectors.toList()));
+        partitionDirAndVals.put("/c3=4/c4=bbb", Arrays.stream((new String[]{"4", "bbb"})).collect(Collectors.toList()));
+        FileCommitInfo fileCommitInfo = new FileCommitInfo(needMoveFiles, partitionDirAndVals, transactionDir);
+        FileUtils.createFile(transactionDir + "/c3=3/c4=rrr/test1.txt");
         FileUtils.createFile(transactionDir + "/c3=4/c4=bbb/test1.txt");
 
         Map<String, String> needMoveFiles1 = new HashMap<>();
         needMoveFiles1.put(transactionDir + "/c3=4/c4=rrr/test2.txt", targetDir + "/c3=4/c4=rrr/test2.txt");
         needMoveFiles1.put(transactionDir + "/c3=4/c4=bbb/test2.txt", targetDir + "/c3=4/c4=bbb/test2.txt");
-        FileCommitInfo fileCommitInfo = new FileCommitInfo(needMoveFiles, transactionDir);
-        FileCommitInfo fileCommitInfo1 = new FileCommitInfo(needMoveFiles1, transactionDir);
+        Map<String, List<String>> partitionDirAndVals1 = new HashMap<>();
+        partitionDirAndVals.put("/c3=4/c4=rrr", Arrays.stream((new String[]{"4", "rrr"})).collect(Collectors.toList()));
+        partitionDirAndVals.put("/c3=4/c4=bbb", Arrays.stream((new String[]{"4", "bbb"})).collect(Collectors.toList()));
+        FileCommitInfo fileCommitInfo1 = new FileCommitInfo(needMoveFiles1, partitionDirAndVals1, transactionDir);
         List<FileCommitInfo> fileCommitInfoList = new ArrayList<>();
         fileCommitInfoList.add(fileCommitInfo);
         fileCommitInfoList.add(fileCommitInfo1);
+
         FileAggregatedCommitInfo combine = fileSinkAggregatedCommitter.combine(fileCommitInfoList);
         Assert.assertEquals(1, combine.getTransactionMap().size());
         Assert.assertEquals(4, combine.getTransactionMap().get(transactionDir).size());
-        Assert.assertEquals(targetDir + "/c3=4/c4=rrr/test1.txt", combine.getTransactionMap().get(transactionDir).get(transactionDir + "/c3=4/c4=rrr/test1.txt"));
+        Assert.assertEquals(targetDir + "/c3=3/c4=rrr/test1.txt", combine.getTransactionMap().get(transactionDir).get(transactionDir + "/c3=3/c4=rrr/test1.txt"));
         Assert.assertEquals(targetDir + "/c3=4/c4=bbb/test1.txt", combine.getTransactionMap().get(transactionDir).get(transactionDir + "/c3=4/c4=bbb/test1.txt"));
         Assert.assertEquals(targetDir + "/c3=4/c4=rrr/test2.txt", combine.getTransactionMap().get(transactionDir).get(transactionDir + "/c3=4/c4=rrr/test2.txt"));
         Assert.assertEquals(targetDir + "/c3=4/c4=bbb/test2.txt", combine.getTransactionMap().get(transactionDir).get(transactionDir + "/c3=4/c4=bbb/test2.txt"));
+        Assert.assertEquals(3, combine.getPartitionDirAndValsMap().keySet().size());
     }
 
+    @SuppressWarnings("checkstyle:UnnecessaryParentheses")
     @Test
     public void testAbort() throws Exception {
         FileSinkAggregatedCommitter fileSinkAggregatedCommitter = new FileSinkAggregatedCommitter(new LocalFileSystemCommitter());
@@ -102,18 +118,21 @@ public class FileSinkAggregatedCommitterTest {
         Map<String, String> needMoveFiles = new HashMap<>();
         needMoveFiles.put(transactionDir + "/c3=4/c4=rrr/test1.txt", targetDir + "/c3=4/c4=rrr/test1.txt");
         needMoveFiles.put(transactionDir + "/c3=4/c4=bbb/test1.txt", targetDir + "/c3=4/c4=bbb/test1.txt");
+        Map<String, List<String>> partitionDirAndVals = new HashMap<>();
+        partitionDirAndVals.put("/c3=4/c4=rrr", Arrays.stream((new String[]{"4", "rrr"})).collect(Collectors.toList()));
+        partitionDirAndVals.put("/c3=4/c4=bbb", Arrays.stream((new String[]{"4", "bbb"})).collect(Collectors.toList()));
         FileUtils.createFile(transactionDir + "/c3=4/c4=rrr/test1.txt");
         FileUtils.createFile(transactionDir + "/c3=4/c4=bbb/test1.txt");
 
         transactionFiles.put(transactionDir, needMoveFiles);
-        FileAggregatedCommitInfo fileAggregatedCommitInfo = new FileAggregatedCommitInfo(transactionFiles);
+        FileAggregatedCommitInfo fileAggregatedCommitInfo = new FileAggregatedCommitInfo(transactionFiles, partitionDirAndVals);
         List<FileAggregatedCommitInfo> fileAggregatedCommitInfoList = new ArrayList<>();
         fileAggregatedCommitInfoList.add(fileAggregatedCommitInfo);
         fileSinkAggregatedCommitter.commit(fileAggregatedCommitInfoList);
 
         Assert.assertTrue(FileUtils.fileExist(targetDir + "/c3=4/c4=bbb/test1.txt"));
         Assert.assertTrue(FileUtils.fileExist(targetDir + "/c3=4/c4=rrr/test1.txt"));
-        Assert.assertFalse(FileUtils.fileExist(transactionDir));
+        Assert.assertTrue(!FileUtils.fileExist(transactionDir));
 
         fileSinkAggregatedCommitter.abort(fileAggregatedCommitInfoList);
         Assert.assertTrue(!FileUtils.fileExist(targetDir + "/c3=4/c4=bbb/test1.txt"));
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-local/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/sink/local/TestLocalTxtTransactionStateFileWriter.java b/seatunnel-connectors-v2/connector-file/connector-file-local/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/sink/local/TestLocalTxtTransactionStateFileWriter.java
index 007ea39ef..d739ac41c 100644
--- a/seatunnel-connectors-v2/connector-file/connector-file-local/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/sink/local/TestLocalTxtTransactionStateFileWriter.java
+++ b/seatunnel-connectors-v2/connector-file/connector-file-local/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/sink/local/TestLocalTxtTransactionStateFileWriter.java
@@ -93,5 +93,15 @@ public class TestLocalTxtTransactionStateFileWriter {
         Map<String, String> needMoveFiles = fileCommitInfo.getNeedMoveFiles();
         Assert.assertEquals(targetPath + "/c3=str1/c4=str2/" + transactionId + ".txt", needMoveFiles.get(transactionDir + "/c3=str1/c4=str2/" + transactionId + ".txt"));
         Assert.assertEquals(targetPath + "/c3=str1/c4=str3/" + transactionId + ".txt", needMoveFiles.get(transactionDir + "/c3=str1/c4=str3/" + transactionId + ".txt"));
+
+        Map<String, List<String>> partitionDirAndValsMap = fileCommitInfo.getPartitionDirAndValsMap();
+        Assert.assertEquals(2, partitionDirAndValsMap.size());
+        Assert.assertTrue(partitionDirAndValsMap.keySet().contains("c3=str1/c4=str2"));
+        Assert.assertTrue(partitionDirAndValsMap.keySet().contains("c3=str1/c4=str3"));
+        Assert.assertTrue(partitionDirAndValsMap.get("c3=str1/c4=str2").size() == 2);
+        Assert.assertEquals("str1", partitionDirAndValsMap.get("c3=str1/c4=str2").get(0));
+        Assert.assertEquals("str2", partitionDirAndValsMap.get("c3=str1/c4=str2").get(1));
+        Assert.assertEquals("str1", partitionDirAndValsMap.get("c3=str1/c4=str3").get(0));
+        Assert.assertEquals("str3", partitionDirAndValsMap.get("c3=str1/c4=str3").get(1));
     }
 }
diff --git a/seatunnel-connectors-v2/connector-hive/pom.xml b/seatunnel-connectors-v2/connector-hive/pom.xml
index 62a2eed1f..a4842e0ed 100644
--- a/seatunnel-connectors-v2/connector-hive/pom.xml
+++ b/seatunnel-connectors-v2/connector-hive/pom.xml
@@ -31,9 +31,9 @@
 
     <dependencies>
         <dependency>
-            <groupId>org.apache.seatunnel</groupId>
-            <artifactId>seatunnel-hive-shade</artifactId>
-            <version>${project.version}</version>
+            <groupId>org.apache.hive</groupId>
+            <artifactId>hive-exec</artifactId>
+            <scope>provided</scope>
         </dependency>
         
         <dependency>
@@ -80,9 +80,31 @@
             <artifactId>commons-collections4</artifactId>
         </dependency>
 
+        <dependency>
+            <groupId>org.apache.seatunnel</groupId>
+            <artifactId>connector-file-hadoop</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.codehaus.janino</groupId>
+            <artifactId>janino</artifactId>
+            <version>${janino.version}</version>
+        </dependency>
+
         <dependency>
             <groupId>junit</groupId>
             <artifactId>junit</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.powermock</groupId>
+            <artifactId>powermock-module-junit4</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.powermock</groupId>
+            <artifactId>powermock-api-mockito2</artifactId>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 </project>
diff --git a/seatunnel-ui/src/store/theme/types.ts b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/Constant.java
similarity index 77%
copy from seatunnel-ui/src/store/theme/types.ts
copy to seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/Constant.java
index be5b77260..dd0b2ab54 100644
--- a/seatunnel-ui/src/store/theme/types.ts
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/Constant.java
@@ -1,4 +1,3 @@
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -16,8 +15,9 @@
  * limitations under the License.
  */
 
-interface ThemeState {
-  darkTheme: boolean
-}
+package org.apache.seatunnel.connectors.seatunnel.hive.config;
 
-export default ThemeState
\ No newline at end of file
+public class Constant {
+    public static final String HIVE_RESULT_TABLE_NAME = "hive_table_name";
+    public static final String HIVE_METASTORE_URIS = "hive_metastore_uris";
+}
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveAggregatedCommitInfo.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveAggregatedCommitInfo.java
index 025fbefbf..6259389c4 100644
--- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveAggregatedCommitInfo.java
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveAggregatedCommitInfo.java
@@ -17,20 +17,18 @@
 
 package org.apache.seatunnel.connectors.seatunnel.hive.sink;
 
+import org.apache.seatunnel.connectors.seatunnel.file.sink.FileAggregatedCommitInfo;
+
 import lombok.AllArgsConstructor;
 import lombok.Data;
+import org.apache.hadoop.hive.metastore.api.Table;
 
 import java.io.Serializable;
-import java.util.Map;
 
 @Data
 @AllArgsConstructor
 public class HiveAggregatedCommitInfo implements Serializable {
-
-    /**
-     * Storage the commit info in map.
-     * K is the file path need to be moved to hive data dir.
-     * V is the target file path of the data file.
-     */
-    private Map<String, String> needMoveFiles;
+    private FileAggregatedCommitInfo fileAggregatedCommitInfo;
+    private String hiveMetastoreUris;
+    private Table table;
 }
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveCommitInfo.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveCommitInfo.java
index 0dd58f8f4..002beea32 100644
--- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveCommitInfo.java
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveCommitInfo.java
@@ -17,20 +17,22 @@
 
 package org.apache.seatunnel.connectors.seatunnel.hive.sink;
 
+import org.apache.seatunnel.connectors.seatunnel.file.sink.FileCommitInfo;
+
 import lombok.AllArgsConstructor;
 import lombok.Data;
+import org.apache.hadoop.hive.metastore.api.Table;
 
 import java.io.Serializable;
-import java.util.Map;
 
 @Data
 @AllArgsConstructor
 public class HiveCommitInfo implements Serializable {
 
-    /**
-     * Storage the commit info in map.
-     * K is the file path need to be moved to hive data dir.
-     * V is the target file path of the data file.
-     */
-    private Map<String, String> needMoveFiles;
+    private FileCommitInfo fileCommitInfo;
+
+    private String hiveMetastoreUris;
+
+    private Table table;
+
 }
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSink.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSink.java
index f1ba12edd..4df91b1a5 100644
--- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSink.java
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSink.java
@@ -18,6 +18,7 @@
 package org.apache.seatunnel.connectors.seatunnel.hive.sink;
 
 import org.apache.seatunnel.api.common.PrepareFailException;
+import org.apache.seatunnel.api.common.SeaTunnelContext;
 import org.apache.seatunnel.api.serialization.DefaultSerializer;
 import org.apache.seatunnel.api.serialization.Serializer;
 import org.apache.seatunnel.api.sink.SeaTunnelSink;
@@ -26,6 +27,8 @@ import org.apache.seatunnel.api.sink.SinkWriter;
 import org.apache.seatunnel.api.table.type.SeaTunnelDataType;
 import org.apache.seatunnel.api.table.type.SeaTunnelRow;
 import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
+import org.apache.seatunnel.common.constants.JobMode;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.config.SaveMode;
 
 import org.apache.seatunnel.shade.com.typesafe.config.Config;
 
@@ -43,8 +46,11 @@ import java.util.Optional;
 public class HiveSink implements SeaTunnelSink<SeaTunnelRow, HiveSinkState, HiveCommitInfo, HiveAggregatedCommitInfo> {
 
     private Config config;
-    private long jobId;
-    private SeaTunnelRowType seaTunnelRowType;
+    private String jobId;
+    private Long checkpointId;
+    private SeaTunnelRowType seaTunnelRowTypeInfo;
+    private SeaTunnelContext seaTunnelContext;
+    private HiveSinkConfig hiveSinkConfig;
 
     @Override
     public String getPluginName() {
@@ -52,34 +58,47 @@ public class HiveSink implements SeaTunnelSink<SeaTunnelRow, HiveSinkState, Hive
     }
 
     @Override
-    public void setTypeInfo(SeaTunnelRowType seaTunnelRowType) {
-        this.seaTunnelRowType = seaTunnelRowType;
+    public void setTypeInfo(SeaTunnelRowType seaTunnelRowTypeInfo) {
+        this.seaTunnelRowTypeInfo = seaTunnelRowTypeInfo;
+        this.hiveSinkConfig = new HiveSinkConfig(config, seaTunnelRowTypeInfo);
     }
 
     @Override
     public SeaTunnelDataType<SeaTunnelRow> getConsumedType() {
-        return this.seaTunnelRowType;
+        return this.seaTunnelRowTypeInfo;
     }
 
     @Override
     public void prepare(Config pluginConfig) throws PrepareFailException {
         this.config = pluginConfig;
-        this.jobId = System.currentTimeMillis();
+        this.checkpointId = 1L;
     }
 
     @Override
     public SinkWriter<SeaTunnelRow, HiveCommitInfo, HiveSinkState> createWriter(SinkWriter.Context context) throws IOException {
-        return new HiveSinkWriter(seaTunnelRowType, config, context, System.currentTimeMillis());
+        if (!seaTunnelContext.getJobMode().equals(JobMode.BATCH) && hiveSinkConfig.getTextFileSinkConfig().getSaveMode().equals(SaveMode.OVERWRITE)) {
+            throw new RuntimeException("only batch job can overwrite hive table");
+        }
+
+        if (!this.getSinkConfig().getTextFileSinkConfig().isEnableTransaction()) {
+            throw new RuntimeException("Hive Sink Connector only support transaction now");
+        }
+        return new HiveSinkWriter(seaTunnelRowTypeInfo,
+            config,
+            context,
+            getSinkConfig(),
+            jobId);
     }
 
     @Override
     public SinkWriter<SeaTunnelRow, HiveCommitInfo, HiveSinkState> restoreWriter(SinkWriter.Context context, List<HiveSinkState> states) throws IOException {
-        return new HiveSinkWriter(seaTunnelRowType, config, context, System.currentTimeMillis());
+        return new HiveSinkWriter(seaTunnelRowTypeInfo, config, context, hiveSinkConfig, jobId, states);
     }
 
     @Override
-    public Optional<Serializer<HiveCommitInfo>> getCommitInfoSerializer() {
-        return Optional.of(new DefaultSerializer<>());
+    public void setSeaTunnelContext(SeaTunnelContext seaTunnelContext) {
+        this.seaTunnelContext = seaTunnelContext;
+        this.jobId = seaTunnelContext.getJobId();
     }
 
     @Override
@@ -87,8 +106,25 @@ public class HiveSink implements SeaTunnelSink<SeaTunnelRow, HiveSinkState, Hive
         return Optional.of(new HiveSinkAggregatedCommitter());
     }
 
+    @Override
+    public Optional<Serializer<HiveSinkState>> getWriterStateSerializer() {
+        return Optional.of(new DefaultSerializer<>());
+    }
+
     @Override
     public Optional<Serializer<HiveAggregatedCommitInfo>> getAggregatedCommitInfoSerializer() {
+        return Optional.of(new DefaultSerializer<HiveAggregatedCommitInfo>());
+    }
+
+    @Override
+    public Optional<Serializer<HiveCommitInfo>> getCommitInfoSerializer() {
         return Optional.of(new DefaultSerializer<>());
     }
+
+    private HiveSinkConfig getSinkConfig() {
+        if (this.hiveSinkConfig == null && (this.seaTunnelRowTypeInfo != null && this.config != null)) {
+            this.hiveSinkConfig = new HiveSinkConfig(config, seaTunnelRowTypeInfo);
+        }
+        return this.hiveSinkConfig;
+    }
 }
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkAggregatedCommitter.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkAggregatedCommitter.java
index 673923a51..3a0448179 100644
--- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkAggregatedCommitter.java
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkAggregatedCommitter.java
@@ -18,8 +18,14 @@
 package org.apache.seatunnel.connectors.seatunnel.hive.sink;
 
 import org.apache.seatunnel.api.sink.SinkAggregatedCommitter;
-import org.apache.seatunnel.connectors.seatunnel.hive.sink.file.writer.HdfsUtils;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.FileAggregatedCommitInfo;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.hdfs.HdfsUtils;
+import org.apache.seatunnel.connectors.seatunnel.hive.utils.HiveMetaStoreProxy;
 
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -28,41 +34,82 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 public class HiveSinkAggregatedCommitter implements SinkAggregatedCommitter<HiveCommitInfo, HiveAggregatedCommitInfo> {
     private static final Logger LOGGER = LoggerFactory.getLogger(HiveSinkAggregatedCommitter.class);
 
     @Override
     public List<HiveAggregatedCommitInfo> commit(List<HiveAggregatedCommitInfo> aggregatedCommitInfoList) throws IOException {
-        if (aggregatedCommitInfoList == null || aggregatedCommitInfoList.size() == 0) {
+        LOGGER.info("=============================agg commit=================================");
+        if (CollectionUtils.isEmpty(aggregatedCommitInfoList)) {
             return null;
         }
         List errorAggregatedCommitInfoList = new ArrayList();
-        aggregatedCommitInfoList.stream().forEach(aggregateCommitInfo -> {
-            try {
-                Map<String, String> needMoveFiles = aggregateCommitInfo.getNeedMoveFiles();
-                for (Map.Entry<String, String> entry : needMoveFiles.entrySet()) {
-                    HdfsUtils.renameFile(entry.getKey(), entry.getValue(), true);
+        HiveMetaStoreProxy hiveMetaStoreProxy = new HiveMetaStoreProxy(aggregatedCommitInfoList.get(0).getHiveMetastoreUris());
+        HiveMetaStoreClient hiveMetaStoreClient = hiveMetaStoreProxy.getHiveMetaStoreClient();
+        try {
+            aggregatedCommitInfoList.stream().forEach(aggregateCommitInfo -> {
+                try {
+                    for (Map.Entry<String, Map<String, String>> entry : aggregateCommitInfo.getFileAggregatedCommitInfo().getTransactionMap().entrySet()) {
+                        // rollback the file
+                        for (Map.Entry<String, String> mvFileEntry : entry.getValue().entrySet()) {
+                            HdfsUtils.renameFile(mvFileEntry.getKey(), mvFileEntry.getValue(), true);
+                        }
+                        // delete the transaction dir
+                        HdfsUtils.deleteFile(entry.getKey());
+                    }
+                    // add hive partition
+                    aggregateCommitInfo.getFileAggregatedCommitInfo().getPartitionDirAndValsMap().entrySet().forEach(entry -> {
+                        Partition part = new Partition();
+                        part.setDbName(aggregateCommitInfo.getTable().getDbName());
+                        part.setTableName(aggregateCommitInfo.getTable().getTableName());
+                        part.setValues(entry.getValue());
+                        part.setParameters(new HashMap<>());
+                        part.setSd(aggregateCommitInfo.getTable().getSd().deepCopy());
+                        part.getSd().setSerdeInfo(aggregateCommitInfo.getTable().getSd().getSerdeInfo());
+                        part.getSd().setLocation(aggregateCommitInfo.getTable().getSd().getLocation() + "/" + entry.getKey());
+                        try {
+                            hiveMetaStoreClient.add_partition(part);
+                        } catch (TException e) {
+                            throw new RuntimeException(e);
+                        }
+                    });
+                } catch (Exception e) {
+                    LOGGER.error("commit aggregateCommitInfo error ", e);
+                    errorAggregatedCommitInfoList.add(aggregateCommitInfo);
                 }
-            } catch (IOException e) {
-                LOGGER.error("commit aggregateCommitInfo error ", e);
-                errorAggregatedCommitInfoList.add(aggregateCommitInfo);
-            }
-        });
+            });
+        } finally {
+            hiveMetaStoreClient.close();
+        }
 
         return errorAggregatedCommitInfoList;
     }
 
     @Override
     public HiveAggregatedCommitInfo combine(List<HiveCommitInfo> commitInfos) {
-        if (commitInfos == null || commitInfos.size() == 0) {
+        if (CollectionUtils.isEmpty(commitInfos)) {
             return null;
         }
-        Map<String, String> aggregateCommitInfo = new HashMap<>();
+        Map<String, Map<String, String>> aggregateCommitInfo = new HashMap<>();
+        Map<String, List<String>> partitionDirAndValsMap = new HashMap<>();
         commitInfos.stream().forEach(commitInfo -> {
-            aggregateCommitInfo.putAll(commitInfo.getNeedMoveFiles());
+            Map<String, String> needMoveFileMap = aggregateCommitInfo.get(commitInfo.getFileCommitInfo().getTransactionDir());
+            if (needMoveFileMap == null) {
+                needMoveFileMap = new HashMap<>();
+                aggregateCommitInfo.put(commitInfo.getFileCommitInfo().getTransactionDir(), needMoveFileMap);
+            }
+            needMoveFileMap.putAll(commitInfo.getFileCommitInfo().getNeedMoveFiles());
+            Set<Map.Entry<String, List<String>>> entries = commitInfo.getFileCommitInfo().getPartitionDirAndValsMap().entrySet();
+            if (!CollectionUtils.isEmpty(entries)) {
+                partitionDirAndValsMap.putAll(commitInfo.getFileCommitInfo().getPartitionDirAndValsMap());
+            }
         });
-        return new HiveAggregatedCommitInfo(aggregateCommitInfo);
+        return new HiveAggregatedCommitInfo(
+            new FileAggregatedCommitInfo(aggregateCommitInfo, partitionDirAndValsMap),
+            commitInfos.get(0).getHiveMetastoreUris(),
+            commitInfos.get(0).getTable());
     }
 
     @Override
@@ -72,9 +119,17 @@ public class HiveSinkAggregatedCommitter implements SinkAggregatedCommitter<Hive
         }
         aggregatedCommitInfoList.stream().forEach(aggregateCommitInfo -> {
             try {
-                Map<String, String> needMoveFiles = aggregateCommitInfo.getNeedMoveFiles();
-                for (Map.Entry<String, String> entry : needMoveFiles.entrySet()) {
-                    HdfsUtils.renameFile(entry.getValue(), entry.getKey(), true);
+                for (Map.Entry<String, Map<String, String>> entry : aggregateCommitInfo.getFileAggregatedCommitInfo().getTransactionMap().entrySet()) {
+                    // rollback the file
+                    for (Map.Entry<String, String> mvFileEntry : entry.getValue().entrySet()) {
+                        if (HdfsUtils.fileExist(mvFileEntry.getValue()) && !HdfsUtils.fileExist(mvFileEntry.getKey())) {
+                            HdfsUtils.renameFile(mvFileEntry.getValue(), mvFileEntry.getKey(), true);
+                        }
+                    }
+                    // delete the transaction dir
+                    HdfsUtils.deleteFile(entry.getKey());
+
+                    // The partitions that have been added will be preserved and will not be deleted
                 }
             } catch (IOException e) {
                 LOGGER.error("abort aggregateCommitInfo error ", e);
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkConfig.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkConfig.java
index e4495121b..a37dd3a51 100644
--- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkConfig.java
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkConfig.java
@@ -17,90 +17,132 @@
 
 package org.apache.seatunnel.connectors.seatunnel.hive.sink;
 
+import static org.apache.seatunnel.connectors.seatunnel.file.config.Constant.FIELD_DELIMITER;
+import static org.apache.seatunnel.connectors.seatunnel.file.config.Constant.FILE_FORMAT;
+import static org.apache.seatunnel.connectors.seatunnel.file.config.Constant.FILE_NAME_EXPRESSION;
+import static org.apache.seatunnel.connectors.seatunnel.file.config.Constant.IS_PARTITION_FIELD_WRITE_IN_FILE;
+import static org.apache.seatunnel.connectors.seatunnel.file.config.Constant.PATH;
+import static org.apache.seatunnel.connectors.seatunnel.file.config.Constant.ROW_DELIMITER;
+import static org.apache.seatunnel.connectors.seatunnel.file.config.Constant.SAVE_MODE;
+import static org.apache.seatunnel.connectors.seatunnel.hive.config.Constant.HIVE_METASTORE_URIS;
+import static org.apache.seatunnel.connectors.seatunnel.hive.config.Constant.HIVE_RESULT_TABLE_NAME;
+import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
 
+import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
+import org.apache.seatunnel.connectors.seatunnel.file.config.Constant;
+import org.apache.seatunnel.connectors.seatunnel.file.config.FileFormat;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.config.SaveMode;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.config.TextFileSinkConfig;
+import org.apache.seatunnel.connectors.seatunnel.hive.utils.HiveMetaStoreProxy;
+
 import org.apache.seatunnel.shade.com.typesafe.config.Config;
+import org.apache.seatunnel.shade.com.typesafe.config.ConfigValueFactory;
 
 import lombok.Data;
 import lombok.NonNull;
-import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.thrift.TException;
 
+import java.io.Serializable;
+import java.util.Arrays;
 import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
 
 @Data
-public class HiveSinkConfig {
-
-    private static final String HIVE_SAVE_MODE = "save_mode";
-
-    private static final String HIVE_SINK_COLUMNS = "sink_columns";
-
-    private static final String HIVE_PARTITION_BY = "partition_by";
-
-    private static final String HIVE_RESULT_TABLE_NAME = "result_table_name";
+public class HiveSinkConfig implements Serializable {
+    private String hiveTableName;
+    private List<String> hivePartitionFieldList;
+    private String hiveMetaUris;
 
-    private static final String SINK_TMP_FS_ROOT_PATH = "sink_tmp_fs_root_path";
+    private String dbName;
 
-    private static final String HIVE_TABLE_FS_PATH = "hive_table_fs_path";
+    private String tableName;
 
-    private static final String HIVE_TXT_FILE_FIELD_DELIMITER = "hive_txt_file_field_delimiter";
+    private Table table;
 
-    private static final String HIVE_TXT_FILE_LINE_DELIMITER = "hive_txt_file_line_delimiter";
+    private TextFileSinkConfig textFileSinkConfig;
 
-    private SaveMode saveMode = SaveMode.APPEND;
+    public HiveSinkConfig(@NonNull Config config, @NonNull SeaTunnelRowType seaTunnelRowTypeInfo) {
+        checkArgument(!CollectionUtils.isEmpty(Arrays.asList(seaTunnelRowTypeInfo.getFieldNames())));
 
-    private String sinkTmpFsRootPath = "/tmp/seatunnel";
+        if (config.hasPath(HIVE_RESULT_TABLE_NAME) && !StringUtils.isBlank(config.getString(HIVE_RESULT_TABLE_NAME))) {
+            this.hiveTableName = config.getString(HIVE_RESULT_TABLE_NAME);
+        }
+        checkNotNull(hiveTableName);
 
-    private List<String> partitionFieldNames;
+        if (config.hasPath(HIVE_METASTORE_URIS) && !StringUtils.isBlank(config.getString(HIVE_METASTORE_URIS))) {
+            this.hiveMetaUris = config.getString(HIVE_METASTORE_URIS);
+        }
+        checkNotNull(hiveMetaUris);
 
-    private String hiveTableName;
+        String[] dbAndTableName = hiveTableName.split("\\.");
+        if (dbAndTableName == null || dbAndTableName.length != 2) {
+            throw new RuntimeException("Please config " + HIVE_RESULT_TABLE_NAME + " as db.table format");
+        }
+        this.dbName = dbAndTableName[0];
+        this.tableName = dbAndTableName[1];
+        HiveMetaStoreProxy hiveMetaStoreProxy = new HiveMetaStoreProxy(hiveMetaUris);
+        HiveMetaStoreClient hiveMetaStoreClient = hiveMetaStoreProxy.getHiveMetaStoreClient();
+
+        try {
+            table = hiveMetaStoreClient.getTable(dbName, tableName);
+            String inputFormat = table.getSd().getInputFormat();
+            if ("org.apache.hadoop.mapred.TextInputFormat".equals(inputFormat)) {
+                config = config.withValue(FILE_FORMAT, ConfigValueFactory.fromAnyRef(FileFormat.TEXT.toString()));
+            } else {
+                throw new RuntimeException("Only support text file now");
+            }
 
-    private List<String> sinkColumns;
+            Map<String, String> parameters = table.getSd().getSerdeInfo().getParameters();
+            config = config.withValue(IS_PARTITION_FIELD_WRITE_IN_FILE, ConfigValueFactory.fromAnyRef(false))
+                .withValue(FIELD_DELIMITER, ConfigValueFactory.fromAnyRef(parameters.get("field.delim")))
+                .withValue(ROW_DELIMITER, ConfigValueFactory.fromAnyRef(parameters.get("line.delim")))
+                .withValue(FILE_NAME_EXPRESSION, ConfigValueFactory.fromAnyRef("${transactionId}"))
+                .withValue(PATH, ConfigValueFactory.fromAnyRef(table.getSd().getLocation()));
 
-    private String hiveTableFsPath;
+            if (!config.hasPath(SAVE_MODE) || StringUtils.isBlank(config.getString(Constant.SAVE_MODE))) {
+                config = config.withValue(SAVE_MODE, ConfigValueFactory.fromAnyRef(SaveMode.APPEND.toString()));
+            }
 
-    private String hiveTxtFileFieldDelimiter = String.valueOf('\001');
+            this.textFileSinkConfig = new TextFileSinkConfig(config, seaTunnelRowTypeInfo);
 
-    private String hiveTxtFileLineDelimiter = "\n";
+            // --------------------Check textFileSinkConfig with the hive table info-------------------
+            List<FieldSchema> fields = hiveMetaStoreClient.getFields(dbAndTableName[0], dbAndTableName[1]);
+            List<FieldSchema> partitionKeys = table.getPartitionKeys();
 
-    public enum SaveMode {
-        APPEND(),
-        OVERWRITE();
+            // Remove partitionKeys from table fields
+            List<FieldSchema> fieldNotContainPartitionKey = fields.stream().filter(filed -> !partitionKeys.contains(filed)).collect(Collectors.toList());
 
-        public static SaveMode fromStr(String str) {
-            if ("overwrite".equals(str)) {
-                return OVERWRITE;
-            } else {
-                return APPEND;
+            // check fields size must same as sinkColumnList size
+            if (fieldNotContainPartitionKey.size() != textFileSinkConfig.getSinkColumnList().size()) {
+                throw new RuntimeException("sink columns size must same as hive table field size");
             }
-        }
-    }
-
-    public HiveSinkConfig(@NonNull Config pluginConfig) {
-        checkNotNull(pluginConfig.getString(HIVE_RESULT_TABLE_NAME));
-        checkNotNull(pluginConfig.getString(HIVE_TABLE_FS_PATH));
-        this.hiveTableName = pluginConfig.getString(HIVE_RESULT_TABLE_NAME);
-        this.hiveTableFsPath = pluginConfig.getString(HIVE_TABLE_FS_PATH);
 
-        this.saveMode = StringUtils.isBlank(pluginConfig.getString(HIVE_SAVE_MODE)) ? SaveMode.APPEND : SaveMode.fromStr(pluginConfig.getString(HIVE_SAVE_MODE));
-        if (!StringUtils.isBlank(pluginConfig.getString(SINK_TMP_FS_ROOT_PATH))) {
-            this.sinkTmpFsRootPath = pluginConfig.getString(SINK_TMP_FS_ROOT_PATH);
-        }
-
-        this.partitionFieldNames = pluginConfig.getStringList(HIVE_PARTITION_BY);
-        this.sinkColumns = pluginConfig.getStringList(HIVE_SINK_COLUMNS);
+            // check hivePartitionFieldList size must same as partitionFieldList size
+            if (partitionKeys.size() != textFileSinkConfig.getPartitionFieldList().size()) {
+                throw new RuntimeException("partition by columns size must same as hive table partition columns size");
+            }
 
-        if (!StringUtils.isBlank(pluginConfig.getString(HIVE_TXT_FILE_FIELD_DELIMITER))) {
-            this.hiveTxtFileFieldDelimiter = pluginConfig.getString(HIVE_TXT_FILE_FIELD_DELIMITER);
+            // --------------------Check textFileSinkConfig with the hive table info end----------------
+        } catch (TException e) {
+            throw new RuntimeException(e);
+        } finally {
+            hiveMetaStoreClient.close();
         }
 
-        if (!StringUtils.isBlank(pluginConfig.getString(HIVE_TXT_FILE_LINE_DELIMITER))) {
-            this.hiveTxtFileLineDelimiter = pluginConfig.getString(HIVE_TXT_FILE_LINE_DELIMITER);
+        // hive only support append or overwrite
+        if (!this.textFileSinkConfig.getSaveMode().equals(SaveMode.APPEND) && !this.textFileSinkConfig.getSaveMode().equals(SaveMode.OVERWRITE)) {
+            throw new RuntimeException("hive only support append or overwrite save mode");
         }
+    }
 
-        // partition fields must in sink columns
-        if (!CollectionUtils.isEmpty(this.sinkColumns) && !CollectionUtils.isEmpty(this.partitionFieldNames) && !this.sinkColumns.containsAll(this.partitionFieldNames)) {
-            throw new RuntimeException("partition fields must in sink columns");
-        }
+    public TextFileSinkConfig getTextFileSinkConfig() {
+        return textFileSinkConfig;
     }
 }
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkState.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkState.java
index 4f9f5d12e..a104151c3 100644
--- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkState.java
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkState.java
@@ -25,5 +25,6 @@ import java.io.Serializable;
 @Data
 @AllArgsConstructor
 public class HiveSinkState implements Serializable {
-    private HiveSinkConfig hiveSinkConfig;
+    private String transactionId;
+    private Long checkpointId;
 }
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkWriter.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkWriter.java
index e04f57729..4bdeae478 100644
--- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkWriter.java
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkWriter.java
@@ -20,48 +20,120 @@ package org.apache.seatunnel.connectors.seatunnel.hive.sink;
 import org.apache.seatunnel.api.sink.SinkWriter;
 import org.apache.seatunnel.api.table.type.SeaTunnelRow;
 import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
-import org.apache.seatunnel.connectors.seatunnel.hive.sink.file.writer.FileWriter;
-import org.apache.seatunnel.connectors.seatunnel.hive.sink.file.writer.HdfsTxtFileWriter;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.FileCommitInfo;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.FileSinkState;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.hdfs.HdfsFileSinkPlugin;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.SinkFileSystemPlugin;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.transaction.TransactionStateFileWriter;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.FileSinkPartitionDirNameGenerator;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.FileSinkTransactionFileNameGenerator;
 
 import org.apache.seatunnel.shade.com.typesafe.config.Config;
 
-import com.google.common.collect.Lists;
 import lombok.NonNull;
+import org.apache.commons.collections4.CollectionUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 import java.util.Optional;
+import java.util.stream.Collectors;
 
 public class HiveSinkWriter implements SinkWriter<SeaTunnelRow, HiveCommitInfo, HiveSinkState> {
     private static final Logger LOGGER = LoggerFactory.getLogger(HiveSinkWriter.class);
 
-    private SeaTunnelRowType seaTunnelRowType;
+    private SeaTunnelRowType seaTunnelRowTypeInfo;
     private Config pluginConfig;
-    private SinkWriter.Context context;
-    private long jobId;
+    private Context context;
+    private String jobId;
 
-    private FileWriter fileWriter;
+    private TransactionStateFileWriter fileWriter;
 
     private HiveSinkConfig hiveSinkConfig;
 
-    public HiveSinkWriter(@NonNull SeaTunnelRowType seaTunnelRowType,
+    public HiveSinkWriter(@NonNull SeaTunnelRowType seaTunnelRowTypeInfo,
                           @NonNull Config pluginConfig,
                           @NonNull SinkWriter.Context context,
-                          long jobId) {
-        this.seaTunnelRowType = seaTunnelRowType;
+                          @NonNull HiveSinkConfig hiveSinkConfig,
+                          @NonNull String jobId) {
+        this.seaTunnelRowTypeInfo = seaTunnelRowTypeInfo;
         this.pluginConfig = pluginConfig;
         this.context = context;
         this.jobId = jobId;
+        this.hiveSinkConfig = hiveSinkConfig;
+
+        SinkFileSystemPlugin sinkFileSystemPlugin = new HdfsFileSinkPlugin();
+        Optional<TransactionStateFileWriter> transactionStateFileWriter = sinkFileSystemPlugin.getTransactionStateFileWriter(this.seaTunnelRowTypeInfo,
+            new FileSinkTransactionFileNameGenerator(
+                this.hiveSinkConfig.getTextFileSinkConfig().getFileFormat(),
+                this.hiveSinkConfig.getTextFileSinkConfig().getFileNameExpression(),
+                this.hiveSinkConfig.getTextFileSinkConfig().getFileNameTimeFormat()),
+            new FileSinkPartitionDirNameGenerator(
+                this.hiveSinkConfig.getTextFileSinkConfig().getPartitionFieldList(),
+                this.hiveSinkConfig.getTextFileSinkConfig().getPartitionFieldsIndexInRow(),
+                this.hiveSinkConfig.getTextFileSinkConfig().getPartitionDirExpression()),
+            this.hiveSinkConfig.getTextFileSinkConfig().getSinkColumnsIndexInRow(),
+            this.hiveSinkConfig.getTextFileSinkConfig().getTmpPath(),
+            this.hiveSinkConfig.getTextFileSinkConfig().getPath(),
+            this.jobId,
+            this.context.getIndexOfSubtask(),
+            this.hiveSinkConfig.getTextFileSinkConfig().getFieldDelimiter(),
+            this.hiveSinkConfig.getTextFileSinkConfig().getRowDelimiter(),
+            sinkFileSystemPlugin.getFileSystem().get());
+
+        if (!transactionStateFileWriter.isPresent()) {
+            throw new RuntimeException("A TransactionStateFileWriter is need");
+        }
+
+        this.fileWriter = transactionStateFileWriter.get();
+
+        fileWriter.beginTransaction(1L);
+    }
 
-        hiveSinkConfig = new HiveSinkConfig(this.pluginConfig);
-        fileWriter = new HdfsTxtFileWriter(this.seaTunnelRowType,
-            hiveSinkConfig,
+    public HiveSinkWriter(@NonNull SeaTunnelRowType seaTunnelRowTypeInfo,
+                          @NonNull Config pluginConfig,
+                          @NonNull SinkWriter.Context context,
+                          @NonNull HiveSinkConfig hiveSinkConfig,
+                          @NonNull String jobId,
+                          @NonNull List<HiveSinkState> hiveSinkStates) {
+        this.seaTunnelRowTypeInfo = seaTunnelRowTypeInfo;
+        this.pluginConfig = pluginConfig;
+        this.context = context;
+        this.jobId = jobId;
+        this.hiveSinkConfig = hiveSinkConfig;
+
+        SinkFileSystemPlugin sinkFileSystemPlugin = new HdfsFileSinkPlugin();
+        Optional<TransactionStateFileWriter> transactionStateFileWriter = sinkFileSystemPlugin.getTransactionStateFileWriter(this.seaTunnelRowTypeInfo,
+            new FileSinkTransactionFileNameGenerator(
+                this.hiveSinkConfig.getTextFileSinkConfig().getFileFormat(),
+                this.hiveSinkConfig.getTextFileSinkConfig().getFileNameExpression(),
+                this.hiveSinkConfig.getTextFileSinkConfig().getFileNameTimeFormat()),
+            new FileSinkPartitionDirNameGenerator(
+                this.hiveSinkConfig.getTextFileSinkConfig().getPartitionFieldList(),
+                this.hiveSinkConfig.getTextFileSinkConfig().getPartitionFieldsIndexInRow(),
+                this.hiveSinkConfig.getTextFileSinkConfig().getPartitionDirExpression()),
+            this.hiveSinkConfig.getTextFileSinkConfig().getSinkColumnsIndexInRow(),
+            this.hiveSinkConfig.getTextFileSinkConfig().getTmpPath(),
+            this.hiveSinkConfig.getTextFileSinkConfig().getPath(),
             this.jobId,
-            this.context.getIndexOfSubtask());
+            this.context.getIndexOfSubtask(),
+            this.hiveSinkConfig.getTextFileSinkConfig().getFieldDelimiter(),
+            this.hiveSinkConfig.getTextFileSinkConfig().getRowDelimiter(),
+            sinkFileSystemPlugin.getFileSystem().get());
+
+        if (!transactionStateFileWriter.isPresent()) {
+            throw new RuntimeException("A TransactionStateFileWriter is need");
+        }
+
+        this.fileWriter = transactionStateFileWriter.get();
+
+        // Rollback dirty transaction
+        if (hiveSinkStates.size() > 0) {
+            List<String> transactionAfter = fileWriter.getTransactionAfter(hiveSinkStates.get(0).getTransactionId());
+            fileWriter.abortTransactions(transactionAfter);
+        }
+        fileWriter.beginTransaction(hiveSinkStates.get(0).getCheckpointId() + 1);
     }
 
     @Override
@@ -71,18 +143,12 @@ public class HiveSinkWriter implements SinkWriter<SeaTunnelRow, HiveCommitInfo,
 
     @Override
     public Optional<HiveCommitInfo> prepareCommit() throws IOException {
-        fileWriter.finishAndCloseWriteFile();
-        /**
-         * We will clear the needMoveFiles in {@link #snapshotState()}, So we need copy the needMoveFiles map here.
-         */
-        Map<String, String> commitInfoMap = new HashMap<>(fileWriter.getNeedMoveFiles().size());
-        commitInfoMap.putAll(fileWriter.getNeedMoveFiles());
-        return Optional.of(new HiveCommitInfo(commitInfoMap));
-    }
-
-    @Override
-    public void abortPrepare() {
-        fileWriter.abort();
+        Optional<FileCommitInfo> fileCommitInfoOptional = fileWriter.prepareCommit();
+        if (fileCommitInfoOptional.isPresent()) {
+            FileCommitInfo fileCommitInfo = fileCommitInfoOptional.get();
+            return Optional.of(new HiveCommitInfo(fileCommitInfo, hiveSinkConfig.getHiveMetaUris(), this.hiveSinkConfig.getTable()));
+        }
+        return Optional.empty();
     }
 
     @Override
@@ -92,8 +158,17 @@ public class HiveSinkWriter implements SinkWriter<SeaTunnelRow, HiveCommitInfo,
 
     @Override
     public List<HiveSinkState> snapshotState(long checkpointId) throws IOException {
-        //reset FileWrite
-        fileWriter.resetFileWriter(System.currentTimeMillis() + "");
-        return Lists.newArrayList(new HiveSinkState(hiveSinkConfig));
+        List<FileSinkState> fileSinkStates = fileWriter.snapshotState(checkpointId);
+        if (!CollectionUtils.isEmpty(fileSinkStates)) {
+            return fileSinkStates.stream().map(state ->
+                    new HiveSinkState(state.getTransactionId(), state.getCheckpointId()))
+                .collect(Collectors.toList());
+        }
+        return null;
+    }
+
+    @Override
+    public void abortPrepare() {
+        fileWriter.abortTransaction();
     }
 }
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/file/writer/AbstractFileWriter.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/file/writer/AbstractFileWriter.java
deleted file mode 100644
index 57ca3c901..000000000
--- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/file/writer/AbstractFileWriter.java
+++ /dev/null
@@ -1,155 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.seatunnel.connectors.seatunnel.hive.sink.file.writer;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import org.apache.seatunnel.api.table.type.SeaTunnelRow;
-import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
-import org.apache.seatunnel.connectors.seatunnel.hive.sink.HiveSinkConfig;
-
-import lombok.NonNull;
-
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.stream.Collectors;
-import java.util.stream.IntStream;
-
-public abstract class AbstractFileWriter implements FileWriter {
-    protected Map<String, String> needMoveFiles;
-    protected SeaTunnelRowType seaTunnelRowType;
-    protected long jobId;
-    protected int subTaskIndex;
-    protected HiveSinkConfig hiveSinkConfig;
-
-    private static final String SEATUNNEL = "seatunnel";
-    private static final String NON_PARTITION = "NON_PARTITION";
-
-    protected Map<String, String> beingWrittenFile;
-
-    protected String checkpointId;
-    protected final int[] partitionKeyIndexes;
-
-    public AbstractFileWriter(@NonNull SeaTunnelRowType seaTunnelRowType,
-                              @NonNull HiveSinkConfig hiveSinkConfig,
-                              long jobId,
-                              int subTaskIndex) {
-        checkArgument(jobId > 0);
-        checkArgument(subTaskIndex > -1);
-
-        this.needMoveFiles = new HashMap<>();
-        this.seaTunnelRowType = seaTunnelRowType;
-        this.jobId = jobId;
-        this.subTaskIndex = subTaskIndex;
-        this.hiveSinkConfig = hiveSinkConfig;
-
-        this.beingWrittenFile = new HashMap<>();
-        if (this.hiveSinkConfig.getPartitionFieldNames() == null) {
-            this.partitionKeyIndexes = new int[0];
-        } else {
-            this.partitionKeyIndexes = IntStream.range(0, seaTunnelRowType.getTotalFields())
-                .filter(i -> hiveSinkConfig.getPartitionFieldNames().contains(seaTunnelRowType.getFieldName(i)))
-                .toArray();
-        }
-    }
-
-    public String getOrCreateFilePathBeingWritten(@NonNull SeaTunnelRow seaTunnelRow) {
-        String beingWrittenFileKey = getBeingWrittenFileKey(seaTunnelRow);
-        // get filePath from beingWrittenFile
-        String beingWrittenFilePath = beingWrittenFile.get(beingWrittenFileKey);
-        if (beingWrittenFilePath != null) {
-            return beingWrittenFilePath;
-        } else {
-            StringBuilder sbf = new StringBuilder(hiveSinkConfig.getSinkTmpFsRootPath());
-            sbf.append("/")
-                .append(SEATUNNEL)
-                .append("/")
-                .append(jobId)
-                .append("/")
-                .append(checkpointId)
-                .append("/")
-                .append(hiveSinkConfig.getHiveTableName())
-                .append("/")
-                .append(beingWrittenFileKey)
-                .append("/")
-                .append(jobId)
-                .append("_")
-                .append(subTaskIndex)
-                .append(".")
-                .append(getFileSuffix());
-            String newBeingWrittenFilePath = sbf.toString();
-            beingWrittenFile.put(beingWrittenFileKey, newBeingWrittenFilePath);
-            return newBeingWrittenFilePath;
-        }
-    }
-
-    private String getBeingWrittenFileKey(@NonNull SeaTunnelRow seaTunnelRow) {
-        if (partitionKeyIndexes.length > 0) {
-            return Arrays.stream(partitionKeyIndexes)
-                .boxed()
-                .map(i -> seaTunnelRowType.getFieldName(i) + "=" + seaTunnelRow.getField(i))
-                .collect(Collectors.joining("/"));
-        } else {
-            // If there is no partition field in data, We use the fixed value NON_PARTITION as the partition directory
-            return NON_PARTITION;
-        }
-    }
-
-    /**
-     * FileWriter need return the file suffix. eg: tex, orc, parquet
-     *
-     * @return
-     */
-    @NonNull
-    public abstract String getFileSuffix();
-
-    public String getHiveLocation(@NonNull String seaTunnelFilePath) {
-        StringBuilder sbf = new StringBuilder(hiveSinkConfig.getSinkTmpFsRootPath());
-        sbf.append("/")
-            .append(SEATUNNEL)
-            .append("/")
-            .append(jobId)
-            .append("/")
-            .append(checkpointId)
-            .append("/")
-            .append(hiveSinkConfig.getHiveTableName());
-        String seaTunnelPath = sbf.toString();
-        String tmpPath = seaTunnelFilePath.replaceAll(seaTunnelPath, hiveSinkConfig.getHiveTableFsPath());
-        return tmpPath.replaceAll(NON_PARTITION + "/", "");
-    }
-
-    @Override
-    public void resetFileWriter(@NonNull String checkpointId) {
-        this.checkpointId = checkpointId;
-        this.needMoveFiles = new HashMap<>();
-        this.beingWrittenFile = new HashMap<>();
-        this.resetMoreFileWriter(checkpointId);
-    }
-
-    public abstract void resetMoreFileWriter(@NonNull String checkpointId);
-
-    @Override
-    public void abort() {
-        this.needMoveFiles = new HashMap<>();
-        this.beingWrittenFile = new HashMap<>();
-        this.abortMore();
-    }
-
-    public abstract void abortMore();
-}
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/file/writer/FileWriter.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/file/writer/FileWriter.java
deleted file mode 100644
index 8ee8777a1..000000000
--- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/file/writer/FileWriter.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.seatunnel.connectors.seatunnel.hive.sink.file.writer;
-
-import org.apache.seatunnel.api.table.type.SeaTunnelRow;
-
-import lombok.NonNull;
-
-import java.util.Map;
-
-public interface FileWriter {
-
-    void write(@NonNull SeaTunnelRow seaTunnelRow);
-
-    @NonNull
-    Map<String, String> getNeedMoveFiles();
-
-    /**
-     * In this method we need finish write the file. The following operations are often required:
-     * 1. Flush memory to disk.
-     * 2. Close output stream.
-     * 3. Add the mapping relationship between seatunnel file path and hive file path to needMoveFiles.
-     */
-    void finishAndCloseWriteFile();
-
-    /**
-     * The writer needs to be reset after each checkpoint is completed
-     *
-     * @param checkpointId checkpointId
-     */
-    void resetFileWriter(@NonNull String checkpointId);
-
-    void abort();
-}
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/file/writer/HdfsTxtFileWriter.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/file/writer/HdfsTxtFileWriter.java
deleted file mode 100644
index 71b26568f..000000000
--- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/file/writer/HdfsTxtFileWriter.java
+++ /dev/null
@@ -1,151 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.seatunnel.connectors.seatunnel.hive.sink.file.writer;
-
-import org.apache.seatunnel.api.table.type.SeaTunnelRow;
-import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
-import org.apache.seatunnel.connectors.seatunnel.hive.sink.HiveSinkConfig;
-
-import lombok.Lombok;
-import lombok.NonNull;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Collectors;
-import java.util.stream.IntStream;
-
-public class HdfsTxtFileWriter extends AbstractFileWriter {
-    private static final Logger LOGGER = LoggerFactory.getLogger(HdfsTxtFileWriter.class);
-    private Map<String, FSDataOutputStream> beingWrittenOutputStream;
-    protected final int[] sinkColumnIndexes;
-
-    public HdfsTxtFileWriter(SeaTunnelRowType seaTunnelRowType,
-                             HiveSinkConfig hiveSinkConfig,
-                             long sinkId,
-                             int subTaskIndex) {
-        super(seaTunnelRowType, hiveSinkConfig, sinkId, subTaskIndex);
-        beingWrittenOutputStream = new HashMap<>();
-        List<String> sinkColumns = hiveSinkConfig.getSinkColumns();
-        if (sinkColumns == null || sinkColumns.size() == 0) {
-            this.sinkColumnIndexes = IntStream.range(0, seaTunnelRowType.getTotalFields()).toArray();
-        } else {
-            this.sinkColumnIndexes = IntStream.range(0, seaTunnelRowType.getTotalFields())
-                .filter(i -> sinkColumns.contains(seaTunnelRowType.getFieldName(i)))
-                .toArray();
-        }
-    }
-
-    @Override
-    @NonNull
-    public String getFileSuffix() {
-        return "txt";
-    }
-
-    @Override
-    public void resetMoreFileWriter(@NonNull String checkpointId) {
-        this.beingWrittenOutputStream = new HashMap<>();
-    }
-
-    @Override
-    public void abortMore() {
-        // delete files
-        beingWrittenOutputStream.keySet().stream().forEach(file -> {
-            try {
-                boolean deleted = HdfsUtils.deleteFile(file);
-                if (!deleted) {
-                    LOGGER.error("delete file {} error", file);
-                    throw new IOException(String.format("delete file {} error", file));
-                }
-            } catch (IOException e) {
-                LOGGER.error("delete file {} error", file);
-                throw new RuntimeException(e);
-            }
-        });
-
-        this.beingWrittenOutputStream = new HashMap<>();
-    }
-
-    @Override
-    public void write(@NonNull SeaTunnelRow seaTunnelRow) {
-        Lombok.checkNotNull(seaTunnelRow, "seaTunnelRow is null");
-        String filePath = getOrCreateFilePathBeingWritten(seaTunnelRow);
-        FSDataOutputStream fsDataOutputStream = getOrCreateOutputStream(filePath);
-        String line = transformRowToLine(seaTunnelRow);
-        try {
-            fsDataOutputStream.write(line.getBytes());
-            fsDataOutputStream.write(hiveSinkConfig.getHiveTxtFileLineDelimiter().getBytes());
-        } catch (IOException e) {
-            LOGGER.error("write data to file {} error", filePath);
-            throw new RuntimeException(e);
-        }
-    }
-
-    @NonNull
-    @Override
-    public Map<String, String> getNeedMoveFiles() {
-        return this.needMoveFiles;
-    }
-
-    @Override
-    public void finishAndCloseWriteFile() {
-        beingWrittenOutputStream.entrySet().forEach(entry -> {
-            try {
-                entry.getValue().flush();
-            } catch (IOException e) {
-                LOGGER.error("error when flush file {}", entry.getKey());
-                throw new RuntimeException(e);
-            } finally {
-                try {
-                    entry.getValue().close();
-                } catch (IOException e) {
-                    LOGGER.error("error when close output stream {}", entry.getKey());
-                }
-            }
-
-            needMoveFiles.put(entry.getKey(), getHiveLocation(entry.getKey()));
-        });
-    }
-
-    private FSDataOutputStream getOrCreateOutputStream(@NonNull String filePath) {
-        FSDataOutputStream fsDataOutputStream = beingWrittenOutputStream.get(filePath);
-        if (fsDataOutputStream == null) {
-            try {
-                fsDataOutputStream = HdfsUtils.getOutputStream(filePath);
-                beingWrittenOutputStream.put(filePath, fsDataOutputStream);
-            } catch (IOException e) {
-                LOGGER.error("can not get output file stream");
-                throw new RuntimeException(e);
-            }
-        }
-        return fsDataOutputStream;
-    }
-
-    private String transformRowToLine(@NonNull SeaTunnelRow seaTunnelRow) {
-        return Arrays.stream(sinkColumnIndexes)
-            .boxed()
-            .map(seaTunnelRow::getField)
-            .map(value -> value == null ? "" : value.toString())
-            .collect(Collectors.joining(hiveSinkConfig.getHiveTxtFileFieldDelimiter()));
-    }
-}
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/file/writer/HdfsUtils.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/file/writer/HdfsUtils.java
deleted file mode 100644
index 23b1e5843..000000000
--- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/file/writer/HdfsUtils.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.seatunnel.connectors.seatunnel.hive.sink.file.writer;
-
-import lombok.NonNull;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-
-public class HdfsUtils {
-    private static final Logger LOGGER = LoggerFactory.getLogger(HdfsUtils.class);
-
-    public static final int WRITE_BUFFER_SIZE = 2048;
-
-    public static FileSystem getHdfsFs(@NonNull String path)
-        throws IOException {
-        Configuration conf = new Configuration();
-        conf.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem");
-        conf.set("fs.defaultFs", path);
-        return FileSystem.get(conf);
-    }
-
-    public static FSDataOutputStream getOutputStream(@NonNull String outFilePath) throws IOException {
-        FileSystem hdfsFs = getHdfsFs(outFilePath);
-        Path path = new Path(outFilePath);
-        FSDataOutputStream fsDataOutputStream = hdfsFs.create(path, true, WRITE_BUFFER_SIZE);
-        return fsDataOutputStream;
-    }
-
-    public static boolean deleteFile(@NonNull String file) throws IOException {
-        FileSystem hdfsFs = getHdfsFs(file);
-        return hdfsFs.delete(new Path(file), true);
-    }
-
-    /**
-     * rename file
-     *
-     * @param oldName     old file name
-     * @param newName     target file name
-     * @param rmWhenExist if this is true, we will delete the target file when it already exists
-     * @throws IOException throw IOException
-     */
-    public static void renameFile(@NonNull String oldName, @NonNull String newName, boolean rmWhenExist) throws IOException {
-        FileSystem hdfsFs = getHdfsFs(newName);
-        LOGGER.info("begin rename file oldName :[" + oldName + "] to newName :[" + newName + "]");
-
-        Path oldPath = new Path(oldName);
-        Path newPath = new Path(newName);
-        if (rmWhenExist) {
-            if (fileExist(newName) && fileExist(oldName)) {
-                hdfsFs.delete(newPath, true);
-            }
-        }
-        if (!fileExist(newName.substring(0, newName.lastIndexOf("/")))) {
-            createDir(newName.substring(0, newName.lastIndexOf("/")));
-        }
-        LOGGER.info("rename file :[" + oldPath + "] to [" + newPath + "] finish");
-
-        hdfsFs.rename(oldPath, newPath);
-    }
-
-    public static boolean createDir(@NonNull String filePath)
-        throws IOException {
-
-        FileSystem hdfsFs = getHdfsFs(filePath);
-        Path dfs = new Path(filePath);
-        return hdfsFs.mkdirs(dfs);
-    }
-
-    public static boolean fileExist(@NonNull String filePath)
-        throws IOException {
-        FileSystem hdfsFs = getHdfsFs(filePath);
-        Path fileName = new Path(filePath);
-        return hdfsFs.exists(fileName);
-    }
-}
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSource.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSource.java
index 769c1d768..ebaf2c51b 100644
--- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSource.java
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSource.java
@@ -22,8 +22,6 @@ import static org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY;
 
 import org.apache.seatunnel.api.common.PrepareFailException;
 import org.apache.seatunnel.api.common.SeaTunnelContext;
-import org.apache.seatunnel.api.serialization.DefaultSerializer;
-import org.apache.seatunnel.api.serialization.Serializer;
 import org.apache.seatunnel.api.source.Boundedness;
 import org.apache.seatunnel.api.source.SeaTunnelSource;
 import org.apache.seatunnel.api.source.SourceReader;
@@ -122,8 +120,4 @@ public class HiveSource implements SeaTunnelSource<SeaTunnelRow, HiveSourceSplit
         return new HiveSourceSplitEnumerator(enumeratorContext, filesPath, checkpointState);
     }
 
-    @Override
-    public Serializer<HiveSourceState> getEnumeratorStateSerializer() {
-        return new DefaultSerializer<>();
-    }
 }
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/AbstractReadStrategy.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/AbstractReadStrategy.java
index 50163235c..2df0a21f7 100644
--- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/AbstractReadStrategy.java
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/AbstractReadStrategy.java
@@ -72,8 +72,10 @@ public abstract class AbstractReadStrategy implements ReadStrategy {
                 continue;
             }
             if (fileStatus.isFile()) {
-
-                fileNames.add(fileStatus.getPath().toString());
+                // filter '_SUCCESS' file
+                if (!fileStatus.getPath().getName().equals("_SUCCESS")) {
+                    fileNames.add(fileStatus.getPath().toString());
+                }
             }
         }
         return fileNames;
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/TextReadStrategy.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/TextReadStrategy.java
index aa899cd51..6b014d737 100644
--- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/TextReadStrategy.java
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/TextReadStrategy.java
@@ -44,7 +44,7 @@ public class TextReadStrategy extends AbstractReadStrategy {
         FileSystem fs = FileSystem.get(conf);
         Path filePath = new Path(path);
         try (BufferedReader reader = new BufferedReader(new InputStreamReader(fs.open(filePath), StandardCharsets.UTF_8))) {
-            reader.lines().forEach(line -> output.collect(new SeaTunnelRow(new String[]{"TEXT_FIELD_NAME", line})));
+            reader.lines().forEach(line -> output.collect(new SeaTunnelRow(new String[]{line})));
         }
     }
 
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/utils/HiveMetaStoreProxy.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/utils/HiveMetaStoreProxy.java
new file mode 100644
index 000000000..30c9a2eba
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/utils/HiveMetaStoreProxy.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.hive.utils;
+
+import lombok.NonNull;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.thrift.TException;
+
+public class HiveMetaStoreProxy {
+
+    private HiveMetaStoreClient hiveMetaStoreClient;
+
+    public HiveMetaStoreProxy(@NonNull String uris) {
+        HiveConf hiveConf = new HiveConf();
+        hiveConf.set("hive.metastore.uris", uris);
+        try {
+            hiveMetaStoreClient = new HiveMetaStoreClient(hiveConf);
+        } catch (MetaException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public Table getTable(@NonNull String dbName, @NonNull String tableName) {
+        try {
+            return hiveMetaStoreClient.getTable(dbName, tableName);
+        } catch (TException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public HiveMetaStoreClient getHiveMetaStoreClient() {
+        return hiveMetaStoreClient;
+    }
+}
diff --git a/seatunnel-connectors-v2/connector-hive/src/test/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/TestHiveSinkConfig.java b/seatunnel-connectors-v2/connector-hive/src/test/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/TestHiveSinkConfig.java
new file mode 100644
index 000000000..92e5a6931
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-hive/src/test/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/TestHiveSinkConfig.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.hive.sink;
+
+import org.apache.seatunnel.api.table.type.BasicType;
+import org.apache.seatunnel.api.table.type.SeaTunnelDataType;
+import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
+
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+import org.apache.seatunnel.shade.com.typesafe.config.ConfigFactory;
+import org.apache.seatunnel.shade.com.typesafe.config.ConfigResolveOptions;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.File;
+import java.util.List;
+
+@RunWith(JUnit4.class)
+public class TestHiveSinkConfig {
+
+    @Test
+    public void testCreateHiveSinkConfig() {
+        String[] fieldNames = new String[]{"name", "age"};
+        SeaTunnelDataType[] seaTunnelDataTypes = new SeaTunnelDataType[]{BasicType.STRING_TYPE, BasicType.INT_TYPE};
+        SeaTunnelRowType seaTunnelRowTypeInfo = new SeaTunnelRowType(fieldNames, seaTunnelDataTypes);
+        String configFile = "fakesource_to_hive.conf";
+        String configFilePath = System.getProperty("user.dir") + "/src/test/resources/" + configFile;
+        Config config = ConfigFactory
+            .parseFile(new File(configFilePath))
+            .resolve(ConfigResolveOptions.defaults().setAllowUnresolved(true))
+            .resolveWith(ConfigFactory.systemProperties(),
+                ConfigResolveOptions.defaults().setAllowUnresolved(true));
+        List<? extends Config> sink = config.getConfigList("sink");
+        HiveSinkConfig hiveSinkConfig = new HiveSinkConfig(sink.get(0), seaTunnelRowTypeInfo);
+    }
+}
diff --git a/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/resources/examples/fakesource_to_file.conf b/seatunnel-connectors-v2/connector-hive/src/test/resources/fakesource_to_hive.conf
similarity index 80%
copy from seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/resources/examples/fakesource_to_file.conf
copy to seatunnel-connectors-v2/connector-hive/src/test/resources/fakesource_to_hive.conf
index c1ce63055..3412ea663 100644
--- a/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/resources/examples/fakesource_to_file.conf
+++ b/seatunnel-connectors-v2/connector-hive/src/test/resources/fakesource_to_hive.conf
@@ -21,8 +21,8 @@
 env {
   # You can set flink configuration here
   execution.parallelism = 1
-  job.mode = "STREAMING"
-  execution.checkpoint.interval = 5000
+  job.mode = "BATCH"
+  #execution.checkpoint.interval = 10000
   #execution.checkpoint.data-uri = "hdfs://localhost:9000/checkpoint"
 }
 
@@ -38,29 +38,16 @@ source {
 }
 
 transform {
-
-    sql {
-      sql = "select name,age from fake"
-    }
   # If you would like to get more information about how to configure seatunnel and see full list of transform plugins,
   # please go to https://seatunnel.apache.org/docs/flink/configuration/transform-plugins/Sql
 }
 
 sink {
-  File {
-    path="file:///tmp/hive/warehouse/test2"
-    field_delimiter="\t"
-    row_delimiter="\n"
+  Hive {
+    hive_table_name="default.test_fake_to_hive"
+    hive_metastore_uris="thrift://localhost:9083"
     partition_by=["age"]
-    partition_dir_expression="${k0}=${v0}"
-    is_partition_field_write_in_file=true
-    file_name_expression="${transactionId}_${now}"
-    file_format="text"
     sink_columns=["name","age"]
-    filename_time_format="yyyy.MM.dd"
-    is_enable_transaction=true
-    save_mode="error"
-
   }
 
   # If you would like to get more information about how to configure seatunnel and see full list of sink plugins,
diff --git a/seatunnel-connectors-v2/connector-hudi/pom.xml b/seatunnel-connectors-v2/connector-hudi/pom.xml
index 26b3c7b7c..0b2ee9ce8 100644
--- a/seatunnel-connectors-v2/connector-hudi/pom.xml
+++ b/seatunnel-connectors-v2/connector-hudi/pom.xml
@@ -32,9 +32,9 @@
     <dependencies>
 
         <dependency>
-            <groupId>org.apache.seatunnel</groupId>
-            <artifactId>seatunnel-hive-shade</artifactId>
-            <version>${project.version}</version>
+            <groupId>org.apache.hive</groupId>
+            <artifactId>hive-exec</artifactId>
+            <scope>provided</scope>
         </dependency>
 
         <dependency>
diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/source/HudiSource.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/source/HudiSource.java
index dbd68da7f..2ca69d784 100644
--- a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/source/HudiSource.java
+++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/source/HudiSource.java
@@ -26,8 +26,6 @@ import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiSourceCo
 
 import org.apache.seatunnel.api.common.PrepareFailException;
 import org.apache.seatunnel.api.common.SeaTunnelContext;
-import org.apache.seatunnel.api.serialization.DefaultSerializer;
-import org.apache.seatunnel.api.serialization.Serializer;
 import org.apache.seatunnel.api.source.Boundedness;
 import org.apache.seatunnel.api.source.SeaTunnelSource;
 import org.apache.seatunnel.api.source.SourceReader;
@@ -138,8 +136,4 @@ public class HudiSource implements SeaTunnelSource<SeaTunnelRow, HudiSourceSplit
         return new HudiSourceSplitEnumerator(enumeratorContext, tablePath, this.confFiles, checkpointState);
     }
 
-    @Override
-    public Serializer<HudiSourceState> getEnumeratorStateSerializer() {
-        return new DefaultSerializer<>();
-    }
 }
diff --git a/seatunnel-connectors-v2/connector-jdbc/pom.xml b/seatunnel-connectors-v2/connector-jdbc/pom.xml
index 1bb972ac9..39839eba7 100644
--- a/seatunnel-connectors-v2/connector-jdbc/pom.xml
+++ b/seatunnel-connectors-v2/connector-jdbc/pom.xml
@@ -46,6 +46,7 @@
         <dependency>
             <groupId>org.postgresql</groupId>
             <artifactId>postgresql</artifactId>
+            <scope>provided</scope>
         </dependency>
 
     </dependencies>
diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/MySqlCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/MySqlCatalog.java
index 4f6e61ce5..c3aa84ae4 100644
--- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/MySqlCatalog.java
+++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/MySqlCatalog.java
@@ -130,7 +130,7 @@ public class MySqlCatalog extends AbstractJdbcCatalog {
                 getPrimaryKey(metaData, tablePath.getDatabaseName(), tablePath.getTableName());
 
             PreparedStatement ps =
-                conn.prepareStatement(String.format("SELECT * FROM %s LIMIT 1;", tablePath.getFullName()));
+                conn.prepareStatement(String.format("SELECT * FROM %s WHERE 1 = 0;", tablePath.getFullName()));
 
             ResultSetMetaData tableMetaData = ps.getMetaData();
 
diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSource.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSource.java
index 73a61b3a3..2717436f1 100644
--- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSource.java
+++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSource.java
@@ -19,7 +19,6 @@ package org.apache.seatunnel.connectors.seatunnel.jdbc.source;
 
 import org.apache.seatunnel.api.common.PrepareFailException;
 import org.apache.seatunnel.api.common.SeaTunnelContext;
-import org.apache.seatunnel.api.serialization.DefaultSerializer;
 import org.apache.seatunnel.api.serialization.Serializer;
 import org.apache.seatunnel.api.source.Boundedness;
 import org.apache.seatunnel.api.source.SeaTunnelSource;
@@ -141,11 +140,6 @@ public class JdbcSource implements SeaTunnelSource<SeaTunnelRow, JdbcSourceSplit
         return new JdbcSourceSplitEnumerator(enumeratorContext, jdbcSourceOptions, partitionParameter);
     }
 
-    @Override
-    public Serializer<JdbcSourceState> getEnumeratorStateSerializer() {
-        return new DefaultSerializer<>();
-    }
-
     private SeaTunnelRowType initTableField(Connection conn) {
         JdbcDialectTypeMapper jdbcDialectTypeMapper = jdbcDialect.getJdbcDialectTypeMapper();
         ArrayList<SeaTunnelDataType<?>> seaTunnelDataTypes = new ArrayList<>();
diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java
index b5a3d42d6..a4d534e67 100644
--- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java
+++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java
@@ -25,8 +25,6 @@ import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.TOPI
 
 import org.apache.seatunnel.api.common.PrepareFailException;
 import org.apache.seatunnel.api.common.SeaTunnelContext;
-import org.apache.seatunnel.api.serialization.DefaultSerializer;
-import org.apache.seatunnel.api.serialization.Serializer;
 import org.apache.seatunnel.api.source.Boundedness;
 import org.apache.seatunnel.api.source.SeaTunnelSource;
 import org.apache.seatunnel.api.source.SourceReader;
@@ -120,11 +118,6 @@ public class KafkaSource implements SeaTunnelSource<SeaTunnelRow, KafkaSourceSpl
         return new KafkaSourceSplitEnumerator(this.metadata, enumeratorContext, checkpointState);
     }
 
-    @Override
-    public Serializer<KafkaSourceState> getEnumeratorStateSerializer() {
-        return new DefaultSerializer<>();
-    }
-
     @Override
     public void setSeaTunnelContext(SeaTunnelContext seaTunnelContext) {
         this.seaTunnelContext = seaTunnelContext;
diff --git a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java
index 9247e1d2b..88c36d816 100644
--- a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java
+++ b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java
@@ -40,9 +40,7 @@ import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProp
 import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.TOPIC_PATTERN;
 
 import org.apache.seatunnel.api.common.PrepareFailException;
-import org.apache.seatunnel.api.serialization.DefaultSerializer;
 import org.apache.seatunnel.api.serialization.DeserializationSchema;
-import org.apache.seatunnel.api.serialization.Serializer;
 import org.apache.seatunnel.api.source.Boundedness;
 import org.apache.seatunnel.api.source.SeaTunnelSource;
 import org.apache.seatunnel.api.source.SourceReader;
@@ -274,8 +272,4 @@ public class PulsarSource<T> implements SeaTunnelSource<T, PulsarPartitionSplit,
             checkpointState.assignedPartitions());
     }
 
-    @Override
-    public Serializer<PulsarSplitEnumeratorState> getEnumeratorStateSerializer() {
-        return new DefaultSerializer<>();
-    }
 }
diff --git a/seatunnel-connectors-v2/pom.xml b/seatunnel-connectors-v2/pom.xml
index 516c86cfc..ba947e30e 100644
--- a/seatunnel-connectors-v2/pom.xml
+++ b/seatunnel-connectors-v2/pom.xml
@@ -56,5 +56,16 @@
             <artifactId>junit-jupiter-params</artifactId>
         </dependency>
     </dependencies>
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-dependency-plugin</artifactId>
+                <configuration>
+                    <skip>true</skip>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
 
 </project>
\ No newline at end of file
diff --git a/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-assert/src/test/java/org/apache/seatunnel/flink/assertion/AssertExecutorTest.java b/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-assert/src/test/java/org/apache/seatunnel/flink/assertion/AssertExecutorTest.java
index 34c533cfc..1ddd866b9 100644
--- a/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-assert/src/test/java/org/apache/seatunnel/flink/assertion/AssertExecutorTest.java
+++ b/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-assert/src/test/java/org/apache/seatunnel/flink/assertion/AssertExecutorTest.java
@@ -26,7 +26,6 @@ import org.apache.flink.types.Row;
 
 import java.util.List;
 
-@SuppressWarnings("magicnumber")
 public class AssertExecutorTest extends TestCase {
     Row row = Row.withNames();
     AssertExecutor assertExecutor = new AssertExecutor();
diff --git a/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-assert/src/test/java/org/apache/seatunnel/flink/assertion/rule/AssertRuleParserTest.java b/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-assert/src/test/java/org/apache/seatunnel/flink/assertion/rule/AssertRuleParserTest.java
index fad5ed322..1df875e9d 100644
--- a/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-assert/src/test/java/org/apache/seatunnel/flink/assertion/rule/AssertRuleParserTest.java
+++ b/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-assert/src/test/java/org/apache/seatunnel/flink/assertion/rule/AssertRuleParserTest.java
@@ -25,7 +25,6 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 
 import java.util.List;
 
-@SuppressWarnings("magicnumber")
 public class AssertRuleParserTest extends TestCase {
     AssertRuleParser parser = new AssertRuleParser();
 
diff --git a/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-clickhouse/src/main/java/org/apache/seatunnel/flink/clickhouse/pojo/Shard.java b/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-clickhouse/src/main/java/org/apache/seatunnel/flink/clickhouse/pojo/Shard.java
index 20f34be1e..e2d1394df 100644
--- a/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-clickhouse/src/main/java/org/apache/seatunnel/flink/clickhouse/pojo/Shard.java
+++ b/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-clickhouse/src/main/java/org/apache/seatunnel/flink/clickhouse/pojo/Shard.java
@@ -114,7 +114,6 @@ public class Shard implements Serializable {
     }
 
     @Override
-    @SuppressWarnings("magicnumber")
     public int hashCode() {
         if (hashCode != -1) {
             return hashCode;
diff --git a/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-clickhouse/src/main/java/org/apache/seatunnel/flink/clickhouse/pojo/ShardMetadata.java b/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-clickhouse/src/main/java/org/apache/seatunnel/flink/clickhouse/pojo/ShardMetadata.java
index 96e273364..033c99d90 100644
--- a/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-clickhouse/src/main/java/org/apache/seatunnel/flink/clickhouse/pojo/ShardMetadata.java
+++ b/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-clickhouse/src/main/java/org/apache/seatunnel/flink/clickhouse/pojo/ShardMetadata.java
@@ -122,7 +122,6 @@ public class ShardMetadata implements Serializable {
     }
 
     @Override
-    @SuppressWarnings("magicnumber")
     public int hashCode() {
         int result = shardKey.hashCode();
         result = 31 * result + shardKeyType.hashCode();
diff --git a/seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-hudi/src/main/scala/org/apache/seatunnel/spark/hudi/source/Hudi.scala b/seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-hudi/src/main/scala/org/apache/seatunnel/spark/hudi/source/Hudi.scala
index 82e55f09a..a1d42862c 100644
--- a/seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-hudi/src/main/scala/org/apache/seatunnel/spark/hudi/source/Hudi.scala
+++ b/seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-hudi/src/main/scala/org/apache/seatunnel/spark/hudi/source/Hudi.scala
@@ -38,7 +38,7 @@ class Hudi extends SparkBatchSource {
       reader.option(e.getKey, String.valueOf(e.getValue.unwrapped()))
     }
 
-    reader.load(config.getString(HOODIE_DATASTORE_READ_PATHS))
+    reader.load()
   }
 
   override def getPluginName: String = "Hudi"
diff --git a/seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-jdbc/src/main/scala/org/apache/seatunnel/spark/jdbc/source/Jdbc.scala b/seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-jdbc/src/main/scala/org/apache/seatunnel/spark/jdbc/source/Jdbc.scala
index df3d8c983..6a31beefd 100644
--- a/seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-jdbc/src/main/scala/org/apache/seatunnel/spark/jdbc/source/Jdbc.scala
+++ b/seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-jdbc/src/main/scala/org/apache/seatunnel/spark/jdbc/source/Jdbc.scala
@@ -22,7 +22,7 @@ import org.apache.seatunnel.common.config.{CheckResult, TypesafeConfigUtils}
 import org.apache.seatunnel.common.config.CheckConfigUtil.checkAllExists
 import org.apache.seatunnel.spark.SparkEnvironment
 import org.apache.seatunnel.spark.batch.SparkBatchSource
-import org.apache.seatunnel.spark.jdbc.source.util.HiveDialet
+import org.apache.seatunnel.spark.jdbc.source.util.HiveDialect
 import org.apache.spark.sql.jdbc.JdbcDialects
 import org.apache.spark.sql.{DataFrameReader, Dataset, Row, SparkSession}
 
@@ -60,7 +60,7 @@ class Jdbc extends SparkBatchSource {
     }
 
     if (config.getString("url").startsWith("jdbc:hive2")) {
-      JdbcDialects.registerDialect(new HiveDialet)
+      JdbcDialects.registerDialect(new HiveDialect)
     }
 
     reader
diff --git a/seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-jdbc/src/main/scala/org/apache/seatunnel/spark/jdbc/source/util/HiveDialet.scala b/seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-jdbc/src/main/scala/org/apache/seatunnel/spark/jdbc/source/util/HiveDialect.scala
similarity index 96%
copy from seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-jdbc/src/main/scala/org/apache/seatunnel/spark/jdbc/source/util/HiveDialet.scala
copy to seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-jdbc/src/main/scala/org/apache/seatunnel/spark/jdbc/source/util/HiveDialect.scala
index c686cdfba..81eb48bf6 100644
--- a/seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-jdbc/src/main/scala/org/apache/seatunnel/spark/jdbc/source/util/HiveDialet.scala
+++ b/seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-jdbc/src/main/scala/org/apache/seatunnel/spark/jdbc/source/util/HiveDialect.scala
@@ -18,7 +18,7 @@ package org.apache.seatunnel.spark.jdbc.source.util
 
 import org.apache.spark.sql.jdbc.JdbcDialect
 
-class HiveDialet extends JdbcDialect {
+class HiveDialect extends JdbcDialect {
   override def canHandle(url: String): Boolean = {
     url.startsWith("jdbc:hive2")
   }
diff --git a/seatunnel-core/seatunnel-core-base/src/main/java/org/apache/seatunnel/core/base/command/AbstractCommandArgs.java b/seatunnel-core/seatunnel-core-base/src/main/java/org/apache/seatunnel/core/base/command/AbstractCommandArgs.java
index a19fe285f..9042b2c14 100644
--- a/seatunnel-core/seatunnel-core-base/src/main/java/org/apache/seatunnel/core/base/command/AbstractCommandArgs.java
+++ b/seatunnel-core/seatunnel-core-base/src/main/java/org/apache/seatunnel/core/base/command/AbstractCommandArgs.java
@@ -38,7 +38,7 @@ public abstract class AbstractCommandArgs implements CommandArgs {
     private List<String> variables = Collections.emptyList();
 
     // todo: use command type enum
-    @Parameter(names = {"-t", "--check"},
+    @Parameter(names = {"-ck", "--check"},
             description = "check config")
     private boolean checkConfig = false;
 
diff --git a/seatunnel-core/seatunnel-core-flink/src/test/java/org/apache/seatunnel/core/flink/args/FlinkCommandArgsTest.java b/seatunnel-core/seatunnel-core-flink/src/test/java/org/apache/seatunnel/core/flink/args/FlinkCommandArgsTest.java
index 56f4f5e63..a4dba1b27 100644
--- a/seatunnel-core/seatunnel-core-flink/src/test/java/org/apache/seatunnel/core/flink/args/FlinkCommandArgsTest.java
+++ b/seatunnel-core/seatunnel-core-flink/src/test/java/org/apache/seatunnel/core/flink/args/FlinkCommandArgsTest.java
@@ -27,7 +27,7 @@ public class FlinkCommandArgsTest {
 
     @Test
     public void testParseFlinkArgs() {
-        String[] args = {"-c", "app.conf", "-t", "-i", "city=shenyang", "-i", "date=20200202"};
+        String[] args = {"-c", "app.conf", "-ck", "-i", "city=shenyang", "-i", "date=20200202"};
         FlinkCommandArgs flinkArgs = new FlinkCommandArgs();
         JCommander.newBuilder()
             .addObject(flinkArgs)
diff --git a/seatunnel-core/seatunnel-core-flink/src/test/java/org/apache/seatunnel/core/flink/utils/CommandLineUtilsTest.java b/seatunnel-core/seatunnel-core-flink/src/test/java/org/apache/seatunnel/core/flink/utils/CommandLineUtilsTest.java
index fe206c0f5..2befcdbe8 100644
--- a/seatunnel-core/seatunnel-core-flink/src/test/java/org/apache/seatunnel/core/flink/utils/CommandLineUtilsTest.java
+++ b/seatunnel-core/seatunnel-core-flink/src/test/java/org/apache/seatunnel/core/flink/utils/CommandLineUtilsTest.java
@@ -34,14 +34,14 @@ public class CommandLineUtilsTest {
 
     @Test
     public void testParseCommandArgs() {
-        String[] args = {"--detached", "-c", "app.conf", "-t", "-i", "city=shenyang", "-i", "date=20200202",
+        String[] args = {"--detached", "-c", "app.conf", "-ck", "-i", "city=shenyang", "-i", "date=20200202",
             "-r", "run-application", "--unkown", "unkown-command"};
         FlinkCommandArgs flinkCommandArgs = CommandLineUtils.parseCommandArgs(args, FlinkJobType.JAR);
         Assert.assertEquals(flinkCommandArgs.getFlinkParams(), Arrays.asList("--detached", "--unkown", "unkown-command"));
         Assert.assertEquals(flinkCommandArgs.getRunMode(), FlinkRunMode.APPLICATION_RUN);
         Assert.assertEquals(flinkCommandArgs.getVariables(), Arrays.asList("city=shenyang", "date=20200202"));
 
-        String[] args1 = {"--detached", "-c", "app.conf", "-t", "-i", "city=shenyang", "-i", "date=20200202",
+        String[] args1 = {"--detached", "-c", "app.conf", "-ck", "-i", "city=shenyang", "-i", "date=20200202",
             "-r", "run-application", "--unkown", "unkown-command"};
         flinkCommandArgs = CommandLineUtils.parseCommandArgs(args1, FlinkJobType.SQL);
         Assert.assertEquals(flinkCommandArgs.getFlinkParams(), Arrays.asList("--detached", "--unkown", "unkown-command"));
@@ -51,7 +51,7 @@ public class CommandLineUtilsTest {
 
     @Test
     public void testBuildFlinkJarCommand() throws FileNotFoundException {
-        String[] args = {"--detached", "-c", APP_CONF_PATH, "-t", "-i", "city=shenyang", "-i", "date=20200202",
+        String[] args = {"--detached", "-c", APP_CONF_PATH, "-ck", "-i", "city=shenyang", "-i", "date=20200202",
             "-r", "run-application", "--unkown", "unkown-command"};
         FlinkCommandArgs flinkCommandArgs = CommandLineUtils.parseCommandArgs(args, FlinkJobType.JAR);
         List<String> commands = CommandLineUtils.buildFlinkCommand(flinkCommandArgs, "CLASS_NAME", "/path/to/jar");
@@ -65,14 +65,14 @@ public class CommandLineUtilsTest {
             Arrays.asList("${FLINK_HOME}/bin/flink", "run-application", "--detached", "--unkown", "unkown-command", "-c",
                 "CLASS_NAME", "/path/to/jar", "--config", APP_CONF_PATH, "--check", "-Dcity=shenyang", "-Ddate=20200202"));
 
-        String[] args1 = {"--detached", "-c", "app.conf", "-t", "-i", "city=shenyang", "-i", "date=20200202",
+        String[] args1 = {"--detached", "-c", "app.conf", "-ck", "-i", "city=shenyang", "-i", "date=20200202",
             "-r", "run-application", "--unkown", "unkown-command"};
 
     }
 
     @Test
     public void testBuildFlinkSQLCommand() throws FileNotFoundException{
-        String[] args = {"--detached", "-c", SQL_CONF_PATH, "-t", "-i", "city=shenyang", "-i", "date=20200202",
+        String[] args = {"--detached", "-c", SQL_CONF_PATH, "-ck", "-i", "city=shenyang", "-i", "date=20200202",
             "-r", "run-application", "--unkown", "unkown-command"};
         FlinkCommandArgs flinkCommandArgs = CommandLineUtils.parseCommandArgs(args, FlinkJobType.SQL);
         List<String> commands = CommandLineUtils.buildFlinkCommand(flinkCommandArgs, "CLASS_NAME", "/path/to/jar");
diff --git a/seatunnel-core/seatunnel-flink-starter/src/main/bin/start-seatunnel-flink-new-connector.sh b/seatunnel-core/seatunnel-flink-starter/src/main/bin/start-seatunnel-flink-new-connector.sh
index 379378b9f..0c276319b 100755
--- a/seatunnel-core/seatunnel-flink-starter/src/main/bin/start-seatunnel-flink-new-connector.sh
+++ b/seatunnel-core/seatunnel-flink-starter/src/main/bin/start-seatunnel-flink-new-connector.sh
@@ -53,12 +53,12 @@ fi
 CMD=$(java -cp ${APP_JAR} ${APP_MAIN} ${args}) && EXIT_CODE=$? || EXIT_CODE=$?
 if [ ${EXIT_CODE} -eq 234 ]; then
     # print usage
-    echo ${CMD}
+    echo "${CMD}"
     exit 0
 elif [ ${EXIT_CODE} -eq 0 ]; then
     echo "Execute SeaTunnel Flink Job: ${CMD}"
     eval ${CMD}
 else
-    echo ${CMD}
+    echo "${CMD}"
     exit ${EXIT_CODE}
 fi
diff --git a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/env/FlinkEnvironment.java b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/env/FlinkEnvironment.java
deleted file mode 100644
index f9de514b0..000000000
--- a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/env/FlinkEnvironment.java
+++ /dev/null
@@ -1,286 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.seatunnel.core.starter.flink.env;
-
-import org.apache.seatunnel.apis.base.env.RuntimeEnv;
-import org.apache.seatunnel.common.config.CheckResult;
-import org.apache.seatunnel.common.constants.JobMode;
-import org.apache.seatunnel.common.utils.ReflectionUtils;
-import org.apache.seatunnel.flink.util.ConfigKeyName;
-import org.apache.seatunnel.flink.util.EnvironmentUtil;
-
-import org.apache.seatunnel.shade.com.typesafe.config.Config;
-
-import org.apache.flink.api.common.RuntimeExecutionMode;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.PipelineOptions;
-import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
-import org.apache.flink.runtime.state.StateBackend;
-import org.apache.flink.runtime.state.filesystem.FsStateBackend;
-import org.apache.flink.streaming.api.CheckpointingMode;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.environment.CheckpointConfig;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.table.api.EnvironmentSettings;
-import org.apache.flink.table.api.TableConfig;
-import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
-import org.apache.flink.util.TernaryBoolean;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Objects;
-import java.util.stream.Collectors;
-
-public class FlinkEnvironment implements RuntimeEnv {
-
-    private static final Logger LOGGER = LoggerFactory.getLogger(FlinkEnvironment.class);
-
-    private Config config;
-
-    private StreamExecutionEnvironment environment;
-
-    private StreamTableEnvironment tableEnvironment;
-
-    private JobMode jobMode;
-
-    private String jobName = "seatunnel";
-
-    @Override
-    public FlinkEnvironment setConfig(Config config) {
-        this.config = config;
-        return this;
-    }
-
-    @Override
-    public Config getConfig() {
-        return config;
-    }
-
-    @Override
-    public CheckResult checkConfig() {
-        return EnvironmentUtil.checkRestartStrategy(config);
-    }
-
-    @Override
-    public FlinkEnvironment prepare() {
-        // Batch/Streaming both use data stream api in SeaTunnel New API
-        createStreamEnvironment();
-        createStreamTableEnvironment();
-        if (config.hasPath("job.name")) {
-            jobName = config.getString("job.name");
-        }
-        return this;
-    }
-
-    public String getJobName() {
-        return jobName;
-    }
-
-    @Override
-    public FlinkEnvironment setJobMode(JobMode jobMode) {
-        this.jobMode = jobMode;
-        return this;
-    }
-
-    @Override
-    public JobMode getJobMode() {
-        return jobMode;
-    }
-
-    @Override
-    public void registerPlugin(List<URL> pluginPaths) {
-        pluginPaths.forEach(url -> LOGGER.info("register plugins : {}", url));
-        Configuration configuration;
-        try {
-            configuration =
-                (Configuration) Objects.requireNonNull(ReflectionUtils.getDeclaredMethod(StreamExecutionEnvironment.class,
-                    "getConfiguration")).orElseThrow(() -> new RuntimeException("can't find " +
-                    "method: getConfiguration")).invoke(this.environment);
-        } catch (Exception e) {
-            throw new RuntimeException("Get flink configuration from environment failed", e);
-        }
-        List<String> jars = configuration.get(PipelineOptions.JARS);
-        if (jars == null) {
-            jars = new ArrayList<>();
-        }
-        jars.addAll(pluginPaths.stream().map(URL::toString).collect(Collectors.toList()));
-        configuration.set(PipelineOptions.JARS, jars);
-        List<String> classpath = configuration.get(PipelineOptions.CLASSPATHS);
-        if (classpath == null) {
-            classpath = new ArrayList<>();
-        }
-        classpath.addAll(pluginPaths.stream().map(URL::toString).collect(Collectors.toList()));
-        configuration.set(PipelineOptions.CLASSPATHS, classpath);
-    }
-
-    public StreamExecutionEnvironment getStreamExecutionEnvironment() {
-        return environment;
-    }
-
-    public StreamTableEnvironment getStreamTableEnvironment() {
-        return tableEnvironment;
-    }
-
-    private void createStreamTableEnvironment() {
-        // use blink and streammode
-        EnvironmentSettings.Builder envBuilder = EnvironmentSettings.newInstance()
-                .inStreamingMode();
-        if (this.config.hasPath(ConfigKeyName.PLANNER) && "blink"
-                .equals(this.config.getString(ConfigKeyName.PLANNER))) {
-            envBuilder.useBlinkPlanner();
-        } else {
-            envBuilder.useOldPlanner();
-        }
-        EnvironmentSettings environmentSettings = envBuilder.build();
-
-        tableEnvironment = StreamTableEnvironment.create(getStreamExecutionEnvironment(), environmentSettings);
-        TableConfig config = tableEnvironment.getConfig();
-        if (this.config.hasPath(ConfigKeyName.MAX_STATE_RETENTION_TIME) && this.config
-                .hasPath(ConfigKeyName.MIN_STATE_RETENTION_TIME)) {
-            long max = this.config.getLong(ConfigKeyName.MAX_STATE_RETENTION_TIME);
-            long min = this.config.getLong(ConfigKeyName.MIN_STATE_RETENTION_TIME);
-            config.setIdleStateRetentionTime(Time.seconds(min), Time.seconds(max));
-        }
-    }
-
-    private void createStreamEnvironment() {
-        environment = StreamExecutionEnvironment.getExecutionEnvironment();
-        setTimeCharacteristic();
-
-        setCheckpoint();
-
-        EnvironmentUtil.setRestartStrategy(config, environment.getConfig());
-
-        if (config.hasPath(ConfigKeyName.BUFFER_TIMEOUT_MILLIS)) {
-            long timeout = config.getLong(ConfigKeyName.BUFFER_TIMEOUT_MILLIS);
-            environment.setBufferTimeout(timeout);
-        }
-
-        if (config.hasPath(ConfigKeyName.PARALLELISM)) {
-            int parallelism = config.getInt(ConfigKeyName.PARALLELISM);
-            environment.setParallelism(parallelism);
-        }
-
-        if (config.hasPath(ConfigKeyName.MAX_PARALLELISM)) {
-            int max = config.getInt(ConfigKeyName.MAX_PARALLELISM);
-            environment.setMaxParallelism(max);
-        }
-
-        if (this.jobMode.equals(JobMode.BATCH)) {
-            environment.setRuntimeMode(RuntimeExecutionMode.BATCH);
-        }
-    }
-
-    private void setTimeCharacteristic() {
-        if (config.hasPath(ConfigKeyName.TIME_CHARACTERISTIC)) {
-            String timeType = config.getString(ConfigKeyName.TIME_CHARACTERISTIC);
-            switch (timeType.toLowerCase()) {
-                case "event-time":
-                    environment.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
-                    break;
-                case "ingestion-time":
-                    environment.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime);
-                    break;
-                case "processing-time":
-                    environment.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
-                    break;
-                default:
-                    LOGGER.warn(
-                            "set time-characteristic failed, unknown time-characteristic [{}],only support event-time,ingestion-time,processing-time",
-                            timeType);
-                    break;
-            }
-        }
-    }
-
-    private void setCheckpoint() {
-        if (config.hasPath(ConfigKeyName.CHECKPOINT_INTERVAL)) {
-            CheckpointConfig checkpointConfig = environment.getCheckpointConfig();
-            long interval = config.getLong(ConfigKeyName.CHECKPOINT_INTERVAL);
-            environment.enableCheckpointing(interval);
-
-            if (config.hasPath(ConfigKeyName.CHECKPOINT_MODE)) {
-                String mode = config.getString(ConfigKeyName.CHECKPOINT_MODE);
-                switch (mode.toLowerCase()) {
-                    case "exactly-once":
-                        checkpointConfig.setCheckpointingMode(CheckpointingMode.EXACTLY_ONCE);
-                        break;
-                    case "at-least-once":
-                        checkpointConfig.setCheckpointingMode(CheckpointingMode.AT_LEAST_ONCE);
-                        break;
-                    default:
-                        LOGGER.warn(
-                                "set checkpoint.mode failed, unknown checkpoint.mode [{}],only support exactly-once,at-least-once",
-                                mode);
-                        break;
-                }
-            }
-
-            if (config.hasPath(ConfigKeyName.CHECKPOINT_TIMEOUT)) {
-                long timeout = config.getLong(ConfigKeyName.CHECKPOINT_TIMEOUT);
-                checkpointConfig.setCheckpointTimeout(timeout);
-            }
-
-            if (config.hasPath(ConfigKeyName.CHECKPOINT_DATA_URI)) {
-                String uri = config.getString(ConfigKeyName.CHECKPOINT_DATA_URI);
-                StateBackend fsStateBackend = new FsStateBackend(uri);
-                if (config.hasPath(ConfigKeyName.STATE_BACKEND)) {
-                    String stateBackend = config.getString(ConfigKeyName.STATE_BACKEND);
-                    if ("rocksdb".equalsIgnoreCase(stateBackend)) {
-                        StateBackend rocksDBStateBackend = new RocksDBStateBackend(fsStateBackend, TernaryBoolean.TRUE);
-                        environment.setStateBackend(rocksDBStateBackend);
-                    }
-                } else {
-                    environment.setStateBackend(fsStateBackend);
-                }
-            }
-
-            if (config.hasPath(ConfigKeyName.MAX_CONCURRENT_CHECKPOINTS)) {
-                int max = config.getInt(ConfigKeyName.MAX_CONCURRENT_CHECKPOINTS);
-                checkpointConfig.setMaxConcurrentCheckpoints(max);
-            }
-
-            if (config.hasPath(ConfigKeyName.CHECKPOINT_CLEANUP_MODE)) {
-                boolean cleanup = config.getBoolean(ConfigKeyName.CHECKPOINT_CLEANUP_MODE);
-                if (cleanup) {
-                    checkpointConfig.enableExternalizedCheckpoints(
-                            CheckpointConfig.ExternalizedCheckpointCleanup.DELETE_ON_CANCELLATION);
-                } else {
-                    checkpointConfig.enableExternalizedCheckpoints(
-                            CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
-
-                }
-            }
-
-            if (config.hasPath(ConfigKeyName.MIN_PAUSE_BETWEEN_CHECKPOINTS)) {
-                long minPause = config.getLong(ConfigKeyName.MIN_PAUSE_BETWEEN_CHECKPOINTS);
-                checkpointConfig.setMinPauseBetweenCheckpoints(minPause);
-            }
-
-            if (config.hasPath(ConfigKeyName.FAIL_ON_CHECKPOINTING_ERRORS)) {
-                int failNum = config.getInt(ConfigKeyName.FAIL_ON_CHECKPOINTING_ERRORS);
-                checkpointConfig.setTolerableCheckpointFailureNumber(failNum);
-            }
-        }
-    }
-
-}
diff --git a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/AbstractPluginExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/AbstractPluginExecuteProcessor.java
index 6c3ebceb1..5e3178bfb 100644
--- a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/AbstractPluginExecuteProcessor.java
+++ b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/AbstractPluginExecuteProcessor.java
@@ -20,6 +20,7 @@ package org.apache.seatunnel.core.starter.flink.execution;
 import static org.apache.seatunnel.apis.base.plugin.Plugin.RESULT_TABLE_NAME;
 import static org.apache.seatunnel.apis.base.plugin.Plugin.SOURCE_TABLE_NAME;
 
+import org.apache.seatunnel.common.utils.ReflectionUtils;
 import org.apache.seatunnel.flink.FlinkEnvironment;
 import org.apache.seatunnel.flink.util.TableUtil;
 
@@ -30,8 +31,11 @@ import org.apache.flink.table.api.Table;
 import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
 import org.apache.flink.types.Row;
 
+import java.net.URL;
+import java.net.URLClassLoader;
 import java.util.List;
 import java.util.Optional;
+import java.util.function.BiConsumer;
 
 public abstract class AbstractPluginExecuteProcessor<T> implements PluginExecuteProcessor {
 
@@ -41,6 +45,17 @@ public abstract class AbstractPluginExecuteProcessor<T> implements PluginExecute
     protected static final String ENGINE_TYPE = "seatunnel";
     protected static final String PLUGIN_NAME = "plugin_name";
 
+    protected final BiConsumer<ClassLoader, URL> addUrlToClassloader = (classLoader, url) -> {
+        if (classLoader.getClass().getName().endsWith("SafetyNetWrapperClassLoader")) {
+            URLClassLoader c = (URLClassLoader) ReflectionUtils.getField(classLoader, "inner").get();
+            ReflectionUtils.invoke(c, "addURL", url);
+        } else if (classLoader instanceof URLClassLoader) {
+            ReflectionUtils.invoke(classLoader, "addURL", url);
+        } else {
+            throw new RuntimeException("Unsupported classloader: " + classLoader.getClass().getName());
+        }
+    };
+
     protected AbstractPluginExecuteProcessor(FlinkEnvironment flinkEnvironment,
                                              List<? extends Config> pluginConfigs) {
         this.flinkEnvironment = flinkEnvironment;
diff --git a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java
index ef412213e..8de5d422d 100644
--- a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java
+++ b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java
@@ -52,7 +52,7 @@ public class SinkExecuteProcessor extends AbstractPluginExecuteProcessor<SeaTunn
 
     @Override
     protected List<SeaTunnelSink<SeaTunnelRow, Serializable, Serializable, Serializable>> initializePlugins(List<? extends Config> pluginConfigs) {
-        SeaTunnelSinkPluginDiscovery sinkPluginDiscovery = new SeaTunnelSinkPluginDiscovery();
+        SeaTunnelSinkPluginDiscovery sinkPluginDiscovery = new SeaTunnelSinkPluginDiscovery(addUrlToClassloader);
         List<URL> pluginJars = new ArrayList<>();
         List<SeaTunnelSink<SeaTunnelRow, Serializable, Serializable, Serializable>> sinks = pluginConfigs.stream().map(sinkConfig -> {
             PluginIdentifier pluginIdentifier = PluginIdentifier.of(ENGINE_TYPE, PLUGIN_TYPE, sinkConfig.getString(PLUGIN_NAME));
diff --git a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java
index fa0cb65eb..a1b31836f 100644
--- a/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java
+++ b/seatunnel-core/seatunnel-flink-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java
@@ -101,7 +101,7 @@ public class SourceExecuteProcessor extends AbstractPluginExecuteProcessor<SeaTu
 
     @Override
     protected List<SeaTunnelSource> initializePlugins(List<? extends Config> pluginConfigs) {
-        SeaTunnelSourcePluginDiscovery sourcePluginDiscovery = new SeaTunnelSourcePluginDiscovery();
+        SeaTunnelSourcePluginDiscovery sourcePluginDiscovery = new SeaTunnelSourcePluginDiscovery(addUrlToClassloader);
         List<SeaTunnelSource> sources = new ArrayList<>();
         Set<URL> jars = new HashSet<>();
         for (Config sourceConfig : pluginConfigs) {
diff --git a/seatunnel-core/seatunnel-spark-starter/src/main/bin/start-seatunnel-spark-new-connector.sh b/seatunnel-core/seatunnel-spark-starter/src/main/bin/start-seatunnel-spark-new-connector.sh
index c53004bb8..a54dc55ee 100755
--- a/seatunnel-core/seatunnel-spark-starter/src/main/bin/start-seatunnel-spark-new-connector.sh
+++ b/seatunnel-core/seatunnel-spark-starter/src/main/bin/start-seatunnel-spark-new-connector.sh
@@ -53,12 +53,12 @@ fi
 CMD=$(java -cp ${APP_JAR} ${APP_MAIN} ${args} | tail -n 1) && EXIT_CODE=$? || EXIT_CODE=$?
 if [ ${EXIT_CODE} -eq 234 ]; then
     # print usage
-    echo ${CMD}
+    echo "${CMD}"
     exit 0
 elif [ ${EXIT_CODE} -eq 0 ]; then
-    echo "Execute SeaTunnel Spark Job: ${CMD}"
-    eval ${CMD}
+    echo "Execute SeaTunnel Spark Job: $(echo ${CMD} | tail -n 1)"
+    eval $(echo ${CMD} | tail -n 1)
 else
-    echo ${CMD}
+    echo "${CMD}"
     exit ${EXIT_CODE}
 fi
diff --git a/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java b/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java
index 3a8d7043f..00261a80c 100644
--- a/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java
+++ b/seatunnel-core/seatunnel-spark-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java
@@ -24,13 +24,12 @@ import org.apache.seatunnel.common.config.Common;
 import org.apache.seatunnel.common.config.DeployMode;
 import org.apache.seatunnel.core.starter.Starter;
 import org.apache.seatunnel.core.starter.config.ConfigBuilder;
-import org.apache.seatunnel.core.starter.config.EngineType;
 import org.apache.seatunnel.core.starter.config.PluginType;
 import org.apache.seatunnel.core.starter.spark.args.SparkCommandArgs;
 import org.apache.seatunnel.core.starter.utils.CompressionUtils;
 import org.apache.seatunnel.plugin.discovery.PluginIdentifier;
-import org.apache.seatunnel.plugin.discovery.spark.SparkSinkPluginDiscovery;
-import org.apache.seatunnel.plugin.discovery.spark.SparkSourcePluginDiscovery;
+import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelSinkPluginDiscovery;
+import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelSourcePluginDiscovery;
 
 import org.apache.seatunnel.shade.com.typesafe.config.Config;
 import org.apache.seatunnel.shade.com.typesafe.config.ConfigFactory;
@@ -222,16 +221,16 @@ public class SparkStarter implements Starter {
      * return connector's jars, which located in 'connectors/spark/*'.
      */
     private List<Path> getConnectorJarDependencies() {
-        Path pluginRootDir = Common.connectorJarDir("spark");
+        Path pluginRootDir = Common.connectorJarDir("seatunnel");
         if (!Files.exists(pluginRootDir) || !Files.isDirectory(pluginRootDir)) {
             return Collections.emptyList();
         }
         Config config = new ConfigBuilder(Paths.get(commandArgs.getConfigFile())).getConfig();
         Set<URL> pluginJars = new HashSet<>();
-        SparkSourcePluginDiscovery sparkSourcePluginDiscovery = new SparkSourcePluginDiscovery();
-        SparkSinkPluginDiscovery sparkSinkPluginDiscovery = new SparkSinkPluginDiscovery();
-        pluginJars.addAll(sparkSourcePluginDiscovery.getPluginJarPaths(getPluginIdentifiers(config, PluginType.SOURCE)));
-        pluginJars.addAll(sparkSinkPluginDiscovery.getPluginJarPaths(getPluginIdentifiers(config, PluginType.SINK)));
+        SeaTunnelSourcePluginDiscovery seaTunnelSourcePluginDiscovery = new SeaTunnelSourcePluginDiscovery();
+        SeaTunnelSinkPluginDiscovery seaTunnelSinkPluginDiscovery = new SeaTunnelSinkPluginDiscovery();
+        pluginJars.addAll(seaTunnelSourcePluginDiscovery.getPluginJarPaths(getPluginIdentifiers(config, PluginType.SOURCE)));
+        pluginJars.addAll(seaTunnelSinkPluginDiscovery.getPluginJarPaths(getPluginIdentifiers(config, PluginType.SINK)));
         return pluginJars.stream().map(url -> new File(url.getPath()).toPath()).collect(Collectors.toList());
     }
 
@@ -317,10 +316,8 @@ public class SparkStarter implements Starter {
         return Arrays.stream(pluginTypes).flatMap((Function<PluginType, Stream<PluginIdentifier>>) pluginType -> {
             List<? extends Config> configList = config.getConfigList(pluginType.getType());
             return configList.stream()
-                    .map(pluginConfig -> PluginIdentifier
-                            .of(EngineType.SPARK.getEngine(),
-                                    pluginType.getType(),
-                                    pluginConfig.getString("plugin_name")));
+                    .map(pluginConfig -> PluginIdentifier.of("seatunnel", pluginType.getType(),
+                            pluginConfig.getString("plugin_name")));
         }).collect(Collectors.toList());
     }
 
diff --git a/seatunnel-dependency-shade/pom.xml b/seatunnel-dependency-shade/pom.xml
deleted file mode 100644
index 1e6259991..000000000
--- a/seatunnel-dependency-shade/pom.xml
+++ /dev/null
@@ -1,39 +0,0 @@
-<?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">
-
-    <parent>
-        <artifactId>seatunnel</artifactId>
-        <groupId>org.apache.seatunnel</groupId>
-        <version>${revision}</version>
-        <relativePath>../pom.xml</relativePath>
-    </parent>
-    <modelVersion>4.0.0</modelVersion>
-
-    <artifactId>seatunnel-dependency-shade</artifactId>
-    <packaging>pom</packaging>
-    
-    <modules>
-        <module>seatunnel-hive-shade</module>
-    </modules>
-
-</project>
\ No newline at end of file
diff --git a/seatunnel-dependency-shade/seatunnel-hive-shade/pom.xml b/seatunnel-dependency-shade/seatunnel-hive-shade/pom.xml
deleted file mode 100644
index ddab451b6..000000000
--- a/seatunnel-dependency-shade/seatunnel-hive-shade/pom.xml
+++ /dev/null
@@ -1,69 +0,0 @@
-<?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">
-    <parent>
-        <artifactId>seatunnel-dependency-shade</artifactId>
-        <groupId>org.apache.seatunnel</groupId>
-        <version>${revision}</version>
-    </parent>
-    <modelVersion>4.0.0</modelVersion>
-    <artifactId>seatunnel-hive-shade</artifactId>
-
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.hive</groupId>
-            <artifactId>hive-exec</artifactId>
-        </dependency>
-    </dependencies>
-    <build>
-        <finalName>${project.artifactId}-${project.version}</finalName>
-        <plugins>
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-shade-plugin</artifactId>
-                <executions>
-                    <execution>
-                        <phase>package</phase>
-                        <goals>
-                            <goal>shade</goal>
-                        </goals>
-                        <configuration>
-                            <filters>
-                                <filter>
-                                    <artifact>*:*</artifact>
-                                    <excludes>
-                                        <exclude>META-INF/*.SF</exclude>
-                                        <exclude>META-INF/*.DSA</exclude>
-                                        <exclude>META-INF/*.RSA</exclude>
-                                        <exclude>org/apache/calcite/**</exclude>
-                                        <exclude>org/htrace/**</exclude>
-                                    </excludes>
-                                </filter>
-                            </filters>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-          
-        </plugins>
-    </build>
-</project>
\ No newline at end of file
diff --git a/seatunnel-e2e/pom.xml b/seatunnel-e2e/pom.xml
index 8331b68eb..9f5e6882b 100644
--- a/seatunnel-e2e/pom.xml
+++ b/seatunnel-e2e/pom.xml
@@ -51,4 +51,15 @@
             <version>${project.version}</version>
         </dependency>
     </dependencies>
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-dependency-plugin</artifactId>
+                <configuration>
+                    <skip>true</skip>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
 </project>
\ No newline at end of file
diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/fake/FakeSourceToConsoleIT.java b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/fake/FakeSourceToConsoleIT.java
index f715c4e79..d480fc946 100644
--- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/fake/FakeSourceToConsoleIT.java
+++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/fake/FakeSourceToConsoleIT.java
@@ -28,7 +28,6 @@ import java.io.IOException;
 public class FakeSourceToConsoleIT extends FlinkContainer {
 
     @Test
-    @SuppressWarnings("magicnumber")
     public void testFakeSourceToConsoleSink() throws IOException, InterruptedException {
         Container.ExecResult execResult = executeSeaTunnelFlinkJob("/fake/fakesource_to_console.conf");
         Assert.assertEquals(0, execResult.getExitCode());
diff --git a/seatunnel-e2e/seatunnel-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/clickhouse/FakeSourceToClickhouseIT.java b/seatunnel-e2e/seatunnel-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/clickhouse/FakeSourceToClickhouseIT.java
index adbea7806..1bf0fc6e5 100644
--- a/seatunnel-e2e/seatunnel-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/clickhouse/FakeSourceToClickhouseIT.java
+++ b/seatunnel-e2e/seatunnel-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/clickhouse/FakeSourceToClickhouseIT.java
@@ -50,7 +50,6 @@ public class FakeSourceToClickhouseIT extends FlinkContainer {
     private static final Logger LOGGER = LoggerFactory.getLogger(FakeSourceToClickhouseIT.class);
 
     @Before
-    @SuppressWarnings("magicnumber")
     public void startClickhouseContainer() throws InterruptedException {
         clickhouseServer = new GenericContainer<>(CLICKHOUSE_DOCKER_IMAGE)
             .withNetwork(NETWORK)
diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/fake/FakeSourceToConsoleIT.java b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/fake/FakeSourceToConsoleIT.java
index 5f35135bb..0749ec06f 100644
--- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/fake/FakeSourceToConsoleIT.java
+++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/fake/FakeSourceToConsoleIT.java
@@ -32,7 +32,6 @@ import java.io.IOException;
 public class FakeSourceToConsoleIT extends SparkContainer {
 
     @Test
-    @SuppressWarnings("magicnumber")
     public void testFakeSourceToConsoleSine() throws IOException, InterruptedException {
         Container.ExecResult execResult = executeSeaTunnelSparkJob("/fake/fakesource_to_console.conf");
         Assert.assertEquals(0, execResult.getExitCode());
diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/file/FakeSourceToFileIT.java b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/file/FakeSourceToFileIT.java
index 5ebe51d47..c2aeec513 100644
--- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/file/FakeSourceToFileIT.java
+++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/file/FakeSourceToFileIT.java
@@ -32,7 +32,6 @@ import java.io.IOException;
 public class FakeSourceToFileIT extends SparkContainer {
 
     @Test
-    @SuppressWarnings("magicnumber")
     public void testFakeSourceToFile() throws IOException, InterruptedException {
         Container.ExecResult execResult = executeSeaTunnelSparkJob("/file/fakesource_to_file.conf");
         Assert.assertEquals(0, execResult.getExitCode());
diff --git a/seatunnel-examples/seatunnel-flink-connector-v2-example/pom.xml b/seatunnel-examples/seatunnel-flink-connector-v2-example/pom.xml
index 68e0ad2e8..215a586f8 100644
--- a/seatunnel-examples/seatunnel-flink-connector-v2-example/pom.xml
+++ b/seatunnel-examples/seatunnel-flink-connector-v2-example/pom.xml
@@ -56,6 +56,11 @@
             <artifactId>connector-console</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.seatunnel</groupId>
+            <artifactId>connector-file-local</artifactId>
+            <version>${project.version}</version>
+        </dependency>
         <dependency>
             <groupId>org.apache.seatunnel</groupId>
             <artifactId>connector-socket</artifactId>
@@ -103,4 +108,4 @@
 
     </dependencies>
 
-</project>
\ No newline at end of file
+</project>
diff --git a/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/resources/examples/fakesource_to_file.conf b/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/resources/examples/fakesource_to_file.conf
index c1ce63055..f7b790c40 100644
--- a/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/resources/examples/fakesource_to_file.conf
+++ b/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/resources/examples/fakesource_to_file.conf
@@ -21,8 +21,8 @@
 env {
   # You can set flink configuration here
   execution.parallelism = 1
-  job.mode = "STREAMING"
-  execution.checkpoint.interval = 5000
+  job.mode = "BATCH"
+  execution.checkpoint.interval = 10
   #execution.checkpoint.data-uri = "hdfs://localhost:9000/checkpoint"
 }
 
@@ -47,7 +47,7 @@ transform {
 }
 
 sink {
-  File {
+  LocalFile {
     path="file:///tmp/hive/warehouse/test2"
     field_delimiter="\t"
     row_delimiter="\n"
diff --git a/seatunnel-examples/seatunnel-spark-connector-v2-example/pom.xml b/seatunnel-examples/seatunnel-spark-connector-v2-example/pom.xml
index be2abd0b0..8c3978e53 100644
--- a/seatunnel-examples/seatunnel-spark-connector-v2-example/pom.xml
+++ b/seatunnel-examples/seatunnel-spark-connector-v2-example/pom.xml
@@ -80,12 +80,12 @@
             <version>${spark.version}</version>
             <scope>${spark.scope}</scope>
         </dependency>
+
         <dependency>
             <groupId>net.jpountz.lz4</groupId>
             <artifactId>lz4</artifactId>
             <version>1.3.0</version>
         </dependency>
-
     </dependencies>
 
 </project>
\ No newline at end of file
diff --git a/seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/java/org/apache/seatunnel/example/spark/v2/SeaTunnelApiExample.java b/seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/java/org/apache/seatunnel/example/spark/v2/SeaTunnelApiExample.java
index c1e07dd75..a4a84986c 100644
--- a/seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/java/org/apache/seatunnel/example/spark/v2/SeaTunnelApiExample.java
+++ b/seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/java/org/apache/seatunnel/example/spark/v2/SeaTunnelApiExample.java
@@ -38,9 +38,9 @@ public class SeaTunnelApiExample {
         sparkCommandArgs.setCheckConfig(false);
         sparkCommandArgs.setVariables(null);
         sparkCommandArgs.setDeployMode(DeployMode.CLIENT);
-        Command<SparkCommandArgs> flinkCommand =
+        Command<SparkCommandArgs> sparkCommand =
                 new SparkCommandBuilder().buildCommand(sparkCommandArgs);
-        Seatunnel.run(flinkCommand);
+        Seatunnel.run(sparkCommand);
     }
 
     public static String getTestConfigFile(String configFile) throws FileNotFoundException, URISyntaxException {
diff --git a/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/AbstractPluginDiscovery.java b/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/AbstractPluginDiscovery.java
index ccd85bbf7..529049455 100644
--- a/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/AbstractPluginDiscovery.java
+++ b/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/AbstractPluginDiscovery.java
@@ -20,6 +20,7 @@ package org.apache.seatunnel.plugin.discovery;
 import org.apache.seatunnel.api.common.PluginIdentifierInterface;
 import org.apache.seatunnel.apis.base.plugin.Plugin;
 import org.apache.seatunnel.common.config.Common;
+import org.apache.seatunnel.common.utils.ReflectionUtils;
 
 import org.apache.seatunnel.shade.com.typesafe.config.Config;
 import org.apache.seatunnel.shade.com.typesafe.config.ConfigValue;
@@ -29,6 +30,8 @@ import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.annotation.Nullable;
+
 import java.io.File;
 import java.io.FileFilter;
 import java.net.MalformedURLException;
@@ -40,6 +43,7 @@ import java.util.Map;
 import java.util.Optional;
 import java.util.ServiceLoader;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiConsumer;
 import java.util.stream.Collectors;
 
 public abstract class AbstractPluginDiscovery<T> implements PluginDiscovery<T> {
@@ -47,8 +51,26 @@ public abstract class AbstractPluginDiscovery<T> implements PluginDiscovery<T> {
     private static final Logger LOGGER = LoggerFactory.getLogger(AbstractPluginDiscovery.class);
     private final Path pluginDir;
 
+    /**
+     * Add jar url to classloader. The different engine should have different logic to add url into
+     * their own classloader
+     */
+    private BiConsumer<ClassLoader, URL> addURLToClassLoader = (classLoader, url) -> {
+        if (classLoader instanceof URLClassLoader) {
+            ReflectionUtils.invoke(classLoader, "addURL", url);
+        } else {
+            throw new UnsupportedOperationException("can't support custom load jar");
+        }
+    };
+
     protected final ConcurrentHashMap<PluginIdentifier, Optional<URL>> pluginJarPath =
-        new ConcurrentHashMap<>(Common.COLLECTION_SIZE);
+            new ConcurrentHashMap<>(Common.COLLECTION_SIZE);
+
+    public AbstractPluginDiscovery(String pluginSubDir, BiConsumer<ClassLoader, URL> addURLToClassloader) {
+        this.pluginDir = Common.connectorJarDir(pluginSubDir);
+        this.addURLToClassLoader = addURLToClassloader;
+        LOGGER.info("Load {} Plugin from {}", getPluginBaseClass().getSimpleName(), pluginDir);
+    }
 
     public AbstractPluginDiscovery(String pluginSubDir) {
         this.pluginDir = Common.connectorJarDir(pluginSubDir);
@@ -58,10 +80,10 @@ public abstract class AbstractPluginDiscovery<T> implements PluginDiscovery<T> {
     @Override
     public List<URL> getPluginJarPaths(List<PluginIdentifier> pluginIdentifiers) {
         return pluginIdentifiers.stream()
-            .map(this::getPluginJarPath)
-            .filter(Optional::isPresent)
-            .map(Optional::get).distinct()
-            .collect(Collectors.toList());
+                .map(this::getPluginJarPath)
+                .filter(Optional::isPresent)
+                .map(Optional::get).distinct()
+                .collect(Collectors.toList());
     }
 
     @Override
@@ -73,16 +95,35 @@ public abstract class AbstractPluginDiscovery<T> implements PluginDiscovery<T> {
 
     @Override
     public T createPluginInstance(PluginIdentifier pluginIdentifier) {
+        ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
+        T pluginInstance = loadPluginInstance(pluginIdentifier, classLoader);
+        if (pluginInstance != null) {
+            LOGGER.info("Load plugin: {} from classpath", pluginIdentifier);
+            return pluginInstance;
+        }
         Optional<URL> pluginJarPath = getPluginJarPath(pluginIdentifier);
-        ClassLoader classLoader;
-        // if the plugin jar not exist in plugin dir, will load from classpath.
+        // if the plugin jar not exist in classpath, will load from plugin dir.
         if (pluginJarPath.isPresent()) {
-            LOGGER.info("Load plugin: {} from path: {}", pluginIdentifier, pluginJarPath.get());
-            classLoader = new URLClassLoader(new URL[]{pluginJarPath.get()}, Thread.currentThread().getContextClassLoader());
-        } else {
-            LOGGER.info("Load plugin: {} from classpath", pluginIdentifier);
-            classLoader = Thread.currentThread().getContextClassLoader();
+            try {
+                // use current thread classloader to avoid different classloader load same class error.
+                this.addURLToClassLoader.accept(classLoader, pluginJarPath.get());
+            } catch (Exception e) {
+                LOGGER.warn("can't load jar use current thread classloader, use URLClassLoader instead now." +
+                        " message: " + e.getMessage());
+                classLoader = new URLClassLoader(new URL[]{pluginJarPath.get()}, Thread.currentThread().getContextClassLoader());
+            }
+            pluginInstance = loadPluginInstance(pluginIdentifier, classLoader);
+            if (pluginInstance != null) {
+                LOGGER.info("Load plugin: {} from path: {} use classloader: {}",
+                        pluginIdentifier, pluginJarPath.get(), classLoader.getClass().getName());
+                return pluginInstance;
+            }
         }
+        throw new RuntimeException("Plugin " + pluginIdentifier + " not found.");
+    }
+
+    @Nullable
+    private T loadPluginInstance(PluginIdentifier pluginIdentifier, ClassLoader classLoader) {
         ServiceLoader<T> serviceLoader = ServiceLoader.load(getPluginBaseClass(), classLoader);
         for (T t : serviceLoader) {
             if (t instanceof Plugin) {
@@ -101,7 +142,7 @@ public abstract class AbstractPluginDiscovery<T> implements PluginDiscovery<T> {
                 throw new UnsupportedOperationException("Plugin instance: " + t + " is not supported.");
             }
         }
-        throw new RuntimeException("Plugin " + pluginIdentifier + " not found.");
+        return null;
     }
 
     /**
diff --git a/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/PluginIdentifier.java b/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/PluginIdentifier.java
index 97d6e9f81..37e322b73 100644
--- a/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/PluginIdentifier.java
+++ b/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/PluginIdentifier.java
@@ -70,7 +70,6 @@ public class PluginIdentifier {
     }
 
     @Override
-    @SuppressWarnings("checkstyle:magicnumber")
     public int hashCode() {
         int result = engineType != null ? engineType.hashCode() : 0;
         result = 31 * result + (pluginType != null ? pluginType.hashCode() : 0);
diff --git a/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/flink/FlinkTransformPluginDiscovery.java b/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/flink/FlinkAbstractPluginDiscovery.java
similarity index 52%
copy from seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/flink/FlinkTransformPluginDiscovery.java
copy to seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/flink/FlinkAbstractPluginDiscovery.java
index cc77b49a0..a9956fd3f 100644
--- a/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/flink/FlinkTransformPluginDiscovery.java
+++ b/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/flink/FlinkAbstractPluginDiscovery.java
@@ -17,27 +17,24 @@
 
 package org.apache.seatunnel.plugin.discovery.flink;
 
-import org.apache.seatunnel.flink.BaseFlinkTransform;
+import org.apache.seatunnel.common.utils.ReflectionUtils;
 import org.apache.seatunnel.plugin.discovery.AbstractPluginDiscovery;
-import org.apache.seatunnel.plugin.discovery.PluginIdentifier;
 
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.List;
+import java.net.URLClassLoader;
 
-public class FlinkTransformPluginDiscovery extends AbstractPluginDiscovery<BaseFlinkTransform> {
+public abstract class FlinkAbstractPluginDiscovery<T> extends AbstractPluginDiscovery<T> {
 
-    public FlinkTransformPluginDiscovery() {
-        super("flink");
+    public FlinkAbstractPluginDiscovery(String pluginSubDir) {
+        super(pluginSubDir, (classLoader, url) -> {
+            if (classLoader.getClass().getName().endsWith("SafetyNetWrapperClassLoader")) {
+                URLClassLoader c = (URLClassLoader) ReflectionUtils.getField(classLoader, "inner").get();
+                ReflectionUtils.invoke(c, "addURL", url);
+            } else if (classLoader instanceof URLClassLoader) {
+                ReflectionUtils.invoke(classLoader, "addURL", url);
+            } else {
+                throw new RuntimeException("Unsupported classloader: " + classLoader.getClass().getName());
+            }
+        });
     }
 
-    @Override
-    public List<URL> getPluginJarPaths(List<PluginIdentifier> pluginIdentifiers) {
-        return new ArrayList<>();
-    }
-
-    @Override
-    protected Class<BaseFlinkTransform> getPluginBaseClass() {
-        return BaseFlinkTransform.class;
-    }
 }
diff --git a/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/flink/FlinkSinkPluginDiscovery.java b/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/flink/FlinkSinkPluginDiscovery.java
index a2185a4ae..8c973cd0e 100644
--- a/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/flink/FlinkSinkPluginDiscovery.java
+++ b/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/flink/FlinkSinkPluginDiscovery.java
@@ -18,9 +18,8 @@
 package org.apache.seatunnel.plugin.discovery.flink;
 
 import org.apache.seatunnel.flink.BaseFlinkSink;
-import org.apache.seatunnel.plugin.discovery.AbstractPluginDiscovery;
 
-public class FlinkSinkPluginDiscovery extends AbstractPluginDiscovery<BaseFlinkSink> {
+public class FlinkSinkPluginDiscovery extends FlinkAbstractPluginDiscovery<BaseFlinkSink> {
 
     public FlinkSinkPluginDiscovery() {
         super("flink");
diff --git a/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/flink/FlinkSourcePluginDiscovery.java b/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/flink/FlinkSourcePluginDiscovery.java
index 24ff89e03..fd9e41564 100644
--- a/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/flink/FlinkSourcePluginDiscovery.java
+++ b/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/flink/FlinkSourcePluginDiscovery.java
@@ -18,9 +18,8 @@
 package org.apache.seatunnel.plugin.discovery.flink;
 
 import org.apache.seatunnel.flink.BaseFlinkSource;
-import org.apache.seatunnel.plugin.discovery.AbstractPluginDiscovery;
 
-public class FlinkSourcePluginDiscovery extends AbstractPluginDiscovery<BaseFlinkSource> {
+public class FlinkSourcePluginDiscovery extends FlinkAbstractPluginDiscovery<BaseFlinkSource> {
     public FlinkSourcePluginDiscovery() {
         super("flink");
     }
diff --git a/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/flink/FlinkTransformPluginDiscovery.java b/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/flink/FlinkTransformPluginDiscovery.java
index cc77b49a0..12a91d088 100644
--- a/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/flink/FlinkTransformPluginDiscovery.java
+++ b/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/flink/FlinkTransformPluginDiscovery.java
@@ -18,14 +18,13 @@
 package org.apache.seatunnel.plugin.discovery.flink;
 
 import org.apache.seatunnel.flink.BaseFlinkTransform;
-import org.apache.seatunnel.plugin.discovery.AbstractPluginDiscovery;
 import org.apache.seatunnel.plugin.discovery.PluginIdentifier;
 
 import java.net.URL;
 import java.util.ArrayList;
 import java.util.List;
 
-public class FlinkTransformPluginDiscovery extends AbstractPluginDiscovery<BaseFlinkTransform> {
+public class FlinkTransformPluginDiscovery extends FlinkAbstractPluginDiscovery<BaseFlinkTransform> {
 
     public FlinkTransformPluginDiscovery() {
         super("flink");
diff --git a/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/seatunnel/SeaTunnelSinkPluginDiscovery.java b/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/seatunnel/SeaTunnelSinkPluginDiscovery.java
index d3286c544..e2ca9427e 100644
--- a/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/seatunnel/SeaTunnelSinkPluginDiscovery.java
+++ b/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/seatunnel/SeaTunnelSinkPluginDiscovery.java
@@ -20,12 +20,19 @@ package org.apache.seatunnel.plugin.discovery.seatunnel;
 import org.apache.seatunnel.api.sink.SeaTunnelSink;
 import org.apache.seatunnel.plugin.discovery.AbstractPluginDiscovery;
 
+import java.net.URL;
+import java.util.function.BiConsumer;
+
 public class SeaTunnelSinkPluginDiscovery extends AbstractPluginDiscovery<SeaTunnelSink> {
 
     public SeaTunnelSinkPluginDiscovery() {
         super("seatunnel");
     }
 
+    public SeaTunnelSinkPluginDiscovery(BiConsumer<ClassLoader, URL> addURLToClassLoader) {
+        super("seatunnel", addURLToClassLoader);
+    }
+
     @Override
     protected Class<SeaTunnelSink> getPluginBaseClass() {
         return SeaTunnelSink.class;
diff --git a/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/seatunnel/SeaTunnelSourcePluginDiscovery.java b/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/seatunnel/SeaTunnelSourcePluginDiscovery.java
index 8618e0378..f9da2a0a9 100644
--- a/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/seatunnel/SeaTunnelSourcePluginDiscovery.java
+++ b/seatunnel-plugin-discovery/src/main/java/org/apache/seatunnel/plugin/discovery/seatunnel/SeaTunnelSourcePluginDiscovery.java
@@ -20,11 +20,19 @@ package org.apache.seatunnel.plugin.discovery.seatunnel;
 import org.apache.seatunnel.api.source.SeaTunnelSource;
 import org.apache.seatunnel.plugin.discovery.AbstractPluginDiscovery;
 
+import java.net.URL;
+import java.util.function.BiConsumer;
+
 public class SeaTunnelSourcePluginDiscovery extends AbstractPluginDiscovery<SeaTunnelSource> {
+
     public SeaTunnelSourcePluginDiscovery() {
         super("seatunnel");
     }
 
+    public SeaTunnelSourcePluginDiscovery(BiConsumer<ClassLoader, URL> addURLToClassLoader) {
+        super("seatunnel", addURLToClassLoader);
+    }
+
     @Override
     protected Class<SeaTunnelSource> getPluginBaseClass() {
         return SeaTunnelSource.class;
diff --git a/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/common/SeatunnelErrorEnum.java b/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/common/SeatunnelErrorEnum.java
index 3f770b725..5e54ef5fa 100644
--- a/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/common/SeatunnelErrorEnum.java
+++ b/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/common/SeatunnelErrorEnum.java
@@ -24,7 +24,7 @@ public enum SeatunnelErrorEnum {
     USER_ALREADY_EXISTS(10003, "user already exist", "The same username [%s] is exist."),
     NO_SUCH_USER(10002, "no such user", "No such user. Maybe deleted by others."),
     ILLEGAL_STATE(99998, "illegal state", "%s"),
-    UNKNOWN(99999, "unknown exception", "unknown exception")
+    UNKNOWN(99999, "unknown exception", "%s")
     ;
 
     private final int code;
diff --git a/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/util/GlobalExceptionHandler.java b/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/util/GlobalExceptionHandler.java
index 3b3678c30..511dbe597 100644
--- a/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/util/GlobalExceptionHandler.java
+++ b/seatunnel-server/seatunnel-app/src/main/java/org/apache/seatunnel/app/util/GlobalExceptionHandler.java
@@ -52,11 +52,11 @@ public class GlobalExceptionHandler {
     @ExceptionHandler(value = Exception.class)
     private Result<String> exceptionHandler(Exception e) {
         logError(e);
-        return Result.failure(SeatunnelErrorEnum.UNKNOWN);
+        return Result.failure(SeatunnelErrorEnum.UNKNOWN, e.getMessage());
     }
 
     private void logError(Throwable throwable) {
-        log.error("", throwable);
+        log.error(throwable.getMessage(), throwable);
     }
 
 }
diff --git a/seatunnel-translation/seatunnel-translation-base/src/main/java/org/apache/seatunnel/translation/source/CoordinatedSource.java b/seatunnel-translation/seatunnel-translation-base/src/main/java/org/apache/seatunnel/translation/source/CoordinatedSource.java
index 549ce2464..3994fc776 100644
--- a/seatunnel-translation/seatunnel-translation-base/src/main/java/org/apache/seatunnel/translation/source/CoordinatedSource.java
+++ b/seatunnel-translation/seatunnel-translation-base/src/main/java/org/apache/seatunnel/translation/source/CoordinatedSource.java
@@ -27,6 +27,7 @@ import org.apache.seatunnel.api.source.SourceSplitEnumerator;
 import org.apache.seatunnel.translation.util.ThreadPoolExecutorFactory;
 
 import java.io.IOException;
+import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -39,7 +40,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.Collectors;
 
-public class CoordinatedSource<T, SplitT extends SourceSplit, StateT> implements BaseSourceFunction<T> {
+public class CoordinatedSource<T, SplitT extends SourceSplit, StateT extends Serializable> implements BaseSourceFunction<T> {
     protected static final long SLEEP_TIME_INTERVAL = 5L;
     protected final SeaTunnelSource<T, SplitT, StateT> source;
     protected final Map<Integer, List<byte[]>> restoredState;
diff --git a/seatunnel-translation/seatunnel-translation-base/src/main/java/org/apache/seatunnel/translation/source/ParallelSource.java b/seatunnel-translation/seatunnel-translation-base/src/main/java/org/apache/seatunnel/translation/source/ParallelSource.java
index ab0940d6e..5495a2faa 100644
--- a/seatunnel-translation/seatunnel-translation-base/src/main/java/org/apache/seatunnel/translation/source/ParallelSource.java
+++ b/seatunnel-translation/seatunnel-translation-base/src/main/java/org/apache/seatunnel/translation/source/ParallelSource.java
@@ -28,6 +28,7 @@ import org.apache.seatunnel.api.source.SourceSplitEnumerator;
 import org.apache.seatunnel.translation.util.ThreadPoolExecutorFactory;
 
 import java.io.IOException;
+import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -36,7 +37,7 @@ import java.util.Map;
 import java.util.concurrent.Future;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 
-public class ParallelSource<T, SplitT extends SourceSplit, StateT> implements BaseSourceFunction<T> {
+public class ParallelSource<T, SplitT extends SourceSplit, StateT extends Serializable> implements BaseSourceFunction<T> {
 
     protected final SeaTunnelSource<T, SplitT, StateT> source;
     protected final ParallelEnumeratorContext<SplitT> parallelEnumeratorContext;
diff --git a/seatunnel-translation/seatunnel-translation-flink/src/test/java/org/apache/seatunnel/translation/flink/utils/TypeConverterUtilsTest.java b/seatunnel-translation/seatunnel-translation-flink/src/test/java/org/apache/seatunnel/translation/flink/utils/TypeConverterUtilsTest.java
index 9eb6f959f..b5343fb92 100644
--- a/seatunnel-translation/seatunnel-translation-flink/src/test/java/org/apache/seatunnel/translation/flink/utils/TypeConverterUtilsTest.java
+++ b/seatunnel-translation/seatunnel-translation-flink/src/test/java/org/apache/seatunnel/translation/flink/utils/TypeConverterUtilsTest.java
@@ -73,7 +73,6 @@ public class TypeConverterUtilsTest {
     }
 
     @Test
-    @SuppressWarnings("MagicNumber")
     public void convertBigDecimalType() {
         Assert.assertEquals(new BigDecimalTypeInfo(30, 2), TypeConverterUtils.convert(new DecimalType(30, 2)));
     }
diff --git a/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/batch/CoordinatedBatchPartitionReader.java b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/batch/CoordinatedBatchPartitionReader.java
index 27413cac2..939984b2f 100644
--- a/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/batch/CoordinatedBatchPartitionReader.java
+++ b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/batch/CoordinatedBatchPartitionReader.java
@@ -26,6 +26,7 @@ import org.apache.seatunnel.translation.source.BaseSourceFunction;
 import org.apache.seatunnel.translation.source.CoordinatedSource;
 import org.apache.seatunnel.translation.spark.source.InternalRowCollector;
 
+import java.io.Serializable;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -55,7 +56,7 @@ public class CoordinatedBatchPartitionReader extends ParallelBatchPartitionReade
             parallelism);
     }
 
-    public class InternalCoordinatedSource<SplitT extends SourceSplit, StateT> extends CoordinatedSource<SeaTunnelRow, SplitT, StateT> {
+    public class InternalCoordinatedSource<SplitT extends SourceSplit, StateT extends Serializable> extends CoordinatedSource<SeaTunnelRow, SplitT, StateT> {
 
         public InternalCoordinatedSource(SeaTunnelSource<SeaTunnelRow, SplitT, StateT> source, Map<Integer, List<byte[]>> restoredState, int parallelism) {
             super(source, restoredState, parallelism);
diff --git a/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/batch/ParallelBatchPartitionReader.java b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/batch/ParallelBatchPartitionReader.java
index aa50a0bdf..e8b958346 100644
--- a/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/batch/ParallelBatchPartitionReader.java
+++ b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/batch/ParallelBatchPartitionReader.java
@@ -32,6 +32,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.io.Serializable;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ExecutorService;
@@ -132,7 +133,7 @@ public class ParallelBatchPartitionReader implements InputPartitionReader<Intern
         executorService.shutdown();
     }
 
-    public class InternalParallelSource<SplitT extends SourceSplit, StateT> extends ParallelSource<SeaTunnelRow, SplitT, StateT> {
+    public class InternalParallelSource<SplitT extends SourceSplit, StateT extends Serializable> extends ParallelSource<SeaTunnelRow, SplitT, StateT> {
 
         public InternalParallelSource(SeaTunnelSource<SeaTunnelRow, SplitT, StateT> source, Map<Integer, List<byte[]>> restoredState, int parallelism, int subtaskId) {
             super(source, restoredState, parallelism, subtaskId);
diff --git a/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/micro/CoordinatedMicroBatchPartitionReader.java b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/micro/CoordinatedMicroBatchPartitionReader.java
index 0072a188a..761add0a9 100644
--- a/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/micro/CoordinatedMicroBatchPartitionReader.java
+++ b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/micro/CoordinatedMicroBatchPartitionReader.java
@@ -27,6 +27,7 @@ import org.apache.seatunnel.translation.source.CoordinatedSource;
 import org.apache.seatunnel.translation.spark.source.InternalRowCollector;
 import org.apache.seatunnel.translation.spark.source.ReaderState;
 
+import java.io.Serializable;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -96,7 +97,7 @@ public class CoordinatedMicroBatchPartitionReader extends ParallelMicroBatchPart
             parallelism);
     }
 
-    public class InternalCoordinatedSource<SplitT extends SourceSplit, StateT> extends CoordinatedSource<SeaTunnelRow, SplitT, StateT> {
+    public class InternalCoordinatedSource<SplitT extends SourceSplit, StateT extends Serializable> extends CoordinatedSource<SeaTunnelRow, SplitT, StateT> {
 
         public InternalCoordinatedSource(SeaTunnelSource<SeaTunnelRow, SplitT, StateT> source, Map<Integer, List<byte[]>> restoredState, int parallelism) {
             super(source, restoredState, parallelism);
diff --git a/seatunnel-ui/package.json b/seatunnel-ui/package.json
index dd1b8631e..2b6eaf6d5 100644
--- a/seatunnel-ui/package.json
+++ b/seatunnel-ui/package.json
@@ -22,6 +22,9 @@
     "vfonts": "^0.0.3",
     "vue": "^3.2.37",
     "vue-i18n": "^9.1.10",
+    "tailwindcss": "^3.1.6",
+    "postcss": "^8.4.14",
+    "autoprefixer": "^10.4.7",
     "vue-router": "^4.0.16"
   },
   "devDependencies": {
diff --git a/seatunnel-ui/pnpm-lock.yaml b/seatunnel-ui/pnpm-lock.yaml
index c5818fed9..1196b0833 100644
--- a/seatunnel-ui/pnpm-lock.yaml
+++ b/seatunnel-ui/pnpm-lock.yaml
@@ -27,6 +27,7 @@ specifiers:
   '@vitejs/plugin-vue': ^2.3.3
   '@vitejs/plugin-vue-jsx': ^1.3.10
   '@vueuse/core': ^8.7.5
+  autoprefixer: ^10.4.7
   axios: ^0.27.2
   dart-sass: ^1.25.0
   date-fns: ^2.28.0
@@ -41,9 +42,11 @@ specifiers:
   nprogress: ^0.2.0
   pinia: ^2.0.14
   pinia-plugin-persistedstate: ^1.6.1
+  postcss: ^8.4.14
   prettier: ^2.7.1
   sass: ^1.53.0
   sass-loader: ^13.0.2
+  tailwindcss: ^3.1.6
   typescript: ^4.7.4
   typescript-plugin-css-modules: ^3.4.0
   vfonts: ^0.0.3
@@ -79,14 +82,17 @@ devDependencies:
   '@vicons/antd': 0.12.0
   '@vitejs/plugin-vue': 2.3.3_vite@2.9.13+vue@3.2.37
   '@vitejs/plugin-vue-jsx': 1.3.10
+  autoprefixer: 10.4.7_postcss@8.4.14
   dart-sass: 1.25.0
   eslint: 8.18.0
   eslint-config-prettier: 8.5.0_eslint@8.18.0
   eslint-plugin-prettier: 4.1.0_xu6ewijrtliw5q5lksq5uixwby
   eslint-plugin-vue: 9.1.1_eslint@8.18.0
+  postcss: 8.4.14
   prettier: 2.7.1
   sass: 1.53.0
   sass-loader: 13.0.2_sass@1.53.0
+  tailwindcss: 3.1.6
   typescript: 4.7.4
   typescript-plugin-css-modules: 3.4.0_typescript@4.7.4
   vite: 2.9.13_sass@1.53.0
@@ -934,6 +940,25 @@ packages:
       acorn: 8.7.1
     dev: true
 
+  /acorn-node/1.8.2:
+    resolution: {integrity: sha512-8mt+fslDufLYntIoPAaIMUe/lrbrehIiwmR3t2k9LljIzoigEPF27eLk2hy8zSGzmR/ogr7zbRKINMo1u0yh5A==}
+    dependencies:
+      acorn: 7.4.1
+      acorn-walk: 7.2.0
+      xtend: 4.0.2
+    dev: true
+
+  /acorn-walk/7.2.0:
+    resolution: {integrity: sha512-OPdCF6GsMIP+Az+aWfAAOEt2/+iVDKE7oy6lJ098aoe59oAmK76qV6Gw60SbZ8jHuG2wH058GF4pLFbYamYrVA==}
+    engines: {node: '>=0.4.0'}
+    dev: true
+
+  /acorn/7.4.1:
+    resolution: {integrity: sha512-nQyp0o1/mNdbTO1PO6kHkwSrmgZ0MT/jCCpNiwbUjGoRN4dlBhqJtoQuCnEOKzgTVwg0ZWiCoQy6SxMebQVh8A==}
+    engines: {node: '>=0.4.0'}
+    hasBin: true
+    dev: true
+
   /acorn/8.7.1:
     resolution: {integrity: sha512-Xx54uLJQZ19lKygFXOWsscKUbsBZW0CPykPhVQdhIeIwrbPmJzqeASDInc8nKBnp/JT6igTs82qPXz069H8I/A==}
     engines: {node: '>=0.4.0'}
@@ -979,6 +1004,10 @@ packages:
       picomatch: 2.3.1
     dev: true
 
+  /arg/5.0.2:
+    resolution: {integrity: sha512-PYjyFOLKQ9y57JvQ6QLo8dAgNqswh8M1RMJYdQduT6xbWSgK36P/Z/v+p888pM69jMMfS8Xd8F6I1kQ/I9HUGg==}
+    dev: true
+
   /argparse/2.0.1:
     resolution: {integrity: sha512-8+9WqebbFzpX9OR+Wa6O29asIogeRMzcGtAINdpMHHyAg10f05aSFVBbcEqGf/PXw1EjAZ+q2/bEBg3DvurK3Q==}
     dev: true
@@ -1002,6 +1031,22 @@ packages:
     hasBin: true
     dev: true
 
+  /autoprefixer/10.4.7_postcss@8.4.14:
+    resolution: {integrity: sha512-ypHju4Y2Oav95SipEcCcI5J7CGPuvz8oat7sUtYj3ClK44bldfvtvcxK6IEK++7rqB7YchDGzweZIBG+SD0ZAA==}
+    engines: {node: ^10 || ^12 || >=14}
+    hasBin: true
+    peerDependencies:
+      postcss: ^8.1.0
+    dependencies:
+      browserslist: 4.21.1
+      caniuse-lite: 1.0.30001361
+      fraction.js: 4.2.0
+      normalize-range: 0.1.2
+      picocolors: 1.0.0
+      postcss: 8.4.14
+      postcss-value-parser: 4.2.0
+    dev: true
+
   /axios/0.27.2:
     resolution: {integrity: sha512-t+yRIyySRTp/wua5xEr+z1q60QmLq8ABsS5O9Me1AsE5dfKqgnCFzwiCZZ/cGNd1lq4/7akDWMxdhVlucjmnOQ==}
     dependencies:
@@ -1058,6 +1103,11 @@ packages:
     engines: {node: '>=6'}
     dev: true
 
+  /camelcase-css/2.0.1:
+    resolution: {integrity: sha512-QOSvevhslijgYwRx6Rv7zKdMF8lbRmx+uQGx2+vDc+KI/eBnsy9kit5aj23AgGu3pa4t9AgwbnXWqS+iOY+2aA==}
+    engines: {node: '>= 6'}
+    dev: true
+
   /camelcase/6.3.0:
     resolution: {integrity: sha512-Gmy6FhYlCY7uOElZUSbxo2UCDH8owEk996gkbrpsgGtrJLM3J7jGxl9Ic7Qwwj4ivOE5AWZWRMecDdF7hqGjFA==}
     engines: {node: '>=10'}
@@ -1258,11 +1308,29 @@ packages:
     resolution: {integrity: sha512-oIPzksmTg4/MriiaYGO+okXDT7ztn/w3Eptv/+gSIdMdKsJo0u4CfYNFJPy+4SKMuCqGw2wxnA+URMg3t8a/bQ==}
     dev: true
 
+  /defined/1.0.0:
+    resolution: {integrity: sha512-Y2caI5+ZwS5c3RiNDJ6u53VhQHv+hHKwhkI1iHvceKUHw9Df6EK2zRLfjejRgMuCuxK7PfSWIMwWecceVvThjQ==}
+    dev: true
+
   /delayed-stream/1.0.0:
     resolution: {integrity: sha512-ZySD7Nf91aLB0RxL4KGrKHBXl7Eds1DAmEdcoVawXnLD7SDhpNgtuII2aAkg7a7QS41jxPSZ17p4VdGnMHk3MQ==}
     engines: {node: '>=0.4.0'}
     dev: false
 
+  /detective/5.2.1:
+    resolution: {integrity: sha512-v9XE1zRnz1wRtgurGu0Bs8uHKFSTdteYZNbIPFVhUZ39L/S79ppMpdmVOZAnoz1jfEFodc48n6MX483Xo3t1yw==}
+    engines: {node: '>=0.8.0'}
+    hasBin: true
+    dependencies:
+      acorn-node: 1.8.2
+      defined: 1.0.0
+      minimist: 1.2.6
+    dev: true
+
+  /didyoumean/1.2.2:
+    resolution: {integrity: sha512-gxtyfqMg7GKyhQmb056K7M3xszy/myH8w+B4RT+QXBQsvAOdc3XymqDDPHx1BgPgsdAA5SIifona89YtRATDzw==}
+    dev: true
+
   /diff-sequences/27.5.1:
     resolution: {integrity: sha512-k1gCAXAsNgLwEL+Y8Wvl+M6oEFj5bgazfZULpS5CneoPPXRaCCW7dm+q21Ky2VEE5X+VeRDBVg1Pcvvsr4TtNQ==}
     engines: {node: ^10.13.0 || ^12.13.0 || ^14.15.0 || >=15.0.0}
@@ -1275,6 +1343,10 @@ packages:
       path-type: 4.0.0
     dev: true
 
+  /dlv/1.1.3:
+    resolution: {integrity: sha512-+HlytyjlPKnIG8XuRG8WvmBP8xs8P71y+SKKS6ZXWoEgLuePxtDoUEiH7WkdePWrQ5JBpE6aoVqfZfJUQkjXwA==}
+    dev: true
+
   /doctrine/3.0.0:
     resolution: {integrity: sha512-yS+Q5i3hBf7GBkd4KG8a7eBNNWNGLTaEwwYWUijIYM7zrlYDM0BFXHjjPWlWZ1Rg7UaddZeIDmi9jF3HmqiQ2w==}
     engines: {node: '>=6.0.0'}
@@ -1786,6 +1858,10 @@ packages:
       mime-types: 2.1.35
     dev: false
 
+  /fraction.js/4.2.0:
+    resolution: {integrity: sha512-MhLuK+2gUcnZe8ZHlaaINnQLl0xRIGRfcGk2yl8xoQAfHrSsL3rYu6FCmBdkdbhc9EPlwyGHewaRsvwRMJtAlA==}
+    dev: true
+
   /fs-extra/10.1.0:
     resolution: {integrity: sha512-oRXApq54ETRj4eMiFzGnHWGy+zo5raudjuxN0b8H7s/RU2oW0Wvsx9O0ACRN/kRq9E8Vu/ReskGB5o3ji+FzHQ==}
     engines: {node: '>=12'}
@@ -2294,6 +2370,11 @@ packages:
     engines: {node: '>=0.10.0'}
     dev: true
 
+  /normalize-range/0.1.2:
+    resolution: {integrity: sha512-bdok/XvKII3nUpklnV6P2hxtMNrCboOjAcyBuQnWEhO665FwrSNRxU+AqpsyvO6LgGYPspN+lu5CLtw4jPRKNA==}
+    engines: {node: '>=0.10.0'}
+    dev: true
+
   /nprogress/0.2.0:
     resolution: {integrity: sha512-I19aIingLgR1fmhftnbWWO3dXc0hSxqHQHQb3H8m+K3TnEn/iSeTZZOyvKXWqQESMwuUVnatlCnZdLBZZt2VSA==}
     dev: false
@@ -2309,6 +2390,11 @@ packages:
     engines: {node: '>=0.10.0'}
     dev: true
 
+  /object-hash/3.0.0:
+    resolution: {integrity: sha512-RSn9F68PjH9HqtltsSnqYC1XXoWe9Bju5+213R98cNGttag9q9yAOTzdbsqvIa7aNm5WffBZFpWYr2aWrklWAw==}
+    engines: {node: '>= 6'}
+    dev: true
+
   /once/1.4.0:
     resolution: {integrity: sha512-lNaJgI+2Q5URQBkccEKHTQOPaXdUxnZZElQTZY0MFUAuaEqe1E+Nyvgdz/aIyNi6Z9MzO5dv1H8n58/GELp3+w==}
     dependencies:
@@ -2366,6 +2452,11 @@ packages:
     engines: {node: '>=8.6'}
     dev: true
 
+  /pify/2.3.0:
+    resolution: {integrity: sha512-udgsAY+fTnvv7kI7aaxbqwWNb0AHiB0qBO89PZKPkoTmGOgdbrHDKD+0B2X4uTfJ/FT1R09r9gTsjUjNJotuog==}
+    engines: {node: '>=0.10.0'}
+    dev: true
+
   /pify/4.0.1:
     resolution: {integrity: sha512-uB80kBFb/tfd68bVleG9T5GGsGPjJrLAUpR5PZIrhBnIaRTQRjqdJSsIKkOP6OAIFbj7GOrcudc5pNjZ+geV2g==}
     engines: {node: '>=6'}
@@ -2422,6 +2513,28 @@ packages:
       postcss: 6.0.23
     dev: true
 
+  /postcss-import/14.1.0_postcss@8.4.14:
+    resolution: {integrity: sha512-flwI+Vgm4SElObFVPpTIT7SU7R3qk2L7PyduMcokiaVKuWv9d/U+Gm/QAd8NDLuykTWTkcrjOeD2Pp1rMeBTGw==}
+    engines: {node: '>=10.0.0'}
+    peerDependencies:
+      postcss: ^8.0.0
+    dependencies:
+      postcss: 8.4.14
+      postcss-value-parser: 4.2.0
+      read-cache: 1.0.0
+      resolve: 1.22.1
+    dev: true
+
+  /postcss-js/4.0.0_postcss@8.4.14:
+    resolution: {integrity: sha512-77QESFBwgX4irogGVPgQ5s07vLvFqWr228qZY+w6lW599cRlK/HmnlivnnVUxkjHnCu4J16PDMHcH+e+2HbvTQ==}
+    engines: {node: ^12 || ^14 || >= 16}
+    peerDependencies:
+      postcss: ^8.3.3
+    dependencies:
+      camelcase-css: 2.0.1
+      postcss: 8.4.14
+    dev: true
+
   /postcss-load-config/3.1.4_postcss@8.4.14:
     resolution: {integrity: sha512-6DiM4E7v4coTE4uzA8U//WhtPwyhiim3eyjEMFCnUpzbrkK9wJHgKDT2mR+HbtSrd/NubVaYTOpSpjUl8NQeRg==}
     engines: {node: '>= 10'}
@@ -2439,6 +2552,16 @@ packages:
       yaml: 1.10.2
     dev: true
 
+  /postcss-nested/5.0.6_postcss@8.4.14:
+    resolution: {integrity: sha512-rKqm2Fk0KbA8Vt3AdGN0FB9OBOMDVajMG6ZCf/GoHgdxUJ4sBFp0A/uMIRm+MJUdo33YXEtjqIz8u7DAp8B7DA==}
+    engines: {node: '>=12.0'}
+    peerDependencies:
+      postcss: ^8.2.14
+    dependencies:
+      postcss: 8.4.14
+      postcss-selector-parser: 6.0.10
+    dev: true
+
   /postcss-selector-parser/6.0.10:
     resolution: {integrity: sha512-IQ7TZdoaqbT+LCpShg46jnZVlhWD2w6iQYAcYXfHARZ7X1t/UGhhceQDs5X0cGqKvYlHNOuv7Oa1xmb0oQuA3w==}
     engines: {node: '>=4'}
@@ -2451,6 +2574,10 @@ packages:
     resolution: {integrity: sha512-pISE66AbVkp4fDQ7VHBwRNXzAAKJjw4Vw7nWI/+Q3vuly7SNfgYXvm6i5IgFylHGK5sP/xHAbB7N49OS4gWNyQ==}
     dev: true
 
+  /postcss-value-parser/4.2.0:
+    resolution: {integrity: sha512-1NNCs6uurfkVbeXG4S8JFT9t19m45ICnif8zWLd5oPSZ50QnwMfK+H3jv408d4jw/7Bttv5axS5IiHoLaVNHeQ==}
+    dev: true
+
   /postcss/6.0.23:
     resolution: {integrity: sha512-soOk1h6J3VMTZtVeVpv15/Hpdl2cBLX3CAw4TAbkpTJiNPk9YP/zWcD1ND+xEtvyuuvKzbxliTOIyvkSeSJ6ag==}
     engines: {node: '>=4.0.0'}
@@ -2509,10 +2636,21 @@ packages:
     resolution: {integrity: sha512-NuaNSa6flKT5JaSYQzJok04JzTL1CA6aGhv5rfLW3PgqA+M2ChpZQnAC8h8i4ZFkBS8X5RqkDBHA7r4hej3K9A==}
     dev: true
 
+  /quick-lru/5.1.1:
+    resolution: {integrity: sha512-WuyALRjWPDGtt/wzJiadO5AXY+8hZ80hVpe6MyivgraREW751X3SbhRvG3eLKOYN+8VEvqLcf3wdnt44Z4S4SA==}
+    engines: {node: '>=10'}
+    dev: true
+
   /react-is/17.0.2:
     resolution: {integrity: sha512-w2GsyukL62IJnlaff/nRegPQR94C/XXamvMWmSHRJ4y7Ts/4ocGRmTHvOs8PSE6pB3dWOrD/nueuU5sduBsQ4w==}
     dev: false
 
+  /read-cache/1.0.0:
+    resolution: {integrity: sha512-Owdv/Ft7IjOgm/i0xvNDZ1LrRANRfew4b2prF3OWMQLxLfu3bS8FVhCsrSCMK4lR56Y9ya+AThoTpDCTxCmpRA==}
+    dependencies:
+      pify: 2.3.0
+    dev: true
+
   /readdirp/3.6.0:
     resolution: {integrity: sha512-hOS089on8RduqdbhvQ5Z37A0ESjsqz6qnRcffsMU3495FuTdqSm+7bhJ29JvIOsBDEEnan5DPu9t3To9VRlMzA==}
     engines: {node: '>=8.10.0'}
@@ -2749,6 +2887,37 @@ packages:
     resolution: {integrity: sha512-ovssysQTa+luh7A5Weu3Rta6FJlFBBbInjOh722LIt6klpU2/HtdUbszju/G4devcvk8PGt7FCLv5wftu3THUA==}
     dev: true
 
+  /tailwindcss/3.1.6:
+    resolution: {integrity: sha512-7skAOY56erZAFQssT1xkpk+kWt2NrO45kORlxFPXUt3CiGsVPhH1smuH5XoDH6sGPXLyBv+zgCKA2HWBsgCytg==}
+    engines: {node: '>=12.13.0'}
+    hasBin: true
+    dependencies:
+      arg: 5.0.2
+      chokidar: 3.5.3
+      color-name: 1.1.4
+      detective: 5.2.1
+      didyoumean: 1.2.2
+      dlv: 1.1.3
+      fast-glob: 3.2.11
+      glob-parent: 6.0.2
+      is-glob: 4.0.3
+      lilconfig: 2.0.5
+      normalize-path: 3.0.0
+      object-hash: 3.0.0
+      picocolors: 1.0.0
+      postcss: 8.4.14
+      postcss-import: 14.1.0_postcss@8.4.14
+      postcss-js: 4.0.0_postcss@8.4.14
+      postcss-load-config: 3.1.4_postcss@8.4.14
+      postcss-nested: 5.0.6_postcss@8.4.14
+      postcss-selector-parser: 6.0.10
+      postcss-value-parser: 4.2.0
+      quick-lru: 5.1.1
+      resolve: 1.22.1
+    transitivePeerDependencies:
+      - ts-node
+    dev: true
+
   /text-table/0.2.0:
     resolution: {integrity: sha512-N+8UisAXDGk8PFXP4HAzVR9nbfmVJ3zYLAWiTIoqC5v5isinhr+r5uaO8+7r3BMfuNIufIsA7RdpVgacC2cSpw==}
     dev: true
@@ -3046,6 +3215,11 @@ packages:
     engines: {node: '>=12'}
     dev: true
 
+  /xtend/4.0.2:
+    resolution: {integrity: sha512-LKYU1iAXJXUgAXn9URjiu+MWhyUXHsvfp7mcuYm9dSUKK0/CjtrUwFAxD82/mCWbtLsGjFIad0wIsod4zrTAEQ==}
+    engines: {node: '>=0.4'}
+    dev: true
+
   /yallist/4.0.0:
     resolution: {integrity: sha512-3wdGidZyq5PB084XLES5TpOSRA3wjXAlIWMhum2kRcv/41Sn2emQ0dycQW4uZXLejwKvg6EsvbdlVL+FYEct7A==}
     dev: true
diff --git a/seatunnel-ui/src/store/theme/types.ts b/seatunnel-ui/postcss.config.js
similarity index 91%
copy from seatunnel-ui/src/store/theme/types.ts
copy to seatunnel-ui/postcss.config.js
index be5b77260..6ae9aea43 100644
--- a/seatunnel-ui/src/store/theme/types.ts
+++ b/seatunnel-ui/postcss.config.js
@@ -1,4 +1,3 @@
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -16,8 +15,9 @@
  * limitations under the License.
  */
 
-interface ThemeState {
-  darkTheme: boolean
-}
-
-export default ThemeState
\ No newline at end of file
+module.exports = {
+  plugins: {
+    tailwindcss: {},
+    autoprefixer: {}
+  }
+}
\ No newline at end of file
diff --git a/seatunnel-ui/src/App.tsx b/seatunnel-ui/src/App.tsx
index 6a5c8cefb..4b0c75bbc 100644
--- a/seatunnel-ui/src/App.tsx
+++ b/seatunnel-ui/src/App.tsx
@@ -53,11 +53,12 @@ const App = defineComponent({
         date-locale={
           String(this.localesStore.getLocales) === 'zh_CN' ? dateZhCN : dateEnUS
         }
-        locale={String(this.localesStore.getLocales) === 'zh_CN' ? zhCN : enUS}>
+        locale={String(this.localesStore.getLocales) === 'zh_CN' ? zhCN : enUS}
+      >
         <router-view />
       </NConfigProvider>
     )
   }
 })
 
-export default App
\ No newline at end of file
+export default App
diff --git a/seatunnel-ui/src/components/chart/index.ts b/seatunnel-ui/src/components/chart/index.ts
index d0ffd1f3a..944c5ba36 100644
--- a/seatunnel-ui/src/components/chart/index.ts
+++ b/seatunnel-ui/src/components/chart/index.ts
@@ -84,4 +84,4 @@ function initChart<Opt extends ECBasicOption>(
   return chart
 }
 
-export default initChart
\ No newline at end of file
+export default initChart
diff --git a/seatunnel-ui/src/env.d.ts b/seatunnel-ui/src/env.d.ts
index c9402bc15..518937339 100644
--- a/seatunnel-ui/src/env.d.ts
+++ b/seatunnel-ui/src/env.d.ts
@@ -31,4 +31,4 @@ declare global {
 
 declare module '*.png'
 declare module '*.jpg'
-declare module '*.jpeg'
\ No newline at end of file
+declare module '*.jpeg'
diff --git a/seatunnel-ui/src/store/theme/types.ts b/seatunnel-ui/src/index.css
similarity index 87%
copy from seatunnel-ui/src/store/theme/types.ts
copy to seatunnel-ui/src/index.css
index be5b77260..4b0aa3dd9 100644
--- a/seatunnel-ui/src/store/theme/types.ts
+++ b/seatunnel-ui/src/index.css
@@ -1,4 +1,3 @@
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -16,8 +15,9 @@
  * limitations under the License.
  */
 
-interface ThemeState {
-  darkTheme: boolean
-}
-
-export default ThemeState
\ No newline at end of file
+@tailwind base;
+@tailwind components;
+@tailwind utilities;
+@tailwind screens;
+@tailwind fonts;
+@tailwind colors;
\ No newline at end of file
diff --git a/seatunnel-ui/src/views/login/index.tsx b/seatunnel-ui/src/layouts/dashboard/header/index.tsx
similarity index 65%
copy from seatunnel-ui/src/views/login/index.tsx
copy to seatunnel-ui/src/layouts/dashboard/header/index.tsx
index 92e1cfdbc..f732f542a 100644
--- a/seatunnel-ui/src/views/login/index.tsx
+++ b/seatunnel-ui/src/layouts/dashboard/header/index.tsx
@@ -16,18 +16,28 @@
  */
 
 import { defineComponent } from 'vue'
+import { NSpace } from 'naive-ui'
+import Logo from './logo'
+import Menu from './menu'
+import Setting from './setting'
+import User from './user'
 
-const Login = defineComponent({
-  setup() {
-
-  },
+const Header = defineComponent({
+  setup() {},
   render() {
     return (
-      <div>
-        <h1>Login</h1>
-      </div>
+      <NSpace justify='space-between' class='h-16 border-b border-gray-200'>
+        <NSpace>
+          <Logo />
+          <Menu />
+        </NSpace>
+        <NSpace>
+          <Setting />
+          <User />
+        </NSpace>
+      </NSpace>
     )
   }
 })
 
-export default Login
\ No newline at end of file
+export default Header
diff --git a/seatunnel-ui/src/views/login/index.tsx b/seatunnel-ui/src/layouts/dashboard/header/logo/index.tsx
similarity index 79%
copy from seatunnel-ui/src/views/login/index.tsx
copy to seatunnel-ui/src/layouts/dashboard/header/logo/index.tsx
index 92e1cfdbc..361345ccd 100644
--- a/seatunnel-ui/src/views/login/index.tsx
+++ b/seatunnel-ui/src/layouts/dashboard/header/logo/index.tsx
@@ -16,18 +16,17 @@
  */
 
 import { defineComponent } from 'vue'
+import { NSpace } from 'naive-ui'
 
-const Login = defineComponent({
-  setup() {
-
-  },
+const Logo = defineComponent({
+  setup() {},
   render() {
     return (
-      <div>
-        <h1>Login</h1>
-      </div>
+      <NSpace justify='center' align='center' class='h-16 w-48'>
+        <h2 class='text-2xl font-bold'>SeaTunnel</h2>
+      </NSpace>
     )
   }
 })
 
-export default Login
\ No newline at end of file
+export default Logo
diff --git a/seatunnel-ui/src/views/login/index.tsx b/seatunnel-ui/src/layouts/dashboard/header/menu/index.tsx
similarity index 57%
copy from seatunnel-ui/src/views/login/index.tsx
copy to seatunnel-ui/src/layouts/dashboard/header/menu/index.tsx
index 92e1cfdbc..2b54ab349 100644
--- a/seatunnel-ui/src/views/login/index.tsx
+++ b/seatunnel-ui/src/layouts/dashboard/header/menu/index.tsx
@@ -15,19 +15,36 @@
  * limitations under the License.
  */
 
-import { defineComponent } from 'vue'
+import { defineComponent, toRefs } from 'vue'
+import { NMenu, NSpace } from 'naive-ui'
+import { useRouter } from 'vue-router'
+import { useMenu } from './use-menu'
 
-const Login = defineComponent({
+const Menu = defineComponent({
   setup() {
+    const { state } = useMenu()
+    const router = useRouter()
 
+    const handleMenuClick = (key: string) => {
+      router.push({ path: `/${key}` })
+    }
+
+    return {
+      ...toRefs(state),
+      handleMenuClick
+    }
   },
   render() {
     return (
-      <div>
-        <h1>Login</h1>
-      </div>
+      <NSpace align='center' class='h-16'>
+        <NMenu
+          mode='horizontal'
+          options={this.menuOptions}
+          onUpdateValue={this.handleMenuClick}
+        />
+      </NSpace>
     )
   }
 })
 
-export default Login
\ No newline at end of file
+export default Menu
diff --git a/seatunnel-ui/src/store/theme/index.ts b/seatunnel-ui/src/layouts/dashboard/header/menu/use-menu.ts
similarity index 68%
copy from seatunnel-ui/src/store/theme/index.ts
copy to seatunnel-ui/src/layouts/dashboard/header/menu/use-menu.ts
index 016751947..ed895712e 100644
--- a/seatunnel-ui/src/store/theme/index.ts
+++ b/seatunnel-ui/src/layouts/dashboard/header/menu/use-menu.ts
@@ -15,23 +15,23 @@
  * limitations under the License.
  */
 
-import { defineStore } from 'pinia'
-import ThemeState from './types'
+import { reactive, h } from 'vue'
+import { NEllipsis } from 'naive-ui'
+import { useI18n } from 'vue-i18n'
 
-export const useThemeStore = defineStore({
-  id: 'theme',
-  state: (): ThemeState => ({
-    darkTheme: false
-  }),
-  persist: true,
-  getters: {
-    getTheme(): boolean {
-      return this.darkTheme
-    }
-  },
-  actions: {
-    setDarkTheme(): void {
-      this.darkTheme = !this.darkTheme
+export function useMenu() {
+  const { t } = useI18n()
+
+  const menuOptions = [
+    {
+      label: () => h(NEllipsis, null, { default: () => t('menu.data_pipes') }),
+      key: 'data-pipes'
     }
-  }
-})
\ No newline at end of file
+  ]
+
+  const state = reactive({
+    menuOptions
+  })
+
+  return { state }
+}
diff --git a/seatunnel-ui/src/views/login/index.tsx b/seatunnel-ui/src/layouts/dashboard/header/setting/index.tsx
similarity index 53%
copy from seatunnel-ui/src/views/login/index.tsx
copy to seatunnel-ui/src/layouts/dashboard/header/setting/index.tsx
index 92e1cfdbc..291740da3 100644
--- a/seatunnel-ui/src/views/login/index.tsx
+++ b/seatunnel-ui/src/layouts/dashboard/header/setting/index.tsx
@@ -15,19 +15,37 @@
  * limitations under the License.
  */
 
-import { defineComponent } from 'vue'
+import { defineComponent, toRefs } from 'vue'
+import { NIcon, NSpace, NDropdown } from 'naive-ui'
+import { SettingOutlined } from '@vicons/antd'
+import { useSettingDropdown } from './use-setting-dropdown'
 
-const Login = defineComponent({
+const Setting = defineComponent({
   setup() {
+    const { state, handleSelect } = useSettingDropdown()
 
+    return { ...toRefs(state), handleSelect }
   },
   render() {
     return (
-      <div>
-        <h1>Login</h1>
-      </div>
+      <NSpace
+        align='center'
+        justify='center'
+        class='h-16 w-12'
+        style={{ cursor: 'pointer' }}
+      >
+        <NDropdown
+          trigger='click'
+          options={this.dropdownOptions}
+          onSelect={this.handleSelect}
+        >
+          <NIcon size='20'>
+            <SettingOutlined />
+          </NIcon>
+        </NDropdown>
+      </NSpace>
     )
   }
 })
 
-export default Login
\ No newline at end of file
+export default Setting
diff --git a/seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-jdbc/src/main/scala/org/apache/seatunnel/spark/jdbc/source/util/HiveDialet.scala b/seatunnel-ui/src/layouts/dashboard/header/setting/use-setting-dropdown.ts
similarity index 51%
copy from seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-jdbc/src/main/scala/org/apache/seatunnel/spark/jdbc/source/util/HiveDialet.scala
copy to seatunnel-ui/src/layouts/dashboard/header/setting/use-setting-dropdown.ts
index c686cdfba..157f04c53 100644
--- a/seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-jdbc/src/main/scala/org/apache/seatunnel/spark/jdbc/source/util/HiveDialet.scala
+++ b/seatunnel-ui/src/layouts/dashboard/header/setting/use-setting-dropdown.ts
@@ -14,21 +14,39 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.seatunnel.spark.jdbc.source.util
 
-import org.apache.spark.sql.jdbc.JdbcDialect
+import { useI18n } from 'vue-i18n'
+import { reactive, h } from 'vue'
+import { useRouter } from 'vue-router'
+import type { Router } from 'vue-router'
 
-class HiveDialet extends JdbcDialect {
-  override def canHandle(url: String): Boolean = {
-    url.startsWith("jdbc:hive2")
-  }
+export function useSettingDropdown() {
+  const { t } = useI18n()
+  const router: Router = useRouter()
+
+  const dropdownOptions = [
+    {
+      key: 'header',
+      type: 'render',
+      render: () =>
+        h('h3', { class: ['py-1.5', 'px-3', 'font-medium'] }, t('menu.manage'))
+    },
+    {
+      key: 'header-divider',
+      type: 'divider'
+    },
+    { key: 'user-manage', label: t('menu.user_manage') }
+  ]
 
-  override def quoteIdentifier(colName: String): String = {
-    if (colName.contains(".")) {
-      val colName1 = colName.substring(colName.indexOf(".") + 1)
-      s"`$colName1`"
-    } else {
-      s"`$colName`"
+  const state = reactive({
+    dropdownOptions
+  })
+
+  const handleSelect = (key: string) => {
+    if (key === 'user-manage') {
+      router.push({ path: '/user-manage' })
     }
   }
+
+  return { state, handleSelect }
 }
diff --git a/seatunnel-ui/src/views/login/index.tsx b/seatunnel-ui/src/layouts/dashboard/header/user/index.tsx
similarity index 52%
copy from seatunnel-ui/src/views/login/index.tsx
copy to seatunnel-ui/src/layouts/dashboard/header/user/index.tsx
index 92e1cfdbc..582b7447b 100644
--- a/seatunnel-ui/src/views/login/index.tsx
+++ b/seatunnel-ui/src/layouts/dashboard/header/user/index.tsx
@@ -15,19 +15,38 @@
  * limitations under the License.
  */
 
-import { defineComponent } from 'vue'
+import { defineComponent, toRefs } from 'vue'
+import { NSpace, NDropdown } from 'naive-ui'
+import { useUserDropdown } from './use-user-dropdown'
 
-const Login = defineComponent({
+const User = defineComponent({
   setup() {
+    const { state, handleSelect } = useUserDropdown()
 
+    return { ...toRefs(state), handleSelect }
   },
   render() {
     return (
-      <div>
-        <h1>Login</h1>
-      </div>
+      <NSpace
+        justify='center'
+        align='center'
+        class='h-16 w-12 mr-2'
+        style={{ cursor: 'pointer' }}
+      >
+        <NDropdown
+          trigger='click'
+          options={this.dropdownOptions}
+          onSelect={this.handleSelect}
+        >
+          <img
+            class='h-10 w-10 rounded-full'
+            src='https://avatars.githubusercontent.com/u/19239641?s=64&v=4'
+            alt=''
+          />
+        </NDropdown>
+      </NSpace>
     )
   }
 })
 
-export default Login
\ No newline at end of file
+export default User
diff --git a/seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-jdbc/src/main/scala/org/apache/seatunnel/spark/jdbc/source/util/HiveDialet.scala b/seatunnel-ui/src/layouts/dashboard/header/user/use-user-dropdown.ts
similarity index 55%
rename from seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-jdbc/src/main/scala/org/apache/seatunnel/spark/jdbc/source/util/HiveDialet.scala
rename to seatunnel-ui/src/layouts/dashboard/header/user/use-user-dropdown.ts
index c686cdfba..0544c647d 100644
--- a/seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-jdbc/src/main/scala/org/apache/seatunnel/spark/jdbc/source/util/HiveDialet.scala
+++ b/seatunnel-ui/src/layouts/dashboard/header/user/use-user-dropdown.ts
@@ -14,21 +14,32 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.seatunnel.spark.jdbc.source.util
 
-import org.apache.spark.sql.jdbc.JdbcDialect
+import { reactive } from 'vue'
+import { useI18n } from 'vue-i18n'
+import { useRouter } from 'vue-router'
+import type { Router } from 'vue-router'
 
-class HiveDialet extends JdbcDialect {
-  override def canHandle(url: String): Boolean = {
-    url.startsWith("jdbc:hive2")
-  }
+export function useUserDropdown() {
+  const router: Router = useRouter()
+  const { t } = useI18n()
+
+  const dropdownOptions = [
+    { key: 'help', label: t('menu.help') },
+    { key: 'logout', label: t('menu.logout') }
+  ]
 
-  override def quoteIdentifier(colName: String): String = {
-    if (colName.contains(".")) {
-      val colName1 = colName.substring(colName.indexOf(".") + 1)
-      s"`$colName1`"
-    } else {
-      s"`$colName`"
+  const state = reactive({
+    dropdownOptions
+  })
+
+  const handleSelect = (key: string) => {
+    if (key === 'help') {
+      window.open('http://seatunnel.incubator.apache.org/versions/')
+    } else if (key === 'logout') {
+      router.push({ path: '/login' })
     }
   }
+
+  return { state, handleSelect }
 }
diff --git a/seatunnel-ui/src/views/login/index.tsx b/seatunnel-ui/src/layouts/dashboard/index.tsx
similarity index 78%
copy from seatunnel-ui/src/views/login/index.tsx
copy to seatunnel-ui/src/layouts/dashboard/index.tsx
index 92e1cfdbc..c4058373e 100644
--- a/seatunnel-ui/src/views/login/index.tsx
+++ b/seatunnel-ui/src/layouts/dashboard/index.tsx
@@ -16,18 +16,19 @@
  */
 
 import { defineComponent } from 'vue'
+import { NSpace } from 'naive-ui'
+import Header from './header'
 
-const Login = defineComponent({
-  setup() {
-
-  },
+const Dashboard = defineComponent({
+  setup() {},
   render() {
     return (
-      <div>
-        <h1>Login</h1>
-      </div>
+      <NSpace vertical>
+        <Header />
+        <router-view class='px-32 py-12' />
+      </NSpace>
     )
   }
 })
 
-export default Login
\ No newline at end of file
+export default Dashboard
diff --git a/seatunnel-ui/src/locales/en_US/index.ts b/seatunnel-ui/src/locales/en_US/index.ts
index 1e7d2e414..eebfb615d 100644
--- a/seatunnel-ui/src/locales/en_US/index.ts
+++ b/seatunnel-ui/src/locales/en_US/index.ts
@@ -15,10 +15,12 @@
  * limitations under the License.
  */
 
-const test = {
-  'test': 'test'
-}
+import login from '@/locales/en_US/login'
+import menu from '@/locales/en_US/menu'
+import user_manage from '@/locales/en_US/user-manage'
 
 export default {
-  test
-}
\ No newline at end of file
+  login,
+  menu,
+  user_manage
+}
diff --git a/seatunnel-ui/src/locales/zh_CN/index.ts b/seatunnel-ui/src/locales/en_US/login.ts
similarity index 80%
copy from seatunnel-ui/src/locales/zh_CN/index.ts
copy to seatunnel-ui/src/locales/en_US/login.ts
index e979946bb..1261d1028 100644
--- a/seatunnel-ui/src/locales/zh_CN/index.ts
+++ b/seatunnel-ui/src/locales/en_US/login.ts
@@ -15,10 +15,11 @@
  * limitations under the License.
  */
 
-const test = {
-  'test': '测试'
-}
-
 export default {
-  test
-}
\ No newline at end of file
+  login_to_sea_tunnel: 'Login To SeaTunnel',
+  login: 'Login',
+  username: 'Username',
+  password: 'Password',
+  username_tips: 'Please input username',
+  password_tips: 'Please input password'
+}
diff --git a/seatunnel-ui/src/locales/zh_CN/index.ts b/seatunnel-ui/src/locales/en_US/menu.ts
similarity index 87%
copy from seatunnel-ui/src/locales/zh_CN/index.ts
copy to seatunnel-ui/src/locales/en_US/menu.ts
index e979946bb..880c5203f 100644
--- a/seatunnel-ui/src/locales/zh_CN/index.ts
+++ b/seatunnel-ui/src/locales/en_US/menu.ts
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-const test = {
-  'test': '测试'
-}
-
 export default {
-  test
-}
\ No newline at end of file
+  data_pipes: 'Data Pipes',
+  manage: 'Manage',
+  user_manage: 'User Manage',
+  help: 'Help',
+  logout: 'Logout'
+}
diff --git a/seatunnel-ui/src/locales/zh_CN/index.ts b/seatunnel-ui/src/locales/en_US/user-manage.ts
similarity index 73%
copy from seatunnel-ui/src/locales/zh_CN/index.ts
copy to seatunnel-ui/src/locales/en_US/user-manage.ts
index e979946bb..7ad4ca183 100644
--- a/seatunnel-ui/src/locales/zh_CN/index.ts
+++ b/seatunnel-ui/src/locales/en_US/user-manage.ts
@@ -15,10 +15,17 @@
  * limitations under the License.
  */
 
-const test = {
-  'test': '测试'
-}
-
 export default {
-  test
-}
\ No newline at end of file
+  user_manage: 'User Manage',
+  create: 'Create',
+  username: 'Username',
+  state: 'State',
+  email: 'Email',
+  creation_time: 'Creation Time',
+  last_landing_time: 'Last Landing Time',
+  operation: 'Operation',
+  enable: 'Enable',
+  disable: 'Disable',
+  edite: 'Edite',
+  delete: 'Delete'
+}
diff --git a/seatunnel-ui/src/locales/index.ts b/seatunnel-ui/src/locales/index.ts
index 1d2769c17..0f26f85bf 100644
--- a/seatunnel-ui/src/locales/index.ts
+++ b/seatunnel-ui/src/locales/index.ts
@@ -29,4 +29,4 @@ const i18n = createI18n({
   }
 })
 
-export default i18n
\ No newline at end of file
+export default i18n
diff --git a/seatunnel-ui/src/locales/zh_CN/index.ts b/seatunnel-ui/src/locales/zh_CN/index.ts
index e979946bb..39626a390 100644
--- a/seatunnel-ui/src/locales/zh_CN/index.ts
+++ b/seatunnel-ui/src/locales/zh_CN/index.ts
@@ -15,10 +15,4 @@
  * limitations under the License.
  */
 
-const test = {
-  'test': '测试'
-}
-
-export default {
-  test
-}
\ No newline at end of file
+export default {}
diff --git a/seatunnel-ui/src/main.ts b/seatunnel-ui/src/main.ts
index 25683a5a2..4962e59f6 100644
--- a/seatunnel-ui/src/main.ts
+++ b/seatunnel-ui/src/main.ts
@@ -21,13 +21,16 @@ import App from './App'
 import piniaPluginPersistedstate from 'pinia-plugin-persistedstate'
 import i18n from '@/locales'
 import router from './router'
+import utils from '@/utils'
+import './index.css'
 
 const app = createApp(App)
 const pinia = createPinia()
 
 pinia.use(piniaPluginPersistedstate)
+app.config.globalProperties.trim = utils.trim
 
 app.use(router)
 app.use(pinia)
 app.use(i18n)
-app.mount('#app')
\ No newline at end of file
+app.mount('#app')
diff --git a/seatunnel-ui/src/router/routes.ts b/seatunnel-ui/src/router/data-pipes.ts
similarity index 72%
copy from seatunnel-ui/src/router/routes.ts
copy to seatunnel-ui/src/router/data-pipes.ts
index 1a62f7db8..6357a3132 100644
--- a/seatunnel-ui/src/router/routes.ts
+++ b/seatunnel-ui/src/router/data-pipes.ts
@@ -16,27 +16,27 @@
  */
 
 import utils from '@/utils'
-import type { RouteRecordRaw } from 'vue-router'
 import type { Component } from 'vue'
 
 const modules = import.meta.glob('/src/views/**/**.tsx')
 const components: { [key: string]: Component } = utils.mapping(modules)
 
-const basePage: RouteRecordRaw[] = [
-
-]
-
-const loginPage: RouteRecordRaw[] = [
-  {
-    path: '/login',
-    name: 'login',
-    component: components['login'],
-    meta: {
-      auth: []
+export default {
+  path: '/data-pipes',
+  name: 'data-pipes',
+  meta: {
+    title: 'data-pipes'
+  },
+  redirect: { name: 'data-pipes-list' },
+  component: () => import('@/layouts/dashboard'),
+  children: [
+    {
+      path: '/data-pipes/list',
+      name: 'data-pipes-list',
+      component: components['data-pipes-list'],
+      meta: {
+        title: 'data-pipes-list'
+      }
     }
-  }
-]
-
-const routes: RouteRecordRaw[] = [...basePage, ...loginPage]
-
-export default routes
\ No newline at end of file
+  ]
+}
diff --git a/seatunnel-ui/src/router/index.ts b/seatunnel-ui/src/router/index.ts
index a46d29b3b..188a9f449 100644
--- a/seatunnel-ui/src/router/index.ts
+++ b/seatunnel-ui/src/router/index.ts
@@ -15,18 +15,13 @@
  * limitations under the License.
  */
 
-import {
-  createRouter,
-  createWebHistory
-} from 'vue-router'
+import { createRouter, createWebHistory } from 'vue-router'
 import routes from './routes'
 import NProgress from 'nprogress'
 import 'nprogress/nprogress.css'
 
 const router = createRouter({
-  history: createWebHistory(
-   '/'
-  ),
+  history: createWebHistory('/'),
   routes
 })
 
@@ -34,4 +29,4 @@ router.afterEach(() => {
   NProgress.done()
 })
 
-export default router
\ No newline at end of file
+export default router
diff --git a/seatunnel-ui/src/router/routes.ts b/seatunnel-ui/src/router/routes.ts
index 1a62f7db8..253992f48 100644
--- a/seatunnel-ui/src/router/routes.ts
+++ b/seatunnel-ui/src/router/routes.ts
@@ -16,15 +16,15 @@
  */
 
 import utils from '@/utils'
+import dataPipes from '@/router/data-pipes'
+import userManage from '@/router/user-manage'
 import type { RouteRecordRaw } from 'vue-router'
 import type { Component } from 'vue'
 
 const modules = import.meta.glob('/src/views/**/**.tsx')
 const components: { [key: string]: Component } = utils.mapping(modules)
 
-const basePage: RouteRecordRaw[] = [
-
-]
+const basePage: RouteRecordRaw[] = [dataPipes, userManage]
 
 const loginPage: RouteRecordRaw[] = [
   {
@@ -39,4 +39,4 @@ const loginPage: RouteRecordRaw[] = [
 
 const routes: RouteRecordRaw[] = [...basePage, ...loginPage]
 
-export default routes
\ No newline at end of file
+export default routes
diff --git a/seatunnel-ui/src/router/routes.ts b/seatunnel-ui/src/router/user-manage.ts
similarity index 71%
copy from seatunnel-ui/src/router/routes.ts
copy to seatunnel-ui/src/router/user-manage.ts
index 1a62f7db8..be0dcdc54 100644
--- a/seatunnel-ui/src/router/routes.ts
+++ b/seatunnel-ui/src/router/user-manage.ts
@@ -16,27 +16,27 @@
  */
 
 import utils from '@/utils'
-import type { RouteRecordRaw } from 'vue-router'
 import type { Component } from 'vue'
 
 const modules = import.meta.glob('/src/views/**/**.tsx')
 const components: { [key: string]: Component } = utils.mapping(modules)
 
-const basePage: RouteRecordRaw[] = [
-
-]
-
-const loginPage: RouteRecordRaw[] = [
-  {
-    path: '/login',
-    name: 'login',
-    component: components['login'],
-    meta: {
-      auth: []
+export default {
+  path: '/user-manage',
+  name: 'user-manage',
+  meta: {
+    title: 'user-manage'
+  },
+  redirect: { name: 'user-manage-list' },
+  component: () => import('@/layouts/dashboard'),
+  children: [
+    {
+      path: '/user-manage/list',
+      name: 'user-manage-list',
+      component: components['user-manage-list'],
+      meta: {
+        title: 'user-manage-list'
+      }
     }
-  }
-]
-
-const routes: RouteRecordRaw[] = [...basePage, ...loginPage]
-
-export default routes
\ No newline at end of file
+  ]
+}
diff --git a/seatunnel-ui/src/store/locale/index.ts b/seatunnel-ui/src/store/locale/index.ts
index b5139aa1c..1b31f7a9b 100644
--- a/seatunnel-ui/src/store/locale/index.ts
+++ b/seatunnel-ui/src/store/locale/index.ts
@@ -34,4 +34,4 @@ export const useLocalesStore = defineStore({
       this.locales = lang
     }
   }
-})
\ No newline at end of file
+})
diff --git a/seatunnel-ui/src/store/locale/types.ts b/seatunnel-ui/src/store/locale/types.ts
index 095d54269..03625ade5 100644
--- a/seatunnel-ui/src/store/locale/types.ts
+++ b/seatunnel-ui/src/store/locale/types.ts
@@ -21,4 +21,4 @@ interface LocalesStore {
   locales: Locales
 }
 
-export { LocalesStore, Locales }
\ No newline at end of file
+export { LocalesStore, Locales }
diff --git a/seatunnel-ui/src/store/theme/index.ts b/seatunnel-ui/src/store/theme/index.ts
index 016751947..0fe8052dd 100644
--- a/seatunnel-ui/src/store/theme/index.ts
+++ b/seatunnel-ui/src/store/theme/index.ts
@@ -34,4 +34,4 @@ export const useThemeStore = defineStore({
       this.darkTheme = !this.darkTheme
     }
   }
-})
\ No newline at end of file
+})
diff --git a/seatunnel-ui/src/store/theme/types.ts b/seatunnel-ui/src/store/theme/types.ts
index be5b77260..c2ca699b4 100644
--- a/seatunnel-ui/src/store/theme/types.ts
+++ b/seatunnel-ui/src/store/theme/types.ts
@@ -1,4 +1,3 @@
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -20,4 +19,4 @@ interface ThemeState {
   darkTheme: boolean
 }
 
-export default ThemeState
\ No newline at end of file
+export default ThemeState
diff --git a/seatunnel-ui/src/themes/index.ts b/seatunnel-ui/src/themes/index.ts
index 5489b3843..a5f70cbf1 100644
--- a/seatunnel-ui/src/themes/index.ts
+++ b/seatunnel-ui/src/themes/index.ts
@@ -1,4 +1,3 @@
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -24,4 +23,4 @@ const themeList = {
   dark
 }
 
-export default themeList
\ No newline at end of file
+export default themeList
diff --git a/seatunnel-ui/src/themes/modules/dark.ts b/seatunnel-ui/src/themes/modules/dark.ts
index 551aeb82f..5473e6f23 100644
--- a/seatunnel-ui/src/themes/modules/dark.ts
+++ b/seatunnel-ui/src/themes/modules/dark.ts
@@ -17,4 +17,4 @@
 
 const dark = {}
 
-export default dark
\ No newline at end of file
+export default dark
diff --git a/seatunnel-ui/src/themes/modules/light.ts b/seatunnel-ui/src/themes/modules/light.ts
index 97243a6f4..d691a04ec 100644
--- a/seatunnel-ui/src/themes/modules/light.ts
+++ b/seatunnel-ui/src/themes/modules/light.ts
@@ -17,4 +17,4 @@
 
 const light = {}
 
-export default light
\ No newline at end of file
+export default light
diff --git a/seatunnel-ui/src/utils/index.ts b/seatunnel-ui/src/utils/index.ts
index f76fbc629..3a7285d70 100644
--- a/seatunnel-ui/src/utils/index.ts
+++ b/seatunnel-ui/src/utils/index.ts
@@ -16,9 +16,11 @@
  */
 
 import mapping from './mapping'
+import trim from './trim'
 
 const utils = {
-  mapping
+  mapping,
+  trim
 }
 
-export default utils
\ No newline at end of file
+export default utils
diff --git a/seatunnel-ui/src/utils/mapping.ts b/seatunnel-ui/src/utils/mapping.ts
index 27d754ff7..a200f94c9 100644
--- a/seatunnel-ui/src/utils/mapping.ts
+++ b/seatunnel-ui/src/utils/mapping.ts
@@ -22,7 +22,9 @@ const mapping = (modules: any) => {
   Object.keys(modules).forEach((key: string) => {
     const nameMatch: string[] | null = key.match(/^\/src\/views\/(.+)\.tsx/)
 
-    if (!nameMatch) { return }
+    if (!nameMatch) {
+      return
+    }
 
     const indexMatch: string[] | null = nameMatch[1].match(/(.*)\/Index$/i)
 
@@ -35,4 +37,4 @@ const mapping = (modules: any) => {
   return components
 }
 
-export default mapping
\ No newline at end of file
+export default mapping
diff --git a/seatunnel-ui/src/store/theme/types.ts b/seatunnel-ui/src/utils/trim.ts
similarity index 87%
copy from seatunnel-ui/src/store/theme/types.ts
copy to seatunnel-ui/src/utils/trim.ts
index be5b77260..d7f731bcd 100644
--- a/seatunnel-ui/src/store/theme/types.ts
+++ b/seatunnel-ui/src/utils/trim.ts
@@ -1,4 +1,3 @@
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -16,8 +15,8 @@
  * limitations under the License.
  */
 
-interface ThemeState {
-  darkTheme: boolean
+const trim = (value: string) => {
+  return !value.startsWith(' ') && !value.endsWith(' ')
 }
 
-export default ThemeState
\ No newline at end of file
+export default trim
diff --git a/seatunnel-ui/src/views/login/index.tsx b/seatunnel-ui/src/views/data-pipes/list/index.tsx
similarity index 86%
copy from seatunnel-ui/src/views/login/index.tsx
copy to seatunnel-ui/src/views/data-pipes/list/index.tsx
index 92e1cfdbc..554892be4 100644
--- a/seatunnel-ui/src/views/login/index.tsx
+++ b/seatunnel-ui/src/views/data-pipes/list/index.tsx
@@ -17,17 +17,11 @@
 
 import { defineComponent } from 'vue'
 
-const Login = defineComponent({
-  setup() {
-
-  },
+const DataPipesList = defineComponent({
+  setup() {},
   render() {
-    return (
-      <div>
-        <h1>Login</h1>
-      </div>
-    )
+    return <div>datapipes</div>
   }
 })
 
-export default Login
\ No newline at end of file
+export default DataPipesList
diff --git a/seatunnel-ui/src/views/login/index.tsx b/seatunnel-ui/src/views/login/index.tsx
index 92e1cfdbc..9be980c0b 100644
--- a/seatunnel-ui/src/views/login/index.tsx
+++ b/seatunnel-ui/src/views/login/index.tsx
@@ -15,19 +15,72 @@
  * limitations under the License.
  */
 
-import { defineComponent } from 'vue'
+import { defineComponent, toRefs, withKeys, getCurrentInstance } from 'vue'
+import { NSpace, NForm, NFormItem, NInput, NButton } from 'naive-ui'
+import { useI18n } from 'vue-i18n'
+import { useForm } from './use-form'
 
 const Login = defineComponent({
   setup() {
+    const { t } = useI18n()
+    const { state } = useForm()
+    const trim = getCurrentInstance()?.appContext.config.globalProperties.trim
 
+    const handleLogin = () => {}
+
+    return {
+      t,
+      ...toRefs(state),
+      trim,
+      handleLogin
+    }
   },
   render() {
     return (
-      <div>
-        <h1>Login</h1>
-      </div>
+      <NSpace justify='center' align='center' class='w-full h-screen'>
+        <div>
+          <h2>{this.t('login.login_to_sea_tunnel')}</h2>
+          <NForm rules={this.rules} ref='loginFormRef'>
+            <NFormItem
+              label={this.t('login.username')}
+              label-style={{ color: 'black' }}
+              path='userName'
+            >
+              <NInput
+                allowInput={this.trim}
+                type='text'
+                v-model={[this.loginForm.username, 'value']}
+                placeholder={this.t('login.username_tips')}
+                autofocus
+                onKeydown={withKeys(this.handleLogin, ['enter'])}
+              />
+            </NFormItem>
+            <NFormItem
+              label={this.t('login.password')}
+              label-style={{ color: 'black' }}
+              path='userPassword'
+            >
+              <NInput
+                allowInput={this.trim}
+                type='password'
+                v-model={[this.loginForm.password, 'value']}
+                placeholder={this.t('login.password_tips')}
+                onKeydown={withKeys(this.handleLogin, ['enter'])}
+              />
+            </NFormItem>
+          </NForm>
+          <NButton
+            type='info'
+            disabled={!this.loginForm.username || !this.loginForm.password}
+            style={{ width: '100%' }}
+            onClick={this.handleLogin}
+          >
+            {this.t('login.login')}
+          </NButton>
+        </div>
+      </NSpace>
     )
   }
 })
 
-export default Login
\ No newline at end of file
+export default Login
diff --git a/seatunnel-ui/src/utils/mapping.ts b/seatunnel-ui/src/views/login/use-form.ts
similarity index 51%
copy from seatunnel-ui/src/utils/mapping.ts
copy to seatunnel-ui/src/views/login/use-form.ts
index 27d754ff7..9a7094146 100644
--- a/seatunnel-ui/src/utils/mapping.ts
+++ b/seatunnel-ui/src/views/login/use-form.ts
@@ -15,24 +15,40 @@
  * limitations under the License.
  */
 
-import type { Component } from 'vue'
-
-const mapping = (modules: any) => {
-  const components: { [key: string]: Component } = {}
-  Object.keys(modules).forEach((key: string) => {
-    const nameMatch: string[] | null = key.match(/^\/src\/views\/(.+)\.tsx/)
-
-    if (!nameMatch) { return }
-
-    const indexMatch: string[] | null = nameMatch[1].match(/(.*)\/Index$/i)
-
-    let name: string = indexMatch ? indexMatch[1] : nameMatch[1]
-
-    name = name.replaceAll('/', '-')
-
-    components[name] = modules[key]
+import { reactive, ref } from 'vue'
+import { useI18n } from 'vue-i18n'
+import type { FormRules } from 'naive-ui'
+
+export function useForm() {
+  const { t } = useI18n()
+
+  const state = reactive({
+    loginFormRef: ref(),
+    loginForm: {
+      username: '',
+      password: ''
+    },
+    rules: {
+      userName: {
+        trigger: ['input', 'blur'],
+        validator() {
+          if (state.loginForm.username === '') {
+            return new Error(t('login.username_tips'))
+          }
+        }
+      },
+      userPassword: {
+        trigger: ['input', 'blur'],
+        validator() {
+          if (state.loginForm.password === '') {
+            return new Error(t('login.password_tips'))
+          }
+        }
+      }
+    } as FormRules
   })
-  return components
-}
 
-export default mapping
\ No newline at end of file
+  return {
+    state
+  }
+}
diff --git a/seatunnel-ui/src/views/user-manage/list/index.tsx b/seatunnel-ui/src/views/user-manage/list/index.tsx
new file mode 100644
index 000000000..d1f5a09e3
--- /dev/null
+++ b/seatunnel-ui/src/views/user-manage/list/index.tsx
@@ -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.
+ */
+
+import { defineComponent, toRefs, onMounted } from 'vue'
+import { NSpace, NCard, NButton, NDataTable, NPagination } from 'naive-ui'
+import { useI18n } from 'vue-i18n'
+import { useTable } from './use-table'
+
+const UserManageList = defineComponent({
+  setup() {
+    const { t } = useI18n()
+    const { state, createColumns } = useTable()
+
+    onMounted(() => {
+      createColumns(state)
+    })
+
+    return { t, ...toRefs(state) }
+  },
+  render() {
+    return (
+      <NSpace vertical>
+        <NCard title={this.t('user_manage.user_manage')}>
+          {{
+            'header-extra': () => (
+              <NButton>{this.t('user_manage.create')}</NButton>
+            )
+          }}
+        </NCard>
+        <NCard>
+          <NSpace vertical>
+            <NDataTable
+              loading={this.loadingRef}
+              columns={this.columns}
+              data={this.tableData}
+            />
+            <NSpace justify='center'>
+              <NPagination
+                v-model:page={this.page}
+                v-model:page-size={this.pageSize}
+                page-count={this.totalPage}
+                show-size-picker
+                page-sizes={[10, 30, 50]}
+                show-quick-jumper
+              />
+            </NSpace>
+          </NSpace>
+        </NCard>
+      </NSpace>
+    )
+  }
+})
+
+export default UserManageList
diff --git a/seatunnel-ui/src/views/user-manage/list/use-table.ts b/seatunnel-ui/src/views/user-manage/list/use-table.ts
new file mode 100644
index 000000000..736fe1204
--- /dev/null
+++ b/seatunnel-ui/src/views/user-manage/list/use-table.ts
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import { reactive, ref, h } from 'vue'
+import { useI18n } from 'vue-i18n'
+import { NSpace, NButton } from 'naive-ui'
+
+export function useTable() {
+  const { t } = useI18n()
+  const state = reactive({
+    columns: [],
+    tableData: [{ username: '' }],
+    page: ref(1),
+    pageSize: ref(10),
+    totalPage: ref(1),
+    row: {},
+    loadingRef: ref(false)
+  })
+
+  const createColumns = (state: any) => {
+    state.columns = [
+      {
+        title: t('user_manage.username'),
+        key: 'username'
+      },
+      {
+        title: t('user_manage.state'),
+        key: 'state'
+      },
+      {
+        title: t('user_manage.email'),
+        key: 'email'
+      },
+      {
+        title: t('user_manage.creation_time'),
+        key: 'creationTime'
+      },
+      {
+        title: t('user_manage.last_landing_time'),
+        key: 'lastLandingTime'
+      },
+      {
+        title: t('user_manage.operation'),
+        key: 'operation',
+        render: () =>
+          h(NSpace, null, {
+            default: () => [
+              h(NButton, { text: true }, t('user_manage.enable')),
+              h(NButton, { text: true }, t('user_manage.edite')),
+              h(NButton, { text: true }, t('user_manage.delete'))
+            ]
+          })
+      }
+    ]
+  }
+
+  return { state, createColumns }
+}
diff --git a/seatunnel-ui/src/store/theme/types.ts b/seatunnel-ui/tailwind.config.js
similarity index 81%
copy from seatunnel-ui/src/store/theme/types.ts
copy to seatunnel-ui/tailwind.config.js
index be5b77260..90a1a03f0 100644
--- a/seatunnel-ui/src/store/theme/types.ts
+++ b/seatunnel-ui/tailwind.config.js
@@ -1,4 +1,3 @@
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -16,8 +15,14 @@
  * limitations under the License.
  */
 
-interface ThemeState {
-  darkTheme: boolean
-}
-
-export default ThemeState
\ No newline at end of file
+module.exports = {
+  content: ['./index.html', './src/**/*.{vue,js,ts,jsx,tsx}'],
+  darkMode: 'media',
+  theme: {
+    extend: {}
+  },
+  variants: {
+    extend: {}
+  },
+  plugins: []
+}
\ No newline at end of file
diff --git a/tools/checkstyle/checkStyle.xml b/tools/checkstyle/checkStyle.xml
index ebbc1f258..ddff7c20f 100755
--- a/tools/checkstyle/checkStyle.xml
+++ b/tools/checkstyle/checkStyle.xml
@@ -85,6 +85,11 @@
         <property name="ignorePattern" value="^package.*|^import.*|a href|href|http://|https://|ftp://"/>
     </module>
 
+    <module name="SuppressionSingleFilter">
+        <property name="files" value=".*(IT|Test).java"/>
+        <property name="checks" value="MagicNumber"/>
+    </module>
+
     <module name="SuppressWarningsFilter"/>
     <module name="TreeWalker">
         <module name="SuppressWarningsHolder"/>
@@ -207,7 +212,9 @@
 
         <module name="OneStatementPerLine"/>
 
-        <module name="MagicNumber"/>
+        <module name="MagicNumber">
+            <property name="ignoreHashCodeMethod" value="true"/>
+        </module>
 
         <module name="MultipleVariableDeclarations"/>
 
diff --git a/tools/dependencies/checkLicense.sh b/tools/dependencies/checkLicense.sh
index 026cdc950..024c95cd0 100755
--- a/tools/dependencies/checkLicense.sh
+++ b/tools/dependencies/checkLicense.sh
@@ -19,7 +19,11 @@
 
 set -e
 
-./mvnw --batch-mode --no-snapshot-updates dependency:copy-dependencies -DincludeScope=runtime -DoutputDirectory=/tmp/seatunnel-dependencies -P release
+if [ -d "/tmp/seatunnel-dependencies" ]; then
+  rm -rf /tmp/seatunnel-dependencies/*
+fi
+
+./mvnw clean -pl '!seatunnel-connectors-v2-dist,!seatunnel-dist,!seatunnel-connectors-v2/connector-fake' --batch-mode  --no-snapshot-updates dependency:copy-dependencies -DincludeScope=runtime -DoutputDirectory=/tmp/seatunnel-dependencies
 
 # List all modules(jars) that belong to the SeaTunnel itself, these will be ignored when checking the dependency
 ls /tmp/seatunnel-dependencies | sort > all-dependencies.txt
diff --git a/tools/dependencies/known-dependencies.txt b/tools/dependencies/known-dependencies.txt
index dd5dd4c69..2fbb325e3 100755
--- a/tools/dependencies/known-dependencies.txt
+++ b/tools/dependencies/known-dependencies.txt
@@ -319,6 +319,7 @@ jakarta.validation-api-2.0.2.jar
 jakarta.websocket-api-1.1.2.jar
 jamon-runtime-2.4.1.jar
 janino-3.1.6.jar
+janino-3.0.9.jar
 java-xmlbuilder-0.4.jar
 javassist-3.18.1-GA.jar
 javassist-3.20.0-GA.jar