diff --git a/.github/workflows/backend.yml b/.github/workflows/backend.yml
index db406b3937e..ee6620ec71c 100644
--- a/.github/workflows/backend.yml
+++ b/.github/workflows/backend.yml
@@ -89,7 +89,7 @@ jobs:
java: [ '8', '11' ]
os: [ 'ubuntu-latest', 'windows-latest' ]
runs-on: ${{ matrix.os }}
- timeout-minutes: 50
+ timeout-minutes: 80
steps:
- uses: actions/checkout@v3
with:
@@ -115,7 +115,7 @@ jobs:
name: Dependency licenses
needs: [ sanity-check ]
runs-on: ubuntu-latest
- timeout-minutes: 30
+ timeout-minutes: 40
steps:
- uses: actions/checkout@v3
with:
@@ -155,7 +155,9 @@ jobs:
cache: 'maven'
- name: Run Unit tests
run: |
- ./mvnw -T 2C -B clean verify -D"maven.test.skip"=false -D"checkstyle.skip"=true -D"scalastyle.skip"=true -D"license.skipAddThirdParty"=true --no-snapshot-updates
+ ./mvnw -B -T 1C clean verify -D"maven.test.skip"=false -D"checkstyle.skip"=true -D"scalastyle.skip"=true -D"license.skipAddThirdParty"=true --no-snapshot-updates
+ env:
+ MAVEN_OPTS: -Xmx2048m
integration-test:
name: Integration Test
@@ -176,4 +178,6 @@ jobs:
cache: 'maven'
- name: Run Integration tests
run: |
- ./mvnw -T 2C -B verify -DskipUT=true -DskipIT=false -D"checkstyle.skip"=true -D"scalastyle.skip"=true -D"license.skipAddThirdParty"=true --no-snapshot-updates
+ ./mvnw -T 1C -B verify -DskipUT=true -DskipIT=false -D"checkstyle.skip"=true -D"scalastyle.skip"=true -D"license.skipAddThirdParty"=true --no-snapshot-updates
+ env:
+ MAVEN_OPTS: -Xmx2048m
diff --git a/.github/workflows/code-analysys.yml b/.github/workflows/code-analysys.yml
index 3032c3428e7..4fda66e5f9f 100644
--- a/.github/workflows/code-analysys.yml
+++ b/.github/workflows/code-analysys.yml
@@ -25,6 +25,7 @@ on:
jobs:
build:
runs-on: ubuntu-latest
+ timeout-minutes: 120
steps:
- uses: actions/checkout@v2
with:
diff --git a/.github/workflows/docker.yml b/.github/workflows/docker.yml
index a263acd68f3..1cf4140aa4f 100644
--- a/.github/workflows/docker.yml
+++ b/.github/workflows/docker.yml
@@ -35,7 +35,7 @@ jobs:
check:
name: Spark
runs-on: ubuntu-latest
- timeout-minutes: 30
+ timeout-minutes: 60
steps:
- uses: actions/checkout@v2
- name: Set up JDK 1.8
diff --git a/.gitignore b/.gitignore
index dbd0c25fb06..2318e11827c 100644
--- a/.gitignore
+++ b/.gitignore
@@ -13,6 +13,7 @@ target/
# Intellij Idea files
.idea/
*.iml
+.idea/*
.DS_Store
@@ -40,4 +41,4 @@ Test.scala
test.conf
log4j.properties
spark-warehouse
-*.flattened-pom.xml
\ No newline at end of file
+*.flattened-pom.xml
diff --git a/plugin-mapping.properties b/plugin-mapping.properties
index 2a43975d0b1..bfd9bb70ccb 100644
--- a/plugin-mapping.properties
+++ b/plugin-mapping.properties
@@ -102,4 +102,6 @@ seatunnel.sink.Clickhouse = connector-clickhouse
seatunnel.sink.ClickhouseFile = connector-clickhouse
seatunnel.source.Jdbc = connector-jdbc
seatunnel.sink.Jdbc = connector-jdbc
+seatunnel.sink.HdfsFile = connector-file-hadoop
+seatunnel.sink.LocalFile = connector-file-local
seatunnel.source.Pulsar = connector-pulsar
diff --git a/pom.xml b/pom.xml
index 2378419b73b..5ac90416a68 100644
--- a/pom.xml
+++ b/pom.xml
@@ -194,6 +194,7 @@
1.7.25
19.0
1.0.1
+ 2.0.9
2.6.5
3.0.0
org.apache.seatunnel.shade
@@ -659,12 +660,23 @@
${guava.version}
+
+ org.powermock
+ powermock-module-junit4
+ ${powermock.version}
+ test
+
+
+ org.powermock
+ powermock-api-mockito2
+ ${powermock.version}
+ test
+
com.github.jsonzou
jmockdata
${jmockdata.version}
-
org.slf4j
slf4j-api
@@ -1277,7 +1289,6 @@
org.apache.maven.plugins
maven-surefire-plugin
-
org.apache.maven.plugins
maven-failsafe-plugin
diff --git a/seatunnel-connectors-v2-dist/pom.xml b/seatunnel-connectors-v2-dist/pom.xml
index 9e349fbb99d..4c5d65a82ef 100644
--- a/seatunnel-connectors-v2-dist/pom.xml
+++ b/seatunnel-connectors-v2-dist/pom.xml
@@ -17,6 +17,7 @@
limitations under the License.
-->
+
@@ -80,9 +81,18 @@
connector-hive
${project.version}
+
+ org.apache.seatunnel
+ connector-file-hadoop
+ ${project.version}
+
+
+ org.apache.seatunnel
+ connector-file-local
+ ${project.version}
+
-
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/pom.xml b/seatunnel-connectors-v2/connector-file/connector-file-base/pom.xml
new file mode 100644
index 00000000000..2f3c38c519b
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/pom.xml
@@ -0,0 +1,72 @@
+
+
+
+
+ connector-file
+ org.apache.seatunnel
+ ${revision}
+
+ 4.0.0
+
+ connector-file-base
+
+
+
+ org.apache.seatunnel
+ seatunnel-api
+ ${project.version}
+
+
+
+ org.apache.seatunnel
+ seatunnel-core-base
+ ${project.version}
+ test
+
+
+
+ org.apache.commons
+ commons-collections4
+
+
+ org.apache.commons
+ commons-lang3
+
+
+
+ junit
+ junit
+ test
+
+
+
+ org.powermock
+ powermock-module-junit4
+ test
+
+
+ org.powermock
+ powermock-api-mockito2
+ test
+
+
+
\ No newline at end of file
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/AbstractTextFileConfig.java
new file mode 100644
index 00000000000..32672066d4e
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/AbstractTextFileConfig.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.config;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+
+import lombok.Data;
+import lombok.NonNull;
+import org.apache.commons.lang3.StringUtils;
+
+import java.io.Serializable;
+import java.util.Locale;
+
+@Data
+public class AbstractTextFileConfig implements DelimiterConfig, CompressConfig, Serializable {
+ private static final long serialVersionUID = 1L;
+
+ protected String compressCodec;
+
+ protected String fieldDelimiter = String.valueOf('\001');
+
+ protected String rowDelimiter = "\n";
+
+ protected String path;
+ protected String fileNameExpression;
+ protected FileFormat fileFormat = FileFormat.TEXT;
+
+ public AbstractTextFileConfig(@NonNull Config config) {
+ checkNotNull(config.getString(Constant.PATH));
+
+ if (config.hasPath(Constant.COMPRESS_CODEC)) {
+ throw new RuntimeException("compress not support now");
+ }
+
+ if (config.hasPath(Constant.FIELD_DELIMITER) && !StringUtils.isBlank(config.getString(Constant.FIELD_DELIMITER))) {
+ this.fieldDelimiter = config.getString(Constant.FIELD_DELIMITER);
+ }
+
+ if (config.hasPath(Constant.ROW_DELIMITER) && !StringUtils.isBlank(config.getString(Constant.ROW_DELIMITER))) {
+ this.rowDelimiter = config.getString(Constant.ROW_DELIMITER);
+ }
+
+ if (config.hasPath(Constant.PATH) && !StringUtils.isBlank(config.getString(Constant.PATH))) {
+ this.path = config.getString(Constant.PATH);
+ }
+
+ if (config.hasPath(Constant.FILE_NAME_EXPRESSION) && !StringUtils.isBlank(config.getString(Constant.FILE_NAME_EXPRESSION))) {
+ this.fileNameExpression = config.getString(Constant.FILE_NAME_EXPRESSION);
+ }
+
+ if (config.hasPath(Constant.FILE_FORMAT) && !StringUtils.isBlank(config.getString(Constant.FILE_FORMAT))) {
+ this.fileFormat = FileFormat.valueOf(config.getString(Constant.FILE_FORMAT).toUpperCase(Locale.ROOT));
+ }
+ }
+
+ protected AbstractTextFileConfig() {
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/CompressConfig.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/CompressConfig.java
new file mode 100644
index 00000000000..48d47c8d1df
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/CompressConfig.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.config;
+
+public interface CompressConfig {
+ String getCompressCodec();
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/Constant.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/Constant.java
new file mode 100644
index 00000000000..8d10024cb3a
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/Constant.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.config;
+
+public class Constant {
+ public static final String SEATUNNEL = "seatunnel";
+ public static final String NON_PARTITION = "NON_PARTITION";
+ public static final String TRANSACTION_ID_SPLIT = "_";
+ public static final String TRANSACTION_EXPRESSION = "transactionId";
+
+ public static final String SAVE_MODE = "save_mode";
+ public static final String COMPRESS_CODEC = "compress_codec";
+
+ public static final String PATH = "path";
+ public static final String FIELD_DELIMITER = "field_delimiter";
+ public static final String ROW_DELIMITER = "row_delimiter";
+ public static final String PARTITION_BY = "partition_by";
+ public static final String PARTITION_DIR_EXPRESSION = "partition_dir_expression";
+ public static final String IS_PARTITION_FIELD_WRITE_IN_FILE = "is_partition_field_write_in_file";
+ public static final String TMP_PATH = "tmp_path";
+ public static final String FILE_NAME_EXPRESSION = "file_name_expression";
+ public static final String FILE_FORMAT = "file_format";
+ public static final String SINK_COLUMNS = "sink_columns";
+ public static final String FILENAME_TIME_FORMAT = "filename_time_format";
+ public static final String IS_ENABLE_TRANSACTION = "is_enable_transaction";
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/DelimiterConfig.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/DelimiterConfig.java
new file mode 100644
index 00000000000..146974c33a7
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/DelimiterConfig.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.config;
+
+public interface DelimiterConfig {
+ String getFieldDelimiter();
+
+ String getRowDelimiter();
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/FileFormat.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/FileFormat.java
new file mode 100644
index 00000000000..6b3f31f79e0
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/FileFormat.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.config;
+
+import java.io.Serializable;
+
+public enum FileFormat implements Serializable {
+ CSV("csv"),
+ TEXT("txt");
+
+ private String suffix;
+
+ private FileFormat(String suffix) {
+ this.suffix = suffix;
+ }
+
+ public String getSuffix() {
+ return "." + suffix;
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/PartitionConfig.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/PartitionConfig.java
new file mode 100644
index 00000000000..f77f69f3d93
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/PartitionConfig.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.config;
+
+import java.util.List;
+
+public interface PartitionConfig {
+ List getPartitionFieldList();
+
+ boolean isPartitionFieldWriteInFile();
+}
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
new file mode 100644
index 00000000000..a296eea53d7
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/AbstractFileSink.java
@@ -0,0 +1,155 @@
+/*
+ * 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.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;
+import org.apache.seatunnel.api.sink.SinkAggregatedCommitter;
+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.connectors.seatunnel.file.sink.config.TextFileSinkConfig;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.FileSystemCommitter;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.SinkFileSystemPlugin;
+
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+
+import java.io.IOException;
+import java.util.List;
+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 {
+ private Config config;
+ private String jobId;
+ private Long checkpointId;
+ private SeaTunnelRowType seaTunnelRowTypeInfo;
+ private SeaTunnelContext seaTunnelContext;
+ private TextFileSinkConfig textFileSinkConfig;
+ private SinkFileSystemPlugin sinkFileSystemPlugin;
+
+ public abstract SinkFileSystemPlugin getSinkFileSystemPlugin();
+
+ @Override
+ public String getPluginName() {
+ this.sinkFileSystemPlugin = getSinkFileSystemPlugin();
+ return this.sinkFileSystemPlugin.getPluginName();
+ }
+
+ @Override
+ public void setTypeInfo(SeaTunnelRowType seaTunnelRowTypeInfo) {
+ this.seaTunnelRowTypeInfo = seaTunnelRowTypeInfo;
+ }
+
+ @Override
+ public void prepare(Config pluginConfig) throws PrepareFailException {
+ this.config = pluginConfig;
+ this.checkpointId = 1L;
+ }
+
+ @Override
+ public SinkWriter createWriter(SinkWriter.Context context) throws IOException {
+ if (!seaTunnelContext.getJobMode().equals(JobMode.BATCH) && this.getSinkConfig().getSaveMode().equals(SaveMode.OVERWRITE)) {
+ throw new RuntimeException("only batch job can overwrite mode");
+ }
+
+ if (this.getSinkConfig().isEnableTransaction()) {
+ return new TransactionStateFileSinkWriter(seaTunnelRowTypeInfo,
+ config,
+ context,
+ getSinkConfig(),
+ jobId,
+ sinkFileSystemPlugin);
+ } else {
+ throw new RuntimeException("File Sink Connector only support transaction now");
+ }
+ }
+
+ @Override
+ public SinkWriter restoreWriter(SinkWriter.Context context, List states) throws IOException {
+ if (this.getSinkConfig().isEnableTransaction()) {
+ return new TransactionStateFileSinkWriter(seaTunnelRowTypeInfo,
+ config,
+ context,
+ textFileSinkConfig,
+ jobId,
+ states,
+ sinkFileSystemPlugin);
+ } else {
+ throw new RuntimeException("File Sink Connector only support transaction now");
+ }
+ }
+
+ @Override
+ public void setSeaTunnelContext(SeaTunnelContext seaTunnelContext) {
+ this.seaTunnelContext = seaTunnelContext;
+ this.jobId = seaTunnelContext.getJobId();
+ }
+
+ @Override
+ public Optional> createAggregatedCommitter() throws IOException {
+ if (this.getSinkConfig().isEnableTransaction()) {
+ Optional fileSystemCommitter = sinkFileSystemPlugin.getFileSystemCommitter();
+ if (fileSystemCommitter.isPresent()) {
+ return Optional.of(new FileSinkAggregatedCommitter(fileSystemCommitter.get()));
+ } else {
+ throw new RuntimeException("FileSystemCommitter is need");
+ }
+ } else {
+ return Optional.empty();
+ }
+ }
+
+ @Override
+ public Optional> getWriterStateSerializer() {
+ return Optional.of(new DefaultSerializer<>());
+ }
+
+ @Override
+ public Optional> getAggregatedCommitInfoSerializer() {
+ return Optional.of(new DefaultSerializer<>());
+ }
+
+ @Override
+ public Optional> getCommitInfoSerializer() {
+ return Optional.of(new DefaultSerializer<>());
+ }
+
+ private TextFileSinkConfig getSinkConfig() {
+ if (this.textFileSinkConfig == null && (this.seaTunnelRowTypeInfo != null && this.config != null)) {
+ this.textFileSinkConfig = new TextFileSinkConfig(config, seaTunnelRowTypeInfo);
+ }
+ return this.textFileSinkConfig;
+ }
+
+ @Override
+ public SeaTunnelDataType getConsumedType() {
+ return this.seaTunnelRowTypeInfo;
+ }
+}
+
+
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
new file mode 100644
index 00000000000..1036c3a59e5
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/FileAggregatedCommitInfo.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.sink;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+import java.io.Serializable;
+import java.util.Map;
+
+@Data
+@AllArgsConstructor
+public class FileAggregatedCommitInfo implements Serializable {
+
+ /**
+ * Storage the commit info in map.
+ * K is the file path need to be moved to target dir.
+ * V is the target file path of the data file.
+ */
+ private Map> transactionMap;
+}
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
new file mode 100644
index 00000000000..689b85ebf80
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/FileCommitInfo.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.sink;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+import java.io.Serializable;
+import java.util.Map;
+
+@Data
+@AllArgsConstructor
+public class FileCommitInfo implements Serializable {
+
+ /**
+ * Storage the commit info in map.
+ * K is the file path need to be moved to target dir.
+ * V is the target file path of the data file.
+ */
+ private Map needMoveFiles;
+
+ 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
new file mode 100644
index 00000000000..cc8ff240486
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/FileSinkAggregatedCommitter.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.sink;
+
+import org.apache.seatunnel.api.sink.SinkAggregatedCommitter;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.FileSystemCommitter;
+
+import lombok.NonNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class FileSinkAggregatedCommitter implements SinkAggregatedCommitter {
+ private static final Logger LOGGER = LoggerFactory.getLogger(FileSinkAggregatedCommitter.class);
+
+ private FileSystemCommitter fileSystemCommitter;
+
+ public FileSinkAggregatedCommitter(@NonNull FileSystemCommitter fileSystemCommitter) {
+ this.fileSystemCommitter = fileSystemCommitter;
+ }
+
+ @Override
+ public List commit(List aggregatedCommitInfoList) throws IOException {
+ if (aggregatedCommitInfoList == null || aggregatedCommitInfoList.size() == 0) {
+ return null;
+ }
+ List errorAggregatedCommitInfoList = new ArrayList();
+ aggregatedCommitInfoList.stream().forEach(aggregateCommitInfo -> {
+ try {
+ fileSystemCommitter.commitTransaction(aggregateCommitInfo);
+ } catch (Exception e) {
+ LOGGER.error("commit aggregateCommitInfo error ", e);
+ errorAggregatedCommitInfoList.add(aggregateCommitInfo);
+ }
+ });
+
+ return errorAggregatedCommitInfoList;
+ }
+
+ @Override
+ public FileAggregatedCommitInfo combine(List commitInfos) {
+ if (commitInfos == null || commitInfos.size() == 0) {
+ return null;
+ }
+ Map> aggregateCommitInfo = new HashMap<>();
+ commitInfos.stream().forEach(commitInfo -> {
+ Map needMoveFileMap = aggregateCommitInfo.get(commitInfo.getTransactionDir());
+ if (needMoveFileMap == null) {
+ needMoveFileMap = new HashMap<>();
+ aggregateCommitInfo.put(commitInfo.getTransactionDir(), needMoveFileMap);
+ }
+ needMoveFileMap.putAll(commitInfo.getNeedMoveFiles());
+ });
+ return new FileAggregatedCommitInfo(aggregateCommitInfo);
+ }
+
+ @Override
+ public void abort(List aggregatedCommitInfoList) throws Exception {
+ if (aggregatedCommitInfoList == null || aggregatedCommitInfoList.size() == 0) {
+ return;
+ }
+ aggregatedCommitInfoList.stream().forEach(aggregateCommitInfo -> {
+ try {
+ fileSystemCommitter.abortTransaction(aggregateCommitInfo);
+
+ } catch (Exception e) {
+ LOGGER.error("abort aggregateCommitInfo error ", e);
+ }
+ });
+ }
+
+ @Override
+ public void close() throws IOException {
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/FileSinkState.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/FileSinkState.java
new file mode 100644
index 00000000000..1b7e6b8c523
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/FileSinkState.java
@@ -0,0 +1,30 @@
+/*
+ * 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 lombok.AllArgsConstructor;
+import lombok.Data;
+
+import java.io.Serializable;
+
+@Data
+@AllArgsConstructor
+public class FileSinkState implements Serializable {
+ private String transactionId;
+ private Long checkpointId;
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/TransactionStateFileSinkWriter.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/TransactionStateFileSinkWriter.java
new file mode 100644
index 00000000000..0bdad1afef1
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/TransactionStateFileSinkWriter.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 TransactionStateFileSinkWriter implements SinkWriter {
+ private static final Logger LOGGER = LoggerFactory.getLogger(TransactionStateFileSinkWriter.class);
+
+ private SeaTunnelRowType seaTunnelRowTypeInfo;
+ private Config pluginConfig;
+ private Context context;
+ private String jobId;
+
+ private TransactionStateFileWriter fileWriter;
+
+ private TextFileSinkConfig textFileSinkConfig;
+
+ public TransactionStateFileSinkWriter(@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 = 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 TransactionStateFileSinkWriter(@NonNull SeaTunnelRowType seaTunnelRowTypeInfo,
+ @NonNull Config pluginConfig,
+ @NonNull SinkWriter.Context context,
+ @NonNull TextFileSinkConfig textFileSinkConfig,
+ @NonNull String jobId,
+ @NonNull List fileSinkStates,
+ @NonNull SinkFileSystemPlugin sinkFileSystemPlugin) {
+ this.seaTunnelRowTypeInfo = seaTunnelRowTypeInfo;
+ this.pluginConfig = pluginConfig;
+ this.context = context;
+ this.jobId = jobId;
+
+ Optional 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 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 prepareCommit() throws IOException {
+ return fileWriter.prepareCommit();
+ }
+
+ @Override
+ public void abortPrepare() {
+ fileWriter.abortTransaction();
+ }
+
+ @Override
+ public void close() throws IOException {
+ fileWriter.finishAndCloseWriteFile();
+ }
+
+ @Override
+ public List snapshotState(long checkpointId) throws IOException {
+ List 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/FileSystemType.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/config/FileSystemType.java
new file mode 100644
index 00000000000..58c1ba15702
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/config/FileSystemType.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.sink.config;
+
+import java.io.Serializable;
+
+public enum FileSystemType implements Serializable {
+ HDFS("HdfsFile"),
+ LOCAL("LocalFile");
+
+ private String sinkFileSystemPluginName;
+
+ private FileSystemType(String sinkFileSystemPluginName) {
+ this.sinkFileSystemPluginName = sinkFileSystemPluginName;
+ }
+
+ public String getSinkFileSystemPluginName() {
+ return sinkFileSystemPluginName;
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/config/SaveMode.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/config/SaveMode.java
new file mode 100644
index 00000000000..d46a75c77ac
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/config/SaveMode.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.sink.config;
+
+import lombok.NonNull;
+
+import java.io.Serializable;
+import java.util.Locale;
+
+public enum SaveMode implements Serializable {
+ APPEND(),
+ OVERWRITE(),
+ IGNORE(),
+ ERROR();
+
+ public static SaveMode fromStr(@NonNull String str) {
+ return SaveMode.valueOf(str.toUpperCase(Locale.ROOT));
+ }
+}
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
new file mode 100644
index 00000000000..63b747bf206
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/config/TextFileSinkConfig.java
@@ -0,0 +1,146 @@
+/*
+ * 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.config;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+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.Constant;
+import org.apache.seatunnel.connectors.seatunnel.file.config.PartitionConfig;
+
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+
+import lombok.Data;
+import lombok.NonNull;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang3.StringUtils;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+@Data
+public class TextFileSinkConfig extends AbstractTextFileConfig implements PartitionConfig {
+
+ private List sinkColumnList;
+
+ private List partitionFieldList;
+
+ /**
+ * default is ${k1}=${v1}/${k2}=${v2}/...
+ */
+ private String partitionDirExpression;
+
+ private boolean isPartitionFieldWriteInFile = false;
+
+ private String tmpPath = "/tmp/seatunnel";
+
+ private SaveMode saveMode = SaveMode.ERROR;
+
+ private String fileNameTimeFormat = "yyyy.MM.dd";
+
+ private boolean isEnableTransaction = true;
+
+ //---------------------generator by config params-------------------
+
+ private List sinkColumnsIndexInRow;
+
+ private List partitionFieldsIndexInRow;
+
+ public TextFileSinkConfig(@NonNull Config config, @NonNull SeaTunnelRowType seaTunnelRowTypeInfo) {
+ super(config);
+ checkArgument(!CollectionUtils.isEmpty(Arrays.asList(seaTunnelRowTypeInfo.getFieldNames())));
+
+ if (config.hasPath(Constant.FILE_FORMAT) && !CollectionUtils.isEmpty(config.getStringList(Constant.SINK_COLUMNS))) {
+ this.sinkColumnList = config.getStringList(Constant.SINK_COLUMNS);
+ }
+
+ // if the config sink_columns is empty, all fields in SeaTunnelRowTypeInfo will being write
+ if (CollectionUtils.isEmpty(this.sinkColumnList)) {
+ this.sinkColumnList = Arrays.asList(seaTunnelRowTypeInfo.getFieldNames());
+ }
+
+ if (config.hasPath(Constant.PARTITION_BY) && !CollectionUtils.isEmpty(config.getStringList(Constant.PARTITION_BY))) {
+ this.partitionFieldList = config.getStringList(Constant.PARTITION_BY);
+ }
+
+ if (config.hasPath(Constant.PARTITION_DIR_EXPRESSION) && !StringUtils.isBlank(config.getString(Constant.PARTITION_DIR_EXPRESSION))) {
+ this.partitionDirExpression = config.getString(Constant.PARTITION_DIR_EXPRESSION);
+ }
+
+ if (config.hasPath(Constant.IS_PARTITION_FIELD_WRITE_IN_FILE) && config.getBoolean(Constant.IS_PARTITION_FIELD_WRITE_IN_FILE)) {
+ this.isPartitionFieldWriteInFile = config.getBoolean(Constant.IS_PARTITION_FIELD_WRITE_IN_FILE);
+ }
+
+ if (config.hasPath(Constant.TMP_PATH) && !StringUtils.isBlank(config.getString(Constant.TMP_PATH))) {
+ this.tmpPath = config.getString(Constant.TMP_PATH);
+ }
+
+ if (config.hasPath(Constant.SAVE_MODE) && !StringUtils.isBlank(config.getString(Constant.SAVE_MODE))) {
+ this.saveMode = SaveMode.fromStr(config.getString(Constant.SAVE_MODE));
+ }
+
+ if (config.hasPath(Constant.FILENAME_TIME_FORMAT) && !StringUtils.isBlank(config.getString(Constant.FILENAME_TIME_FORMAT))) {
+ this.fileNameTimeFormat = config.getString(Constant.FILENAME_TIME_FORMAT);
+ }
+
+ if (config.hasPath(Constant.IS_ENABLE_TRANSACTION) && !config.getBoolean(Constant.IS_ENABLE_TRANSACTION)) {
+ this.isEnableTransaction = isEnableTransaction();
+ }
+
+ if (this.isEnableTransaction && !this.fileNameExpression.contains(Constant.TRANSACTION_EXPRESSION)) {
+ throw new RuntimeException("file_name_expression must contains " + Constant.TRANSACTION_EXPRESSION + " when is_enable_transaction is true");
+ }
+
+ // check partition field must in seaTunnelRowTypeInfo
+ if (!CollectionUtils.isEmpty(this.partitionFieldList)
+ && (CollectionUtils.isEmpty(this.sinkColumnList) || !this.sinkColumnList.containsAll(this.partitionFieldList))) {
+ throw new RuntimeException("partition fields must in sink columns");
+ }
+
+ if (!CollectionUtils.isEmpty(this.partitionFieldList) && !isPartitionFieldWriteInFile) {
+ if (!this.sinkColumnList.removeAll(this.partitionFieldList)) {
+ throw new RuntimeException("remove partition field from sink columns error");
+ }
+ }
+
+ if (CollectionUtils.isEmpty(this.sinkColumnList)) {
+ throw new RuntimeException("sink columns can not be empty");
+ }
+
+ Map columnsMap = new HashMap<>(seaTunnelRowTypeInfo.getFieldNames().length);
+ String[] fieldNames = seaTunnelRowTypeInfo.getFieldNames();
+ for (int i = 0; i < fieldNames.length; i++) {
+ columnsMap.put(fieldNames[i], i);
+ }
+
+ // init sink column index and partition field index, we will use the column index to found the data in SeaTunnelRow
+ this.sinkColumnsIndexInRow = this.sinkColumnList.stream()
+ .map(columnName -> columnsMap.get(columnName))
+ .collect(Collectors.toList());
+
+ if (!CollectionUtils.isEmpty(this.partitionFieldList)) {
+ this.partitionFieldsIndexInRow = this.partitionFieldList.stream()
+ .map(columnName -> columnsMap.get(columnName))
+ .collect(Collectors.toList());
+ }
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/spi/FileSystem.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/spi/FileSystem.java
new file mode 100644
index 00000000000..938a4108cee
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/spi/FileSystem.java
@@ -0,0 +1,29 @@
+/*
+ * 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.spi;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.List;
+
+public interface FileSystem extends Serializable {
+
+ void deleteFile(String path) throws IOException;
+
+ List dirList(String dirPath) throws IOException;
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/spi/FileSystemCommitter.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/spi/FileSystemCommitter.java
new file mode 100644
index 00000000000..4dcba5b21c9
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/spi/FileSystemCommitter.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.sink.spi;
+
+import org.apache.seatunnel.connectors.seatunnel.file.sink.FileAggregatedCommitInfo;
+
+import lombok.NonNull;
+
+import java.io.IOException;
+import java.io.Serializable;
+
+public interface FileSystemCommitter extends Serializable {
+
+ void commitTransaction(@NonNull FileAggregatedCommitInfo fileAggregatedCommitInfo) throws IOException;
+
+ void abortTransaction(@NonNull FileAggregatedCommitInfo fileAggregatedCommitInfo) throws IOException;
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/spi/SinkFileSystemPlugin.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/spi/SinkFileSystemPlugin.java
new file mode 100644
index 00000000000..97c6ab9904a
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/spi/SinkFileSystemPlugin.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.sink.spi;
+
+import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.transaction.TransactionFileNameGenerator;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.transaction.TransactionStateFileWriter;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.PartitionDirNameGenerator;
+
+import lombok.NonNull;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.Optional;
+
+public interface SinkFileSystemPlugin extends Serializable {
+
+ String getPluginName();
+
+ /**
+ * Implements this method and return a class which is implement the interface {@link TransactionStateFileWriter}
+ *
+ * @return
+ */
+ Optional getTransactionStateFileWriter(@NonNull SeaTunnelRowType seaTunnelRowTypeInfo,
+ @NonNull TransactionFileNameGenerator transactionFileNameGenerator,
+ @NonNull PartitionDirNameGenerator partitionDirNameGenerator,
+ @NonNull List sinkColumnsIndexInRow,
+ @NonNull String tmpPath,
+ @NonNull String targetPath,
+ @NonNull String jobId,
+ int subTaskIndex,
+ @NonNull String fieldDelimiter,
+ @NonNull String rowDelimiter,
+ @NonNull FileSystem fileSystem);
+
+ Optional getFileSystemCommitter();
+
+ Optional getFileSystem();
+}
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
new file mode 100644
index 00000000000..dd8d41bf6ef
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/transaction/Transaction.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.sink.transaction;
+
+import org.apache.seatunnel.api.sink.SinkWriter;
+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;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.Optional;
+
+public interface Transaction extends Serializable {
+ /**
+ * A new transaction needs to be started after each checkpoint is completed.
+ *
+ * @param checkpointId A checkpoint indicates that all tasks have a status snapshot operation
+ * @return transactionId
+ */
+ String beginTransaction(@NonNull Long checkpointId);
+
+ /**
+ * Abort current Transaction, called when {@link TransactionStateFileSinkWriter#prepareCommit()} or {@link TransactionStateFileSinkWriter#snapshotState(long)} failed
+ */
+ void abortTransaction();
+
+ /**
+ * Get all transactionIds after the @param transactionId
+ * This method called when {@link AbstractFileSink#restoreWriter(SinkWriter.Context, List)}
+ * We get the transactionId of the last successful commit from {@link FileSinkState} and
+ * then all transactionIds after this transactionId is dirty transactions that need to be rollback.
+ *
+ * @param transactionId The transactionId of the last successful commit get from {@link FileSinkState}
+ * @return transactionId list
+ */
+ List getTransactionAfter(@NonNull String transactionId);
+
+ /**
+ * Called by {@link 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)}
+ */
+ Optional prepareCommit();
+
+ /**
+ * rollback the transaction which is not be commit
+ *
+ * @param transactionIds transactionIds
+ */
+ void abortTransactions(List transactionIds);
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/transaction/TransactionFileNameGenerator.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/transaction/TransactionFileNameGenerator.java
new file mode 100644
index 00000000000..e976910bebe
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/transaction/TransactionFileNameGenerator.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.sink.transaction;
+
+import java.io.Serializable;
+
+public interface TransactionFileNameGenerator extends Serializable {
+ String generateFileName(String transactionId);
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/transaction/TransactionStateFileWriter.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/transaction/TransactionStateFileWriter.java
new file mode 100644
index 00000000000..a1a66ec20a4
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/transaction/TransactionStateFileWriter.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.sink.transaction;
+
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.FileSinkState;
+
+import lombok.NonNull;
+
+import java.util.List;
+
+public interface TransactionStateFileWriter extends Transaction {
+ void write(@NonNull SeaTunnelRow seaTunnelRow);
+
+ /**
+ * 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();
+
+ /**
+ * snapshotState
+ * @param checkpointId checkpointId
+ * @return
+ */
+ List snapshotState(long checkpointId);
+}
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
new file mode 100644
index 00000000000..784d87046ea
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/AbstractTransactionStateFileWriter.java
@@ -0,0 +1,191 @@
+/*
+ * 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.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.file.config.Constant;
+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.spi.FileSystem;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.transaction.TransactionFileNameGenerator;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.transaction.TransactionStateFileWriter;
+
+import com.google.common.collect.Lists;
+import lombok.NonNull;
+import org.apache.commons.collections4.CollectionUtils;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+public abstract class AbstractTransactionStateFileWriter implements TransactionStateFileWriter {
+ protected Map needMoveFiles;
+ protected SeaTunnelRowType seaTunnelRowTypeInfo;
+ protected String jobId;
+ protected int subTaskIndex;
+
+ protected Map beingWrittenFile;
+
+ protected String transactionId;
+
+ protected String transactionDir;
+
+ private long checkpointId;
+
+ private TransactionFileNameGenerator transactionFileNameGenerator;
+
+ protected List sinkColumnsIndexInRow;
+
+ private String targetPath;
+
+ private String tmpPath;
+
+ private PartitionDirNameGenerator partitionDirNameGenerator;
+
+ private FileSystem fileSystem;
+
+ public AbstractTransactionStateFileWriter(@NonNull SeaTunnelRowType seaTunnelRowTypeInfo,
+ @NonNull TransactionFileNameGenerator transactionFileNameGenerator,
+ @NonNull PartitionDirNameGenerator partitionDirNameGenerator,
+ @NonNull List sinkColumnsIndexInRow,
+ @NonNull String tmpPath,
+ @NonNull String targetPath,
+ @NonNull String jobId,
+ int subTaskIndex,
+ @NonNull FileSystem fileSystem) {
+ checkArgument(subTaskIndex > -1);
+
+ this.seaTunnelRowTypeInfo = seaTunnelRowTypeInfo;
+ this.transactionFileNameGenerator = transactionFileNameGenerator;
+ this.sinkColumnsIndexInRow = sinkColumnsIndexInRow;
+ this.tmpPath = tmpPath;
+ this.targetPath = targetPath;
+ this.jobId = jobId;
+ this.subTaskIndex = subTaskIndex;
+ this.partitionDirNameGenerator = partitionDirNameGenerator;
+ this.fileSystem = fileSystem;
+ }
+
+ public String getOrCreateFilePathBeingWritten(@NonNull SeaTunnelRow seaTunnelRow) {
+ String beingWrittenFileKey = this.partitionDirNameGenerator.generatorPartitionDir(seaTunnelRow);
+ // get filePath from beingWrittenFile
+ String beingWrittenFilePath = beingWrittenFile.get(beingWrittenFileKey);
+ if (beingWrittenFilePath != null) {
+ return beingWrittenFilePath;
+ } else {
+ StringBuilder sbf = new StringBuilder(this.transactionDir);
+ sbf.append("/").append(beingWrittenFileKey).append("/").append(transactionFileNameGenerator.generateFileName(this.transactionId));
+ String newBeingWrittenFilePath = sbf.toString();
+ beingWrittenFile.put(beingWrittenFileKey, newBeingWrittenFilePath);
+ return newBeingWrittenFilePath;
+ }
+ }
+
+ public String getTargetLocation(@NonNull String seaTunnelFilePath) {
+ String tmpPath = seaTunnelFilePath.replaceAll(this.transactionDir, targetPath);
+ return tmpPath.replaceAll(Constant.NON_PARTITION + "/", "");
+ }
+
+ @Override
+ public String beginTransaction(@NonNull Long checkpointId) {
+ this.finishAndCloseWriteFile();
+ 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.beingWrittenFile = new HashMap<>();
+ this.beginTransaction(this.transactionId);
+ this.checkpointId = checkpointId;
+ return this.transactionId;
+ }
+
+ private String getTransactionDir(@NonNull String transactionId) {
+ StringBuilder sbf = new StringBuilder(this.tmpPath);
+ sbf.append("/").append(Constant.SEATUNNEL).append("/").append(jobId).append("/").append(transactionId);
+ return sbf.toString();
+ }
+
+ public abstract void beginTransaction(String transactionId);
+
+ @Override
+ public void abortTransaction() {
+ this.finishAndCloseWriteFile();
+ //drop transaction dir
+ try {
+ abortTransaction(this.transactionId);
+ fileSystem.deleteFile(this.transactionDir);
+ } catch (IOException e) {
+ throw new RuntimeException("abort transaction " + this.transactionId + " error.", e);
+ }
+ }
+
+ public abstract void abortTransaction(String transactionId);
+
+ @Override
+ public List getTransactionAfter(@NonNull String transactionId) {
+ StringBuilder sbf = new StringBuilder(this.targetPath);
+ sbf.append("/").append(Constant.SEATUNNEL).append("/").append(jobId).append("/");
+ String jobDir = sbf.toString();
+
+ //get all transaction dir
+ try {
+ List transactionDirList = fileSystem.dirList(jobDir);
+ List transactionIdList = transactionDirList.stream().map(dir -> dir.replaceAll(jobDir, "")).collect(Collectors.toList());
+ return transactionIdList;
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ @Override
+ public Optional prepareCommit() {
+ this.finishAndCloseWriteFile();
+ // this.needMoveFiles will be clear when beginTransaction, so we need copy the needMoveFiles.
+ Map commitMap = new HashMap<>();
+ commitMap.putAll(this.needMoveFiles);
+ return Optional.of(new FileCommitInfo(commitMap, this.transactionDir));
+ }
+
+ @Override
+ public void abortTransactions(List transactionIds) {
+ if (CollectionUtils.isEmpty(transactionIds)) {
+ return;
+ }
+
+ transactionIds.stream().forEach(transactionId -> {
+ try {
+ abortTransaction(transactionId);
+ fileSystem.deleteFile(transactionId);
+ } catch (IOException e) {
+ throw new RuntimeException("abort transaction " + transactionId + " error.", e);
+ }
+ });
+ }
+
+ @Override
+ public List snapshotState(long checkpointId) {
+ ArrayList fileSinkStates = Lists.newArrayList(new FileSinkState(this.transactionId, this.checkpointId));
+ return fileSinkStates;
+ }
+}
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
new file mode 100644
index 00000000000..aa8a3656b92
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/FileSinkPartitionDirNameGenerator.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.sink.writer;
+
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+import org.apache.seatunnel.common.utils.VariablesSubstitute;
+import org.apache.seatunnel.connectors.seatunnel.file.config.Constant;
+
+import lombok.Data;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang3.StringUtils;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+@Data
+public class FileSinkPartitionDirNameGenerator implements PartitionDirNameGenerator {
+ private List partitionFieldList;
+
+ private List partitionFieldsIndexInRow;
+
+ private String partitionDirExpression;
+
+ private String[] keys;
+
+ private String[] values;
+
+ public FileSinkPartitionDirNameGenerator(List partitionFieldList,
+ List partitionFieldsIndexInRow,
+ String partitionDirExpression) {
+ this.partitionFieldList = partitionFieldList;
+ this.partitionFieldsIndexInRow = partitionFieldsIndexInRow;
+ this.partitionDirExpression = partitionDirExpression;
+
+ if (!CollectionUtils.isEmpty(partitionFieldList)) {
+ keys = new String[partitionFieldList.size()];
+ values = new String[partitionFieldList.size()];
+ for (int i = 0; i < partitionFieldList.size(); i++) {
+ keys[i] = "k" + i;
+ values[i] = "v" + i;
+ }
+ }
+ }
+
+ @Override
+ public String generatorPartitionDir(SeaTunnelRow seaTunnelRow) {
+ if (CollectionUtils.isEmpty(this.partitionFieldsIndexInRow)) {
+ return Constant.NON_PARTITION;
+ }
+
+ if (StringUtils.isBlank(partitionDirExpression)) {
+ StringBuilder sbd = new StringBuilder();
+ for (int i = 0; i < partitionFieldsIndexInRow.size(); i++) {
+ sbd.append(partitionFieldList.get(i))
+ .append("=")
+ .append(seaTunnelRow.getFields()[partitionFieldsIndexInRow.get(i)])
+ .append("/");
+ }
+ return sbd.toString();
+ } else {
+ Map 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());
+ }
+ return VariablesSubstitute.substitute(partitionDirExpression, valueMap);
+ }
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/FileSinkTransactionFileNameGenerator.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/FileSinkTransactionFileNameGenerator.java
new file mode 100644
index 00000000000..ba005c7de49
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/FileSinkTransactionFileNameGenerator.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.sink.writer;
+
+import org.apache.seatunnel.common.utils.VariablesSubstitute;
+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.transaction.TransactionFileNameGenerator;
+
+import lombok.NonNull;
+import org.apache.commons.lang3.StringUtils;
+
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+
+public class FileSinkTransactionFileNameGenerator implements TransactionFileNameGenerator {
+ private FileFormat fileFormat;
+
+ private String fileNameExpression;
+
+ private String timeFormat;
+
+ public FileSinkTransactionFileNameGenerator(@NonNull FileFormat fileFormat,
+ String fileNameExpression,
+ @NonNull String timeFormat) {
+ this.fileFormat = fileFormat;
+ this.fileNameExpression = fileNameExpression;
+ this.timeFormat = timeFormat;
+ }
+
+ @Override
+ public String generateFileName(String transactionId) {
+ if (StringUtils.isBlank(fileNameExpression)) {
+ return transactionId + fileFormat.getSuffix();
+ }
+ DateTimeFormatter df = DateTimeFormatter.ofPattern(timeFormat);
+ final String formattedDate = df.format(ZonedDateTime.now());
+
+ final Map valuesMap = new HashMap<>(4);
+ valuesMap.put("uuid", UUID.randomUUID().toString());
+ valuesMap.put("now", formattedDate);
+ valuesMap.put(timeFormat, formattedDate);
+ valuesMap.put(Constant.TRANSACTION_EXPRESSION, transactionId);
+ String substitute = VariablesSubstitute.substitute(fileNameExpression, valuesMap);
+ return substitute + fileFormat.getSuffix();
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/FileWriter.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/FileWriter.java
new file mode 100644
index 00000000000..276c981fa64
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/FileWriter.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.sink.writer;
+
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+
+import lombok.NonNull;
+
+import java.io.Serializable;
+
+public interface FileWriter extends Serializable {
+
+ void write(@NonNull SeaTunnelRow seaTunnelRow);
+
+ /**
+ * 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();
+}
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
new file mode 100644
index 00000000000..1145e847d1c
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/PartitionDirNameGenerator.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.sink.writer;
+
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+
+import java.io.Serializable;
+
+public interface PartitionDirNameGenerator extends Serializable {
+ 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
new file mode 100644
index 00000000000..1989275f497
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/writer/TestFileSinkPartitionDirNameGenerator.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.writer;
+
+import org.apache.seatunnel.api.table.type.BasicType;
+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.connectors.seatunnel.file.config.Constant;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.FileSinkPartitionDirNameGenerator;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.PartitionDirNameGenerator;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import java.util.ArrayList;
+import java.util.List;
+
+@RunWith(PowerMockRunner.class)
+public class TestFileSinkPartitionDirNameGenerator {
+
+ @SuppressWarnings({"checkstyle:MagicNumber", "checkstyle:RegexpSingleline"})
+ @Test
+ public void testPartitionDirNameGenerator() {
+ String[] fieldNames = new String[]{"c1", "c2", "c3", "c4"};
+ SeaTunnelDataType[] seaTunnelDataTypes = new SeaTunnelDataType[]{BasicType.BOOLEAN_TYPE, BasicType.INT_TYPE, BasicType.STRING_TYPE, BasicType.INT_TYPE};
+ SeaTunnelRowType seaTunnelRowTypeInfo = new SeaTunnelRowType(fieldNames, seaTunnelDataTypes);
+
+ Object[] row1 = new Object[]{true, 1, "test", 3};
+ SeaTunnelRow seaTunnelRow = new SeaTunnelRow(row1);
+
+ List partitionFieldList = new ArrayList<>();
+ partitionFieldList.add("c3");
+ partitionFieldList.add("c4");
+
+ List partitionFieldsIndexInRow = new ArrayList<>();
+ partitionFieldsIndexInRow.add(2);
+ partitionFieldsIndexInRow.add(3);
+
+ PartitionDirNameGenerator partitionDirNameGenerator = new FileSinkPartitionDirNameGenerator(partitionFieldList, partitionFieldsIndexInRow, "${v0}/${v1}");
+ String partitionDir = partitionDirNameGenerator.generatorPartitionDir(seaTunnelRow);
+ Assert.assertEquals("test/3", partitionDir);
+
+ partitionDirNameGenerator = new FileSinkPartitionDirNameGenerator(partitionFieldList, partitionFieldsIndexInRow, "${k0}=${v0}/${k1}=${v1}");
+ partitionDir = partitionDirNameGenerator.generatorPartitionDir(seaTunnelRow);
+ Assert.assertEquals("c3=test/c4=3", partitionDir);
+
+ partitionDirNameGenerator = new FileSinkPartitionDirNameGenerator(null, null, "${k0}=${v0}/${k1}=${v1}");
+ partitionDir = partitionDirNameGenerator.generatorPartitionDir(seaTunnelRow);
+ Assert.assertEquals(Constant.NON_PARTITION, partitionDir);
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/writer/TestFileSinkTransactionFileNameGenerator.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/writer/TestFileSinkTransactionFileNameGenerator.java
new file mode 100644
index 00000000000..275aee86317
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/writer/TestFileSinkTransactionFileNameGenerator.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.writer;
+
+import org.apache.seatunnel.connectors.seatunnel.file.config.FileFormat;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.FileSinkTransactionFileNameGenerator;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+
+@RunWith(PowerMockRunner.class)
+public class TestFileSinkTransactionFileNameGenerator {
+
+ @Test
+ public void testGenerateFileName() {
+ FileSinkTransactionFileNameGenerator fileNameGenerator = new FileSinkTransactionFileNameGenerator(FileFormat.TEXT, "test_${transactionId}_${uuid}_${now}", "yyyy.MM.dd");
+ DateTimeFormatter df = DateTimeFormatter.ofPattern("yyyy.MM.dd");
+ final String formattedDate = df.format(ZonedDateTime.now());
+ String fileName = fileNameGenerator.generateFileName("T_12345678_1_0");
+ Assert.assertTrue(fileName.startsWith("test_T_12345678_1_0_"));
+ Assert.assertTrue(fileName.endsWith(formattedDate + ".txt"));
+
+ fileNameGenerator = new FileSinkTransactionFileNameGenerator(FileFormat.TEXT, null, "yyyy.MM.dd");
+ fileName = fileNameGenerator.generateFileName("T_12345678_1_0");
+ Assert.assertEquals("T_12345678_1_0.txt", fileName);
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/pom.xml b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/pom.xml
new file mode 100644
index 00000000000..400f47d76d8
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/pom.xml
@@ -0,0 +1,61 @@
+
+
+
+
+ connector-file
+ org.apache.seatunnel
+ ${revision}
+
+ 4.0.0
+
+ connector-file-hadoop
+
+
+
+ org.apache.seatunnel
+ connector-file-base
+ ${project.version}
+
+
+ org.apache.flink
+ flink-shaded-hadoop-2
+ ${flink-shaded-hadoop-2.version}
+ provided
+
+
+ junit
+ junit
+ test
+
+
+
+ org.powermock
+ powermock-module-junit4
+ test
+
+
+ org.powermock
+ powermock-api-mockito2
+ test
+
+
+
\ No newline at end of file
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/HdfsFileSink.java b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/HdfsFileSink.java
new file mode 100644
index 00000000000..240a4bbc753
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/HdfsFileSink.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.sink.hdfs;
+
+import org.apache.seatunnel.api.sink.SeaTunnelSink;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.AbstractFileSink;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.SinkFileSystemPlugin;
+
+import com.google.auto.service.AutoService;
+
+@AutoService(SeaTunnelSink.class)
+public class HdfsFileSink extends AbstractFileSink {
+ @Override
+ public SinkFileSystemPlugin getSinkFileSystemPlugin() {
+ return new HdfsFileSinkPlugin();
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/HdfsFileSinkPlugin.java b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/HdfsFileSinkPlugin.java
new file mode 100644
index 00000000000..6e6c9380cf0
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/HdfsFileSinkPlugin.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.connectors.seatunnel.file.sink.hdfs;
+
+import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.config.FileSystemType;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.FileSystem;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.FileSystemCommitter;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.SinkFileSystemPlugin;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.transaction.TransactionFileNameGenerator;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.transaction.TransactionStateFileWriter;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.PartitionDirNameGenerator;
+
+import lombok.NonNull;
+
+import java.util.List;
+import java.util.Optional;
+
+public class HdfsFileSinkPlugin implements SinkFileSystemPlugin {
+ @Override
+ public String getPluginName() {
+ return FileSystemType.HDFS.getSinkFileSystemPluginName();
+ }
+
+ @Override
+ public Optional getTransactionStateFileWriter(@NonNull SeaTunnelRowType seaTunnelRowTypeInfo,
+ @NonNull TransactionFileNameGenerator transactionFileNameGenerator,
+ @NonNull PartitionDirNameGenerator partitionDirNameGenerator,
+ @NonNull List sinkColumnsIndexInRow,
+ @NonNull String tmpPath,
+ @NonNull String targetPath,
+ @NonNull String jobId,
+ int subTaskIndex,
+ @NonNull String fieldDelimiter,
+ @NonNull String rowDelimiter,
+ @NonNull FileSystem fileSystem) {
+ return Optional.of(new HdfsTxtTransactionStateFileWriter(seaTunnelRowTypeInfo,
+ transactionFileNameGenerator,
+ partitionDirNameGenerator,
+ sinkColumnsIndexInRow,
+ tmpPath,
+ targetPath,
+ jobId,
+ subTaskIndex,
+ fieldDelimiter,
+ rowDelimiter,
+ fileSystem));
+ }
+
+ @Override
+ public Optional getFileSystemCommitter() {
+ return Optional.of(new HdfsFileSystemCommitter());
+ }
+
+ @Override
+ public Optional getFileSystem() {
+ return Optional.of(new HdfsFileSystem());
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/HdfsFileSystem.java b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/HdfsFileSystem.java
new file mode 100644
index 00000000000..00d7c6f064f
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/HdfsFileSystem.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.sink.hdfs;
+
+import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.FileSystem;
+
+import org.apache.hadoop.fs.Path;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.stream.Collectors;
+
+public class HdfsFileSystem implements FileSystem {
+ @Override
+ public void deleteFile(String path) throws IOException {
+ HdfsUtils.deleteFile(path);
+ }
+
+ @Override
+ public List dirList(String dirPath) throws IOException {
+ List paths = HdfsUtils.dirList(dirPath);
+ return paths.stream().map(dir -> dir.getName()).collect(Collectors.toList());
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/HdfsFileSystemCommitter.java b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/HdfsFileSystemCommitter.java
new file mode 100644
index 00000000000..69884720636
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/HdfsFileSystemCommitter.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.file.sink.hdfs;
+
+import org.apache.seatunnel.connectors.seatunnel.file.sink.FileAggregatedCommitInfo;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.FileSystemCommitter;
+
+import lombok.NonNull;
+
+import java.io.IOException;
+import java.util.Map;
+
+public class HdfsFileSystemCommitter implements FileSystemCommitter {
+ @Override
+ public void commitTransaction(@NonNull FileAggregatedCommitInfo aggregateCommitInfo) throws IOException {
+ for (Map.Entry> entry : aggregateCommitInfo.getTransactionMap().entrySet()) {
+ for (Map.Entry mvFileEntry : entry.getValue().entrySet()) {
+ HdfsUtils.renameFile(mvFileEntry.getKey(), mvFileEntry.getValue(), true);
+ }
+ // delete the transaction dir
+ HdfsUtils.deleteFile(entry.getKey());
+ }
+ }
+
+ @Override
+ public void abortTransaction(@NonNull FileAggregatedCommitInfo aggregateCommitInfo) throws IOException {
+ for (Map.Entry> entry : aggregateCommitInfo.getTransactionMap().entrySet()) {
+ // rollback the file
+ for (Map.Entry 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());
+ }
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/HdfsTxtTransactionStateFileWriter.java b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/HdfsTxtTransactionStateFileWriter.java
new file mode 100644
index 00000000000..81882c414c6
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/HdfsTxtTransactionStateFileWriter.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.sink.hdfs;
+
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.FileSystem;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.transaction.TransactionFileNameGenerator;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.AbstractTransactionStateFileWriter;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.PartitionDirNameGenerator;
+
+import lombok.NonNull;
+import org.apache.hadoop.fs.FSDataOutputStream;
+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.stream.Collectors;
+
+public class HdfsTxtTransactionStateFileWriter extends AbstractTransactionStateFileWriter {
+ private static final Logger LOGGER = LoggerFactory.getLogger(HdfsTxtTransactionStateFileWriter.class);
+ private Map beingWrittenOutputStream;
+
+ private String fieldDelimiter;
+ private String rowDelimiter;
+
+ public HdfsTxtTransactionStateFileWriter(@NonNull SeaTunnelRowType seaTunnelRowTypeInfo,
+ @NonNull TransactionFileNameGenerator transactionFileNameGenerator,
+ @NonNull PartitionDirNameGenerator partitionDirNameGenerator,
+ @NonNull List sinkColumnsIndexInRow,
+ @NonNull String tmpPath,
+ @NonNull String targetPath,
+ @NonNull String jobId,
+ int subTaskIndex,
+ @NonNull String fieldDelimiter,
+ @NonNull String rowDelimiter,
+ @NonNull FileSystem fileSystem) {
+ super(seaTunnelRowTypeInfo, transactionFileNameGenerator, partitionDirNameGenerator, sinkColumnsIndexInRow, tmpPath, targetPath, jobId, subTaskIndex, fileSystem);
+
+ this.fieldDelimiter = fieldDelimiter;
+ this.rowDelimiter = rowDelimiter;
+ beingWrittenOutputStream = new HashMap<>();
+ }
+
+ @Override
+ public void beginTransaction(String transactionId) {
+ this.beingWrittenOutputStream = new HashMap<>();
+ }
+
+ @Override
+ public void abortTransaction(String transactionId) {
+ this.beingWrittenOutputStream = new HashMap<>();
+ }
+
+ @Override
+ public void write(@NonNull SeaTunnelRow seaTunnelRow) {
+ String filePath = getOrCreateFilePathBeingWritten(seaTunnelRow);
+ FSDataOutputStream fsDataOutputStream = getOrCreateOutputStream(filePath);
+ String line = transformRowToLine(seaTunnelRow);
+ try {
+ fsDataOutputStream.write(line.getBytes());
+ fsDataOutputStream.write(rowDelimiter.getBytes());
+ } catch (IOException e) {
+ LOGGER.error("write data to file {} error", filePath);
+ throw new RuntimeException(e);
+ }
+ }
+
+ @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(), getTargetLocation(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 this.sinkColumnsIndexInRow.stream().map(index -> seaTunnelRow.getFields()[index] == null ? "" : seaTunnelRow.getFields()[index].toString()).collect(Collectors.joining(fieldDelimiter));
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/HdfsUtils.java b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/HdfsUtils.java
new file mode 100644
index 00000000000..421c7f7ebbd
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/HdfsUtils.java
@@ -0,0 +1,137 @@
+/*
+ * 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.hdfs;
+
+import lombok.NonNull;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+
+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();
+ LOGGER.info(System.getenv("HADOOP_CONF_DIR"));
+ conf.addResource(new Path(System.getenv("HADOOP_CONF_DIR") + "/core-site.xml"));
+ conf.addResource(new Path(System.getenv("HADOOP_CONF_DIR") + "/hdfs-site.xml"));
+ conf.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem");
+ return FileSystem.get(URI.create(path), 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 void createFile(@NonNull String filePath) throws IOException {
+ FileSystem hdfsFs = getHdfsFs(filePath);
+ Path path = new Path(filePath);
+ if (!hdfsFs.createNewFile(path)) {
+ throw new IOException("create file " + filePath + " error");
+ }
+ }
+
+ public static void deleteFile(@NonNull String file) throws IOException {
+ FileSystem hdfsFs = getHdfsFs(file);
+ if (!hdfsFs.delete(new Path(file), true)) {
+ throw new IOException("delete file " + file + " error");
+ }
+ }
+
+ /**
+ * 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("/")));
+ }
+
+ if (hdfsFs.rename(oldPath, newPath)) {
+ LOGGER.info("rename file :[" + oldPath + "] to [" + newPath + "] finish");
+ } else {
+ throw new IOException("rename file :[" + oldPath + "] to [" + newPath + "] error");
+ }
+ }
+
+ public static void createDir(@NonNull String filePath)
+ throws IOException {
+
+ FileSystem hdfsFs = getHdfsFs(filePath);
+ Path dfs = new Path(filePath);
+ if (!hdfsFs.mkdirs(dfs)) {
+ throw new IOException("create dir " + filePath + " error");
+ }
+ }
+
+ public static boolean fileExist(@NonNull String filePath)
+ throws IOException {
+ FileSystem hdfsFs = getHdfsFs(filePath);
+ Path fileName = new Path(filePath);
+ return hdfsFs.exists(fileName);
+ }
+
+ /**
+ * get the dir in filePath
+ */
+ public static List dirList(@NonNull String filePath)
+ throws FileNotFoundException, IOException {
+ FileSystem hdfsFs = getHdfsFs(filePath);
+ List pathList = new ArrayList();
+ Path fileName = new Path(filePath);
+ FileStatus[] status = hdfsFs.listStatus(fileName);
+ if (status != null && status.length > 0) {
+ for (FileStatus fileStatus : status) {
+ if (fileStatus.isDirectory()) {
+ pathList.add(fileStatus.getPath());
+ }
+ }
+ }
+ return pathList;
+ }
+}
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
new file mode 100644
index 00000000000..79c54fcc9a5
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/FileSinkAggregatedCommitterTest.java
@@ -0,0 +1,122 @@
+/*
+ * 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.hdfs;
+
+import org.apache.seatunnel.connectors.seatunnel.file.sink.FileAggregatedCommitInfo;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.FileCommitInfo;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.FileSinkAggregatedCommitter;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+public class FileSinkAggregatedCommitterTest {
+ @Test
+ public void testCommit() throws Exception {
+ FileSinkAggregatedCommitter fileSinkAggregatedCommitter = new FileSinkAggregatedCommitter(new HdfsFileSystemCommitter());
+ Map> transactionFiles = new HashMap<>();
+ Random random = new Random();
+ Long jobId = random.nextLong();
+ String transactionDir = String.format("/tmp/seatunnel/seatunnel/%s/T_%s_0_1", jobId, jobId);
+ String targetDir = String.format("/tmp/hive/warehouse/%s", jobId);
+ Map 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");
+ 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);
+ List fileAggregatedCommitInfoList = new ArrayList<>();
+ fileAggregatedCommitInfoList.add(fileAggregatedCommitInfo);
+ fileSinkAggregatedCommitter.commit(fileAggregatedCommitInfoList);
+
+ Assert.assertTrue(HdfsUtils.fileExist(targetDir + "/c3=4/c4=bbb/test1.txt"));
+ Assert.assertTrue(HdfsUtils.fileExist(targetDir + "/c3=4/c4=rrr/test1.txt"));
+ Assert.assertTrue(!HdfsUtils.fileExist(transactionDir));
+ }
+
+ @SuppressWarnings("checkstyle:MagicNumber")
+ @Test
+ public void testCombine() throws Exception {
+ FileSinkAggregatedCommitter fileSinkAggregatedCommitter = new FileSinkAggregatedCommitter(new HdfsFileSystemCommitter());
+ Map> transactionFiles = new HashMap<>();
+ Random random = new Random();
+ Long jobId = random.nextLong();
+ String transactionDir = String.format("/tmp/seatunnel/seatunnel/%s/T_%s_0_1", jobId, jobId);
+ String targetDir = String.format("/tmp/hive/warehouse/%s", jobId);
+ Map 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");
+ HdfsUtils.createFile(transactionDir + "/c3=4/c4=rrr/test1.txt");
+ HdfsUtils.createFile(transactionDir + "/c3=4/c4=bbb/test1.txt");
+
+ Map 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);
+ List 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=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"));
+ }
+
+ @Test
+ public void testAbort() throws Exception {
+ FileSinkAggregatedCommitter fileSinkAggregatedCommitter = new FileSinkAggregatedCommitter(new HdfsFileSystemCommitter());
+ Map> transactionFiles = new HashMap<>();
+ Random random = new Random();
+ Long jobId = random.nextLong();
+ String transactionDir = String.format("/tmp/seatunnel/seatunnel/%s/T_%s_0_1", jobId, jobId);
+ String targetDir = String.format("/tmp/hive/warehouse/%s", jobId);
+ Map 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");
+ 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);
+ List fileAggregatedCommitInfoList = new ArrayList<>();
+ fileAggregatedCommitInfoList.add(fileAggregatedCommitInfo);
+ fileSinkAggregatedCommitter.commit(fileAggregatedCommitInfoList);
+
+ Assert.assertTrue(HdfsUtils.fileExist(targetDir + "/c3=4/c4=bbb/test1.txt"));
+ Assert.assertTrue(HdfsUtils.fileExist(targetDir + "/c3=4/c4=rrr/test1.txt"));
+ Assert.assertTrue(!HdfsUtils.fileExist(transactionDir));
+
+ fileSinkAggregatedCommitter.abort(fileAggregatedCommitInfoList);
+ Assert.assertTrue(!HdfsUtils.fileExist(targetDir + "/c3=4/c4=bbb/test1.txt"));
+ Assert.assertTrue(!HdfsUtils.fileExist(targetDir + "/c3=4/c4=rrr/test1.txt"));
+
+ // transactionDir will being delete when abort
+ Assert.assertTrue(!HdfsUtils.fileExist(transactionDir));
+ }
+}
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
new file mode 100644
index 00000000000..f3e1847f431
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/TestHdfsTxtTransactionStateFileWriter.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.sink.hdfs;
+
+import org.apache.seatunnel.api.table.type.BasicType;
+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.connectors.seatunnel.file.config.FileFormat;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.FileCommitInfo;
+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.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+@RunWith(JUnit4.class)
+public class TestHdfsTxtTransactionStateFileWriter {
+
+ @SuppressWarnings("checkstyle:MagicNumber")
+ @Test
+ public void testHdfsTextTransactionStateFileWriter() throws Exception {
+ String[] fieldNames = new String[]{"c1", "c2", "c3", "c4"};
+ SeaTunnelDataType[] seaTunnelDataTypes = new SeaTunnelDataType[]{BasicType.BOOLEAN_TYPE, BasicType.INT_TYPE, BasicType.STRING_TYPE, BasicType.INT_TYPE};
+ SeaTunnelRowType seaTunnelRowTypeInfo = new SeaTunnelRowType(fieldNames, seaTunnelDataTypes);
+
+ List sinkColumnIndexInRow = new ArrayList<>();
+ sinkColumnIndexInRow.add(0);
+ sinkColumnIndexInRow.add(1);
+
+ List hivePartitionFieldList = new ArrayList<>();
+ hivePartitionFieldList.add("c3");
+ hivePartitionFieldList.add("c4");
+
+ List partitionFieldIndexInRow = new ArrayList<>();
+ partitionFieldIndexInRow.add(2);
+ partitionFieldIndexInRow.add(3);
+
+ String jobId = System.currentTimeMillis() + "";
+ String targetPath = "/tmp/hive/warehouse/seatunnel.db/test1";
+ String tmpPath = "/tmp/seatunnel";
+
+ TransactionStateFileWriter fileWriter = new HdfsTxtTransactionStateFileWriter(seaTunnelRowTypeInfo,
+ new FileSinkTransactionFileNameGenerator(FileFormat.TEXT, null, "yyyy.MM.dd"),
+ new FileSinkPartitionDirNameGenerator(hivePartitionFieldList, partitionFieldIndexInRow, "${k0}=${v0}/${k1}=${v1}"),
+ sinkColumnIndexInRow,
+ tmpPath,
+ targetPath,
+ jobId,
+ 0,
+ String.valueOf('\001'),
+ "\n",
+ new HdfsFileSystem());
+
+ String transactionId = fileWriter.beginTransaction(1L);
+
+ SeaTunnelRow seaTunnelRow = new SeaTunnelRow(new Object[]{true, 1, "str1", "str2"});
+ fileWriter.write(seaTunnelRow);
+
+ SeaTunnelRow seaTunnelRow1 = new SeaTunnelRow(new Object[]{true, 1, "str1", "str3"});
+ fileWriter.write(seaTunnelRow1);
+
+ Optional fileCommitInfoOptional = fileWriter.prepareCommit();
+ //check file exists and file content
+ Assert.assertTrue(fileCommitInfoOptional.isPresent());
+ FileCommitInfo fileCommitInfo = fileCommitInfoOptional.get();
+ String transactionDir = tmpPath + "/seatunnel/" + jobId + "/" + transactionId;
+ Assert.assertEquals(transactionDir, fileCommitInfo.getTransactionDir());
+ Assert.assertEquals(2, fileCommitInfo.getNeedMoveFiles().size());
+ Map 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"));
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-local/pom.xml b/seatunnel-connectors-v2/connector-file/connector-file-local/pom.xml
new file mode 100644
index 00000000000..1ac5bb77b55
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-local/pom.xml
@@ -0,0 +1,54 @@
+
+
+
+
+ connector-file
+ org.apache.seatunnel
+ ${revision}
+
+ 4.0.0
+
+ connector-file-local
+
+
+
+ org.apache.seatunnel
+ connector-file-base
+ ${project.version}
+
+
+ junit
+ junit
+ test
+
+
+ org.powermock
+ powermock-module-junit4
+ test
+
+
+ org.powermock
+ powermock-api-mockito2
+ test
+
+
+
\ No newline at end of file
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/local/FileUtils.java b/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/local/FileUtils.java
new file mode 100644
index 00000000000..b951ff8eab9
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/local/FileUtils.java
@@ -0,0 +1,104 @@
+/*
+ * 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.local;
+
+import lombok.NonNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+
+public class FileUtils {
+ private static final Logger LOGGER = LoggerFactory.getLogger(FileUtils.class);
+ public static File createDir(@NonNull String dirPath) {
+ if (dirPath == null || "".equals(dirPath)) {
+ return null;
+ }
+ File file = new File(dirPath);
+ if (!file.exists() || !file.isDirectory()) {
+ file.mkdirs();
+ }
+ return file;
+ }
+
+ public static File createFile(@NonNull String filePath) throws IOException {
+ if (filePath == null || "".equals(filePath)) {
+ return null;
+ }
+ File file = new File(filePath);
+ if (!file.getParentFile().exists()) {
+ file.getParentFile().mkdirs();
+ }
+
+ if (!file.exists() || !file.isFile()) {
+ file.createNewFile();
+ }
+ return file;
+ }
+
+ public static boolean fileExist(@NonNull String filePath) {
+ File file = new File(filePath);
+ return file.exists();
+ }
+
+ public static void renameFile(@NonNull String oldName, @NonNull String newName) throws IOException {
+ LOGGER.info("begin rename file oldName :[" + oldName + "] to newName :[" + newName + "]");
+ File oldPath = new File(oldName);
+ File newPath = new File(newName);
+
+ if (!newPath.getParentFile().exists()) {
+ newPath.getParentFile().mkdirs();
+ }
+
+ if (oldPath.renameTo(newPath)) {
+ LOGGER.info("rename file :[" + oldPath + "] to [" + newPath + "] finish");
+ } else {
+ throw new IOException("rename file :[" + oldPath + "] to [" + newPath + "] error");
+ }
+ }
+
+ public static void deleteFile(@NonNull String filePath) throws IOException {
+ File file = new File(filePath);
+ if (file.exists()) {
+ if (file.isDirectory()) {
+ deleteFiles(file);
+ }
+ file.delete();
+ }
+ }
+
+ private static boolean deleteFiles(@NonNull File file) {
+ try {
+ File[] files = file.listFiles();
+ for (int i = 0; i < files.length; i++) {
+ File thisFile = files[i];
+ if (thisFile.isDirectory()) {
+ deleteFiles(thisFile);
+ }
+ thisFile.delete();
+ }
+ file.delete();
+
+ } catch (Exception e) {
+ LOGGER.error("delete file [" + file.getPath() + "] error");
+ return false;
+ }
+ return true;
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/local/LocalFileSink.java b/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/local/LocalFileSink.java
new file mode 100644
index 00000000000..6e4b503e946
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/local/LocalFileSink.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.sink.local;
+
+import org.apache.seatunnel.api.sink.SeaTunnelSink;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.AbstractFileSink;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.SinkFileSystemPlugin;
+
+import com.google.auto.service.AutoService;
+
+@AutoService(SeaTunnelSink.class)
+public class LocalFileSink extends AbstractFileSink {
+ @Override
+ public SinkFileSystemPlugin getSinkFileSystemPlugin() {
+ return new LocalFileSinkPlugin();
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/local/LocalFileSinkPlugin.java b/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/local/LocalFileSinkPlugin.java
new file mode 100644
index 00000000000..1d4bc43e57f
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/local/LocalFileSinkPlugin.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.connectors.seatunnel.file.sink.local;
+
+import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.config.FileSystemType;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.FileSystem;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.FileSystemCommitter;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.SinkFileSystemPlugin;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.transaction.TransactionFileNameGenerator;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.transaction.TransactionStateFileWriter;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.PartitionDirNameGenerator;
+
+import lombok.NonNull;
+
+import java.util.List;
+import java.util.Optional;
+
+public class LocalFileSinkPlugin implements SinkFileSystemPlugin {
+ @Override
+ public String getPluginName() {
+ return FileSystemType.LOCAL.getSinkFileSystemPluginName();
+ }
+
+ @Override
+ public Optional getTransactionStateFileWriter(@NonNull SeaTunnelRowType seaTunnelRowTypeInfo,
+ @NonNull TransactionFileNameGenerator transactionFileNameGenerator,
+ @NonNull PartitionDirNameGenerator partitionDirNameGenerator,
+ @NonNull List sinkColumnsIndexInRow,
+ @NonNull String tmpPath,
+ @NonNull String targetPath,
+ @NonNull String jobId,
+ int subTaskIndex,
+ @NonNull String fieldDelimiter,
+ @NonNull String rowDelimiter,
+ @NonNull FileSystem fileSystem) {
+ return Optional.of(new LocalTxtTransactionStateFileWriter(seaTunnelRowTypeInfo,
+ transactionFileNameGenerator,
+ partitionDirNameGenerator,
+ sinkColumnsIndexInRow,
+ tmpPath,
+ targetPath,
+ jobId,
+ subTaskIndex,
+ fieldDelimiter,
+ rowDelimiter,
+ fileSystem));
+ }
+
+ @Override
+ public Optional getFileSystemCommitter() {
+ return Optional.of(new LocalFileSystemCommitter());
+ }
+
+ @Override
+ public Optional getFileSystem() {
+ return Optional.of(new LocalFileSystem());
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/local/LocalFileSystem.java b/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/local/LocalFileSystem.java
new file mode 100644
index 00000000000..6f68c2305fe
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/local/LocalFileSystem.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.sink.local;
+
+import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.FileSystem;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
+public class LocalFileSystem implements FileSystem {
+ @Override
+ public void deleteFile(String path) throws IOException {
+ File file = new File(path);
+ file.delete();
+ }
+
+ @Override
+ public List dirList(String dirPath) throws IOException {
+ File file = new File(dirPath);
+ String[] list = file.list();
+ if (list == null) {
+ return null;
+ }
+ return Arrays.asList(list);
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/local/LocalFileSystemCommitter.java b/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/local/LocalFileSystemCommitter.java
new file mode 100644
index 00000000000..38e1d06d623
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/local/LocalFileSystemCommitter.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.sink.local;
+
+import org.apache.seatunnel.connectors.seatunnel.file.sink.FileAggregatedCommitInfo;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.FileSystemCommitter;
+
+import lombok.NonNull;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Map;
+
+public class LocalFileSystemCommitter implements FileSystemCommitter {
+ @Override
+ public void commitTransaction(@NonNull FileAggregatedCommitInfo aggregateCommitInfo) throws IOException {
+ for (Map.Entry> entry : aggregateCommitInfo.getTransactionMap().entrySet()) {
+ for (Map.Entry mvFileEntry : entry.getValue().entrySet()) {
+ FileUtils.renameFile(mvFileEntry.getKey(), mvFileEntry.getValue());
+ }
+ // delete the transaction dir
+ FileUtils.deleteFile(entry.getKey());
+ }
+ }
+
+ @Override
+ public void abortTransaction(@NonNull FileAggregatedCommitInfo aggregateCommitInfo) throws IOException {
+ for (Map.Entry> entry : aggregateCommitInfo.getTransactionMap().entrySet()) {
+ // rollback the file
+ for (Map.Entry mvFileEntry : entry.getValue().entrySet()) {
+ File oldFile = new File(mvFileEntry.getKey());
+ File newFile = new File(mvFileEntry.getValue());
+ if (newFile.exists() && !oldFile.exists()) {
+ FileUtils.renameFile(mvFileEntry.getValue(), mvFileEntry.getKey());
+ }
+ }
+ // delete the transaction dir
+ FileUtils.deleteFile(entry.getKey());
+ }
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/local/LocalTxtTransactionStateFileWriter.java b/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/local/LocalTxtTransactionStateFileWriter.java
new file mode 100644
index 00000000000..d04939a7049
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/local/LocalTxtTransactionStateFileWriter.java
@@ -0,0 +1,126 @@
+/*
+ * 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.local;
+
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.FileSystem;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.transaction.TransactionFileNameGenerator;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.AbstractTransactionStateFileWriter;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.PartitionDirNameGenerator;
+
+import lombok.NonNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class LocalTxtTransactionStateFileWriter extends AbstractTransactionStateFileWriter {
+ private static final Logger LOGGER = LoggerFactory.getLogger(LocalTxtTransactionStateFileWriter.class);
+ private Map beingWrittenOutputStream;
+
+ private String fieldDelimiter;
+ private String rowDelimiter;
+
+ public LocalTxtTransactionStateFileWriter(@NonNull SeaTunnelRowType seaTunnelRowTypeInfo,
+ @NonNull TransactionFileNameGenerator transactionFileNameGenerator,
+ @NonNull PartitionDirNameGenerator partitionDirNameGenerator,
+ @NonNull List sinkColumnsIndexInRow,
+ @NonNull String tmpPath,
+ @NonNull String targetPath,
+ @NonNull String jobId,
+ int subTaskIndex,
+ @NonNull String fieldDelimiter,
+ @NonNull String rowDelimiter,
+ @NonNull FileSystem fileSystem) {
+ super(seaTunnelRowTypeInfo, transactionFileNameGenerator, partitionDirNameGenerator, sinkColumnsIndexInRow, tmpPath, targetPath, jobId, subTaskIndex, fileSystem);
+
+ this.fieldDelimiter = fieldDelimiter;
+ this.rowDelimiter = rowDelimiter;
+ beingWrittenOutputStream = new HashMap<>();
+ }
+
+ @Override
+ public void beginTransaction(String transactionId) {
+ this.beingWrittenOutputStream = new HashMap<>();
+ }
+
+ @Override
+ public void abortTransaction(String transactionId) {
+ this.beingWrittenOutputStream = new HashMap<>();
+ }
+
+ @Override
+ public void write(@NonNull SeaTunnelRow seaTunnelRow) {
+ String filePath = getOrCreateFilePathBeingWritten(seaTunnelRow);
+ FileOutputStream fileOutputStream = getOrCreateOutputStream(filePath);
+ String line = transformRowToLine(seaTunnelRow);
+ try {
+ fileOutputStream.write(line.getBytes());
+ fileOutputStream.write(rowDelimiter.getBytes());
+ } catch (IOException e) {
+ LOGGER.error("write data to file {} error", filePath);
+ throw new RuntimeException(e);
+ }
+ }
+
+ @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(), getTargetLocation(entry.getKey()));
+ });
+ }
+
+ private FileOutputStream getOrCreateOutputStream(@NonNull String filePath) {
+ FileOutputStream fileOutputStream = beingWrittenOutputStream.get(filePath);
+ if (fileOutputStream == null) {
+ try {
+ FileUtils.createFile(filePath);
+ fileOutputStream = new FileOutputStream(new File(filePath));
+ beingWrittenOutputStream.put(filePath, fileOutputStream);
+ } catch (IOException e) {
+ LOGGER.error("can not get output file stream");
+ throw new RuntimeException(e);
+ }
+ }
+ return fileOutputStream;
+ }
+
+ private String transformRowToLine(@NonNull SeaTunnelRow seaTunnelRow) {
+ return this.sinkColumnsIndexInRow.stream().map(index -> seaTunnelRow.getFields()[index] == null ? "" : seaTunnelRow.getFields()[index].toString()).collect(Collectors.joining(fieldDelimiter));
+ }
+}
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
new file mode 100644
index 00000000000..89524aa5fdd
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-local/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/sink/local/FileSinkAggregatedCommitterTest.java
@@ -0,0 +1,125 @@
+/*
+ * 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.local;
+
+import org.apache.seatunnel.connectors.seatunnel.file.sink.FileAggregatedCommitInfo;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.FileCommitInfo;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.FileSinkAggregatedCommitter;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+public class FileSinkAggregatedCommitterTest {
+ @Test
+ public void testCommit() throws Exception {
+ FileSinkAggregatedCommitter fileSinkAggregatedCommitter = new FileSinkAggregatedCommitter(new LocalFileSystemCommitter());
+ Map> transactionFiles = new HashMap<>();
+ Random random = new Random();
+ Long jobIdLong = random.nextLong();
+ String jobId = "Job_" + jobIdLong;
+ String transactionDir = String.format("/tmp/seatunnel/seatunnel/%s/T_%s_0_1", jobId, jobId);
+ String targetDir = String.format("/tmp/hive/warehouse/%s", jobId);
+ Map 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");
+ 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);
+ List 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.assertTrue(!FileUtils.fileExist(transactionDir));
+ }
+
+ @SuppressWarnings("checkstyle:MagicNumber")
+ @Test
+ public void testCombine() throws Exception {
+ FileSinkAggregatedCommitter fileSinkAggregatedCommitter = new FileSinkAggregatedCommitter(new LocalFileSystemCommitter());
+ Map> transactionFiles = new HashMap<>();
+ Random random = new Random();
+ Long jobIdLong = random.nextLong();
+ String jobId = "Job_" + jobIdLong;
+ String transactionDir = String.format("/tmp/seatunnel/seatunnel/%s/T_%s_0_1", jobId, jobId);
+ String targetDir = String.format("/tmp/hive/warehouse/%s", jobId);
+ Map 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");
+ FileUtils.createFile(transactionDir + "/c3=4/c4=rrr/test1.txt");
+ FileUtils.createFile(transactionDir + "/c3=4/c4=bbb/test1.txt");
+
+ Map 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);
+ List 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=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"));
+ }
+
+ @Test
+ public void testAbort() throws Exception {
+ FileSinkAggregatedCommitter fileSinkAggregatedCommitter = new FileSinkAggregatedCommitter(new LocalFileSystemCommitter());
+ Map> transactionFiles = new HashMap<>();
+ Random random = new Random();
+ Long jobIdLong = random.nextLong();
+ String jobId = "Job_" + jobIdLong;
+ String transactionDir = String.format("/tmp/seatunnel/seatunnel/%s/T_%s_0_1", jobId, jobId);
+ String targetDir = String.format("/tmp/hive/warehouse/%s", jobId);
+ Map 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");
+ 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);
+ List 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));
+
+ fileSinkAggregatedCommitter.abort(fileAggregatedCommitInfoList);
+ Assert.assertTrue(!FileUtils.fileExist(targetDir + "/c3=4/c4=bbb/test1.txt"));
+ Assert.assertTrue(!FileUtils.fileExist(targetDir + "/c3=4/c4=rrr/test1.txt"));
+
+ // transactionDir will being delete when abort
+ Assert.assertTrue(!FileUtils.fileExist(transactionDir));
+ }
+}
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
new file mode 100644
index 00000000000..007ea39ef9d
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-local/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/sink/local/TestLocalTxtTransactionStateFileWriter.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.sink.local;
+
+import org.apache.seatunnel.api.table.type.BasicType;
+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.connectors.seatunnel.file.config.FileFormat;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.FileCommitInfo;
+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.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+@RunWith(JUnit4.class)
+public class TestLocalTxtTransactionStateFileWriter {
+
+ @SuppressWarnings("checkstyle:MagicNumber")
+ @Test
+ public void testHdfsTextTransactionStateFileWriter() throws Exception {
+ String[] fieldNames = new String[]{"c1", "c2", "c3", "c4"};
+ SeaTunnelDataType[] seaTunnelDataTypes = new SeaTunnelDataType[]{BasicType.BOOLEAN_TYPE, BasicType.INT_TYPE, BasicType.STRING_TYPE, BasicType.INT_TYPE};
+ SeaTunnelRowType seaTunnelRowTypeInfo = new SeaTunnelRowType(fieldNames, seaTunnelDataTypes);
+
+ List sinkColumnIndexInRow = new ArrayList<>();
+ sinkColumnIndexInRow.add(0);
+ sinkColumnIndexInRow.add(1);
+
+ List hivePartitionFieldList = new ArrayList<>();
+ hivePartitionFieldList.add("c3");
+ hivePartitionFieldList.add("c4");
+
+ List partitionFieldIndexInRow = new ArrayList<>();
+ partitionFieldIndexInRow.add(2);
+ partitionFieldIndexInRow.add(3);
+
+ String jobId = System.currentTimeMillis() + "";
+ String targetPath = "/tmp/hive/warehouse/seatunnel.db/test1";
+ String tmpPath = "/tmp/seatunnel";
+
+ TransactionStateFileWriter fileWriter = new LocalTxtTransactionStateFileWriter(seaTunnelRowTypeInfo,
+ new FileSinkTransactionFileNameGenerator(FileFormat.TEXT, null, "yyyy.MM.dd"),
+ new FileSinkPartitionDirNameGenerator(hivePartitionFieldList, partitionFieldIndexInRow, "${k0}=${v0}/${k1}=${v1}"),
+ sinkColumnIndexInRow,
+ tmpPath,
+ targetPath,
+ jobId,
+ 0,
+ String.valueOf('\001'),
+ "\n",
+ new LocalFileSystem());
+
+ String transactionId = fileWriter.beginTransaction(1L);
+
+ SeaTunnelRow seaTunnelRow = new SeaTunnelRow(new Object[]{true, 1, "str1", "str2"});
+ fileWriter.write(seaTunnelRow);
+
+ SeaTunnelRow seaTunnelRow1 = new SeaTunnelRow(new Object[]{true, 1, "str1", "str3"});
+ fileWriter.write(seaTunnelRow1);
+
+ Optional fileCommitInfoOptional = fileWriter.prepareCommit();
+ //check file exists and file content
+ Assert.assertTrue(fileCommitInfoOptional.isPresent());
+ FileCommitInfo fileCommitInfo = fileCommitInfoOptional.get();
+ String transactionDir = tmpPath + "/seatunnel/" + jobId + "/" + transactionId;
+ Assert.assertEquals(transactionDir, fileCommitInfo.getTransactionDir());
+ Assert.assertEquals(2, fileCommitInfo.getNeedMoveFiles().size());
+ Map 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"));
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-file/pom.xml b/seatunnel-connectors-v2/connector-file/pom.xml
new file mode 100644
index 00000000000..b84010b8aa8
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/pom.xml
@@ -0,0 +1,37 @@
+
+
+
+
+ seatunnel-connectors-v2
+ org.apache.seatunnel
+ ${revision}
+
+ 4.0.0
+ connector-file
+ pom
+
+
+ connector-file-base
+ connector-file-hadoop
+ connector-file-local
+
+
\ No newline at end of file
diff --git a/seatunnel-connectors-v2/connector-hive/pom.xml b/seatunnel-connectors-v2/connector-hive/pom.xml
index db75d801eca..62a2eed1f00 100644
--- a/seatunnel-connectors-v2/connector-hive/pom.xml
+++ b/seatunnel-connectors-v2/connector-hive/pom.xml
@@ -49,6 +49,13 @@
provided
+
+ org.apache.seatunnel
+ seatunnel-core-base
+ ${project.version}
+ test
+
+
org.apache.commons
commons-lang3
diff --git a/seatunnel-connectors-v2/pom.xml b/seatunnel-connectors-v2/pom.xml
index be62464f81b..01431229a3e 100644
--- a/seatunnel-connectors-v2/pom.xml
+++ b/seatunnel-connectors-v2/pom.xml
@@ -28,7 +28,6 @@
4.0.0
pom
-
seatunnel-connectors-v2
@@ -36,12 +35,13 @@
connector-clickhouse
connector-console
connector-fake
- connector-hive
connector-http
connector-jdbc
connector-kafka
connector-pulsar
connector-socket
+ connector-hive
+ connector-file
connector-assert
diff --git a/seatunnel-dist/release-docs/LICENSE b/seatunnel-dist/release-docs/LICENSE
index 3a24746fb1c..071e3d2b287 100644
--- a/seatunnel-dist/release-docs/LICENSE
+++ b/seatunnel-dist/release-docs/LICENSE
@@ -312,8 +312,6 @@ The text of each license is the standard Apache 2.0 license.
(Apache License 2.0) Compress-LZF (com.ning:compress-lzf:1.0.4 - http://github.com/ning/compress)
(Apache License 2.0) FRocksDB JNI (com.ververica:frocksdbjni:5.17.2-ververica-2.1 - https://github.com/ververica/frocksdb)
(Apache License 2.0) Graphite Integration for Metrics (io.dropwizard.metrics:metrics-graphite:3.1.5 - http://metrics.codahale.com/metrics-graphite/)
- (Apache License 2.0) JVM Integration for Metrics (io.dropwizard.metrics:metrics-jvm:3.1.5 - http://metrics.codahale.com/metrics-jvm/)
- (Apache License 2.0) Jackson Integration for Metrics (io.dropwizard.metrics:metrics-json:3.1.5 - http://metrics.codahale.com/metrics-json/)
(Apache License 2.0) Metrics Core (io.dropwizard.metrics:metrics-core:3.1.5 - http://metrics.codahale.com/metrics-core/)
(Apache License 2.0) Metrics Core (io.dropwizard.metrics:metrics-core:3.2.1 - http://metrics.codahale.com/metrics-core/)
(Apache License 2.0) Metrics Core (io.dropwizard.metrics:metrics-core:4.0.0 - http://metrics.dropwizard.io/metrics-core)
@@ -348,7 +346,6 @@ The text of each license is the standard Apache 2.0 license.
(Apache License, Version 2.0) Apache Commons BeanUtils (commons-beanutils:commons-beanutils:1.9.4 - https://commons.apache.org/proper/commons-beanutils/)
(Apache License, Version 2.0) Apache Commons CLI (commons-cli:commons-cli:1.4 - http://commons.apache.org/proper/commons-cli/)
(Apache License, Version 2.0) Apache Commons Codec (commons-codec:commons-codec:1.13 - https://commons.apache.org/proper/commons-codec/)
- (Apache License, Version 2.0) Apache Commons Collections (commons-collections:commons-collections:3.2.2 - http://commons.apache.org/collections/)
(Apache License, Version 2.0) Apache Commons Collections (org.apache.commons:commons-collections4:4.4 - https://commons.apache.org/proper/commons-collections/)
(Apache License, Version 2.0) Apache Commons Compress (org.apache.commons:commons-compress:1.18 - https://commons.apache.org/proper/commons-compress/)
(Apache License, Version 2.0) Apache Commons Compress (org.apache.commons:commons-compress:1.20 - https://commons.apache.org/proper/commons-compress/)
@@ -397,22 +394,35 @@ The text of each license is the standard Apache 2.0 license.
(Apache License, Version 2.0) Apache HBase - Zookeeper (org.apache.hbase:hbase-zookeeper:2.1.0 - http://hbase.apache.org/hbase-build-configuration/hbase-zookeeper)
(Apache License, Version 2.0) Apache Hadoop Annotations (org.apache.hadoop:hadoop-annotations:2.6.5 - no url defined)
(Apache License, Version 2.0) Apache Hadoop Annotations (org.apache.hadoop:hadoop-annotations:3.0.0 - no url defined)
- (Apache License, Version 2.0) Apache Hadoop Auth (org.apache.hadoop:hadoop-auth:2.7.2 - no url defined)
+ (Apache License, Version 2.0) Apache Hadoop Auth (org.apache.hadoop:hadoop-auth:2.6.5 - no url defined)
(Apache License, Version 2.0) Apache Hadoop Auth (org.apache.hadoop:hadoop-auth:2.7.4 - no url defined)
(Apache License, Version 2.0) Apache Hadoop Auth (org.apache.hadoop:hadoop-auth:3.0.0 - no url defined)
- (Apache License, Version 2.0) Apache Hadoop Client Aggregator (org.apache.hadoop:hadoop-client:3.0.0 - no url defined)
+ (Apache License, Version 2.0) Apache Hadoop Client (org.apache.hadoop:hadoop-client:2.6.5 - no url defined)
+ (Apache License, Version 2.0) Apache Hadoop Client (org.apache.hadoop:hadoop-client:3.0.0 - no url defined)
+ (Apache License, Version 2.0) Apache Hadoop Common (org.apache.hadoop:hadoop-common:2.6.5 - no url defined)
(Apache License, Version 2.0) Apache Hadoop Common (org.apache.hadoop:hadoop-common:2.7.7 - no url defined)
(Apache License, Version 2.0) Apache Hadoop Common (org.apache.hadoop:hadoop-common:3.0.0 - no url defined)
(Apache License, Version 2.0) Apache Hadoop Distributed Copy (org.apache.hadoop:hadoop-distcp:2.7.4 - no url defined)
- (Apache License, Version 2.0) Apache Hadoop HDFS (org.apache.hadoop:hadoop-hdfs:2.7.2 - no url defined)
+ (Apache License, Version 2.0) Apache Hadoop HDFS (org.apache.hadoop:hadoop-hdfs:2.6.5 - no url defined)
(Apache License, Version 2.0) Apache Hadoop HDFS (org.apache.hadoop:hadoop-hdfs:2.7.4 - no url defined)
(Apache License, Version 2.0) Apache Hadoop HDFS Client (org.apache.hadoop:hadoop-hdfs-client:3.0.0 - no url defined)
- (Apache License, Version 2.0) Apache Hadoop MapReduce Common (org.apache.hadoop:hadoop-mapreduce-client-common:3.0.0 - no url defined)
- (Apache License, Version 2.0) Apache Hadoop MapReduce Core (org.apache.hadoop:hadoop-mapreduce-client-core:3.0.0 - no url defined)
- (Apache License, Version 2.0) Apache Hadoop MapReduce JobClient (org.apache.hadoop:hadoop-mapreduce-client-jobclient:3.0.0 - no url defined)
- (Apache License, Version 2.0) Apache Hadoop YARN API (org.apache.hadoop:hadoop-yarn-api:3.0.0 - no url defined)
- (Apache License, Version 2.0) Apache Hadoop YARN Client (org.apache.hadoop:hadoop-yarn-client:3.0.0 - no url defined)
- (Apache License, Version 2.0) Apache Hadoop YARN Common (org.apache.hadoop:hadoop-yarn-common:3.0.0 - no url defined)
+ (Apache License, Version 2.0) hadoop-mapreduce-client-app (org.apache.hadoop:hadoop-mapreduce-client-app:2.6.5 - no url defined)
+ (Apache License, Version 2.0) hadoop-mapreduce-client-common (org.apache.hadoop:hadoop-mapreduce-client-common:2.6.5 - no url defined)
+ (Apache License, Version 2.0) hadoop-mapreduce-client-common (org.apache.hadoop:hadoop-mapreduce-client-common:3.0.0 - no url defined)
+ (Apache License, Version 2.0) hadoop-mapreduce-client-core (org.apache.hadoop:hadoop-mapreduce-client-core:2.6.5 - no url defined)
+ (Apache License, Version 2.0) hadoop-mapreduce-client-core (org.apache.hadoop:hadoop-mapreduce-client-core:2.7.7 - no url defined)
+ (Apache License, Version 2.0) hadoop-mapreduce-client-core (org.apache.hadoop:hadoop-mapreduce-client-core:3.0.0 - no url defined)
+ (Apache License, Version 2.0) hadoop-mapreduce-client-jobclient (org.apache.hadoop:hadoop-mapreduce-client-jobclient:2.6.5 - no url defined)
+ (Apache License, Version 2.0) hadoop-mapreduce-client-jobclient (org.apache.hadoop:hadoop-mapreduce-client-jobclient:3.0.0 - no url defined)
+ (Apache License, Version 2.0) hadoop-mapreduce-client-shuffle (org.apache.hadoop:hadoop-mapreduce-client-shuffle:2.6.5 - no url defined)
+ (Apache License, Version 2.0) hadoop-yarn-api (org.apache.hadoop:hadoop-yarn-api:2.6.5 - no url defined)
+ (Apache License, Version 2.0) hadoop-yarn-api (org.apache.hadoop:hadoop-yarn-api:3.0.0 - no url defined)
+ (Apache License, Version 2.0) hadoop-yarn-client (org.apache.hadoop:hadoop-yarn-client:2.6.5 - no url defined)
+ (Apache License, Version 2.0) hadoop-yarn-client (org.apache.hadoop:hadoop-yarn-client:3.0.0 - no url defined)
+ (Apache License, Version 2.0) hadoop-yarn-common (org.apache.hadoop:hadoop-yarn-common:2.6.5 - no url defined)
+ (Apache License, Version 2.0) hadoop-yarn-common (org.apache.hadoop:hadoop-yarn-common:2.7.7 - no url defined)
+ (Apache License, Version 2.0) hadoop-yarn-common (org.apache.hadoop:hadoop-yarn-common:3.0.0 - no url defined)
+ (Apache License, Version 2.0) hadoop-yarn-server-common (org.apache.hadoop:hadoop-yarn-server-common:2.6.5 - no url defined)
(Apache License, Version 2.0) Apache HttpAsyncClient (org.apache.httpcomponents:httpasyncclient:4.1.2 - http://hc.apache.org/httpcomponents-asyncclient)
(Apache License, Version 2.0) Apache HttpAsyncClient (org.apache.httpcomponents:httpasyncclient:4.1.4 - http://hc.apache.org/httpcomponents-asyncclient)
(Apache License, Version 2.0) Apache HttpClient (org.apache.httpcomponents:httpclient:4.5.10 - http://hc.apache.org/httpcomponents-client)
@@ -561,23 +571,7 @@ The text of each license is the standard Apache 2.0 license.
(Apache License, Version 2.0) fastutil (it.unimi.dsi:fastutil:6.5.6 - http://fasutil.dsi.unimi.it/)
(Apache License, Version 2.0) fastutil (it.unimi.dsi:fastutil:7.0.13 - http://fasutil.di.unimi.it/)
(Apache License, Version 2.0) fastutil (it.unimi.dsi:fastutil:8.5.4 - http://fastutil.di.unimi.it/)
- (Apache License, Version 2.0) hadoop-mapreduce-client-app (org.apache.hadoop:hadoop-mapreduce-client-app:2.6.5 - no url defined)
- (Apache License, Version 2.0) hadoop-mapreduce-client-common (org.apache.hadoop:hadoop-mapreduce-client-common:2.6.5 - no url defined)
- (Apache License, Version 2.0) hadoop-mapreduce-client-common (org.apache.hadoop:hadoop-mapreduce-client-common:3.0.0 - no url defined)
- (Apache License, Version 2.0) hadoop-mapreduce-client-core (org.apache.hadoop:hadoop-mapreduce-client-core:2.6.5 - no url defined)
- (Apache License, Version 2.0) hadoop-mapreduce-client-core (org.apache.hadoop:hadoop-mapreduce-client-core:2.7.7 - no url defined)
- (Apache License, Version 2.0) hadoop-mapreduce-client-core (org.apache.hadoop:hadoop-mapreduce-client-core:3.0.0 - no url defined)
- (Apache License, Version 2.0) hadoop-mapreduce-client-jobclient (org.apache.hadoop:hadoop-mapreduce-client-jobclient:2.6.5 - no url defined)
- (Apache License, Version 2.0) hadoop-mapreduce-client-jobclient (org.apache.hadoop:hadoop-mapreduce-client-jobclient:3.0.0 - no url defined)
- (Apache License, Version 2.0) hadoop-mapreduce-client-shuffle (org.apache.hadoop:hadoop-mapreduce-client-shuffle:2.6.5 - no url defined)
- (Apache License, Version 2.0) hadoop-yarn-api (org.apache.hadoop:hadoop-yarn-api:2.6.5 - no url defined)
- (Apache License, Version 2.0) hadoop-yarn-api (org.apache.hadoop:hadoop-yarn-api:3.0.0 - no url defined)
- (Apache License, Version 2.0) hadoop-yarn-client (org.apache.hadoop:hadoop-yarn-client:2.6.5 - no url defined)
- (Apache License, Version 2.0) hadoop-yarn-client (org.apache.hadoop:hadoop-yarn-client:3.0.0 - no url defined)
- (Apache License, Version 2.0) hadoop-yarn-common (org.apache.hadoop:hadoop-yarn-common:2.6.5 - no url defined)
- (Apache License, Version 2.0) hadoop-yarn-common (org.apache.hadoop:hadoop-yarn-common:2.7.7 - no url defined)
- (Apache License, Version 2.0) hadoop-yarn-common (org.apache.hadoop:hadoop-yarn-common:3.0.0 - no url defined)
- (Apache License, Version 2.0) hadoop-yarn-server-common (org.apache.hadoop:hadoop-yarn-server-common:2.6.5 - no url defined)
+
(Apache License, Version 2.0) htrace-core4 (org.apache.htrace:htrace-core4:4.2.0-incubating - http://incubator.apache.org/projects/htrace.html)
(Apache License, Version 2.0) hudi-spark-bundle_2.11 (org.apache.hudi:hudi-spark-bundle_2.11:0.10.0 - https://github.com/apache/hudi/hudi-spark-bundle_2.11)
(Apache License, Version 2.0) java-xmlbuilder (com.jamesmurty.utils:java-xmlbuilder:0.4 - http://code.google.com/p/java-xmlbuilder/)
@@ -655,11 +649,7 @@ The text of each license is the standard Apache 2.0 license.
(The Apache Software License, Version 2.0) Apache Directory API ASN.1 API (org.apache.directory.api:api-asn1-api:1.0.0-M20 - http://directory.apache.org/api-parent/api-asn1-parent/api-asn1-api/)
(The Apache Software License, Version 2.0) Apache Directory LDAP API Utilities (org.apache.directory.api:api-util:1.0.0-M20 - http://directory.apache.org/api-parent/api-util/)
(The Apache Software License, Version 2.0) Apache Extras™ for Apache log4j™. (log4j:apache-log4j-extras:1.2.17 - http://logging.apache.org/log4j/extras)
- (The Apache Software License, Version 2.0) Apache Hadoop Annotations (org.apache.hadoop:hadoop-annotations:2.6.5 - no url defined)
- (The Apache Software License, Version 2.0) Apache Hadoop Auth (org.apache.hadoop:hadoop-auth:2.6.5 - no url defined)
- (The Apache Software License, Version 2.0) Apache Hadoop Client (org.apache.hadoop:hadoop-client:2.6.5 - no url defined)
- (The Apache Software License, Version 2.0) Apache Hadoop Common (org.apache.hadoop:hadoop-common:2.6.5 - no url defined)
- (The Apache Software License, Version 2.0) Apache Hadoop HDFS (org.apache.hadoop:hadoop-hdfs:2.6.5 - no url defined)
+
(The Apache Software License, Version 2.0) Apache Iceberg (org.apache.iceberg:iceberg-api:0.13.1 - https://iceberg.apache.org)
(The Apache Software License, Version 2.0) Apache Iceberg (org.apache.iceberg:iceberg-bundled-guava:0.13.1 - https://iceberg.apache.org)
(The Apache Software License, Version 2.0) Apache Iceberg (org.apache.iceberg:iceberg-common:0.13.1 - https://iceberg.apache.org)
@@ -865,15 +855,6 @@ The text of each license is the standard Apache 2.0 license.
(The Apache Software License, Version 2.0) flink-shaded-jackson-2 (org.apache.flink:flink-shaded-jackson:2.12.1-13.0 - http://flink.apache.org/flink-shaded-jackson-parent/flink-shaded-jackson)
(The Apache Software License, Version 2.0) flink-shaded-netty-4 (org.apache.flink:flink-shaded-netty:4.1.49.Final-13.0 - http://flink.apache.org/flink-shaded-netty)
(The Apache Software License, Version 2.0) flink-shaded-zookeeper-3.4 (org.apache.flink:flink-shaded-zookeeper-3:3.4.14-13.0 - http://flink.apache.org/flink-shaded-zookeeper-parent/flink-shaded-zookeeper-3)
- (The Apache Software License, Version 2.0) hadoop-mapreduce-client-app (org.apache.hadoop:hadoop-mapreduce-client-app:2.6.5 - no url defined)
- (The Apache Software License, Version 2.0) hadoop-mapreduce-client-common (org.apache.hadoop:hadoop-mapreduce-client-common:2.6.5 - no url defined)
- (The Apache Software License, Version 2.0) hadoop-mapreduce-client-core (org.apache.hadoop:hadoop-mapreduce-client-core:2.6.5 - no url defined)
- (The Apache Software License, Version 2.0) hadoop-mapreduce-client-jobclient (org.apache.hadoop:hadoop-mapreduce-client-jobclient:2.6.5 - no url defined)
- (The Apache Software License, Version 2.0) hadoop-mapreduce-client-shuffle (org.apache.hadoop:hadoop-mapreduce-client-shuffle:2.6.5 - no url defined)
- (The Apache Software License, Version 2.0) hadoop-yarn-api (org.apache.hadoop:hadoop-yarn-api:2.6.5 - no url defined)
- (The Apache Software License, Version 2.0) hadoop-yarn-client (org.apache.hadoop:hadoop-yarn-client:2.6.5 - no url defined)
- (The Apache Software License, Version 2.0) hadoop-yarn-common (org.apache.hadoop:hadoop-yarn-common:2.6.5 - no url defined)
- (The Apache Software License, Version 2.0) hadoop-yarn-server-common (org.apache.hadoop:hadoop-yarn-server-common:2.6.5 - no url defined)
(The Apache Software License, Version 2.0) htrace-core (org.apache.htrace:htrace-core:3.1.0-incubating - http://incubator.apache.org/projects/htrace.html)
(The Apache Software License, Version 2.0) htrace-core (org.htrace:htrace-core:3.0.4 - https://github.com/cloudera/htrace)
(The Apache Software License, Version 2.0) htrace-core4 (org.apache.htrace:htrace-core4:4.1.0-incubating - http://incubator.apache.org/projects/htrace.html)
@@ -904,6 +885,8 @@ The text of each license is the standard Apache 2.0 license.
(The Apache Software License, Version 2.0) secure-sm (org.elasticsearch:elasticsearch-secure-sm:6.3.1 - https://github.com/elastic/elasticsearch)
(The Apache Software License, Version 2.0) server (org.elasticsearch:elasticsearch:6.3.1 - https://github.com/elastic/elasticsearch)
(The Apache Software License, Version 2.0) server (org.elasticsearch:elasticsearch:7.5.1 - https://github.com/elastic/elasticsearch)
+ (The Apache Software License, Version 2.0) snappy-java (org.xerial.snappy:snappy-java:1.1.2.6 - https://github.com/xerial/snappy-java)
+ (The Apache Software License, Version 2.0) snappy-java (org.xerial.snappy:snappy-java:1.1.7.3 - https://github.com/xerial/snappy-java)
(The Apache Software License, Version 2.0) snappy-java (org.xerial.snappy:snappy-java:1.1.8.3 - https://github.com/xerial/snappy-java)
(The Apache Software License, Version 2.0) transport (org.elasticsearch.client:transport:6.3.1 - https://github.com/elastic/elasticsearch)
(The Apache Software License, Version 2.0) transport (org.elasticsearch.client:transport:7.5.1 - https://github.com/elastic/elasticsearch)
@@ -916,7 +899,9 @@ The text of each license is the standard Apache 2.0 license.
(The Apache Software License, Version 2.0) EL (commons-el:commons-el:1.0 - http://jakarta.apache.org/commons/el/)
(Apache License 2.0) Metrics Core (io.dropwizard.metrics:metrics-core:3.1.0 - http://metrics.codahale.com/metrics-core/)
(Apache License 2.0) Jackson Integration for Metrics (io.dropwizard.metrics:metrics-json:3.1.0 - http://metrics.codahale.com/metrics-json/)
+ (Apache License 2.0) Jackson Integration for Metrics (io.dropwizard.metrics:metrics-json:3.1.5 - http://metrics.codahale.com/metrics-json/)
(Apache License 2.0) JVM Integration for Metrics (io.dropwizard.metrics:metrics-jvm:3.1.0 - http://metrics.codahale.com/metrics-jvm/)
+ (Apache License 2.0) JVM Integration for Metrics (io.dropwizard.metrics:metrics-jvm:3.1.5 - http://metrics.codahale.com/metrics-jvm/)
(Apache 2) Joda-Time (joda-time:joda-time:2.8.1 - http://www.joda.org/joda-time/)
(Apache License, Version 2.0) eigenbase-properties (net.hydromatic:eigenbase-properties:1.1.5 - http://github.com/julianhyde/eigenbase-properties)
(Apache 2) opencsv (net.sf.opencsv:opencsv:2.3 - http://opencsv.sf.net)
@@ -1187,6 +1172,9 @@ The text of each license is also included at licenses/LICENSE-[project].txt.
(CDDL License) JavaBeans Activation Framework (com.sun.activation:javax.activation:1.2.0 - http://java.net/all/javax.activation/)
(CDDL License) JavaBeans Activation Framework API jar (javax.activation:javax.activation-api:1.2.0 - http://java.net/all/javax.activation-api/)
(CDDL License) JavaMail API (com.sun.mail:javax.mail:1.5.6 - http://javamail.java.net/javax.mail)
+ (CDDL License) jersey-core (com.sun.jersey:jersey-core:1.9 - https://mvnrepository.com/artifact/com.sun.jersey/jersey-core/1.9)
+ (CDDL License) jersey-json (com.sun.jersey:jersey-json:1.9 - https://mvnrepository.com/artifact/com.sun.jersey/jersey-json/1.9)
+ (CDDL License) jersey-server (com.sun.jersey:jersey-server:1.9 - https://mvnrepository.com/artifact/com.sun.jersey/jersey-server/1.9)
(Common Development and Distribution License (CDDL) v1.0) JavaMail API (javax.mail:mail:1.4.1 - https://glassfish.dev.java.net/javaee5/mail/)
(COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0) (GNU General Public Library) Streaming API for XML (javax.xml.stream:stax-api:1.0-2 - no url defined)
diff --git a/seatunnel-dist/release-docs/licenses/LICENSE-com.sun.jersey.txt b/seatunnel-dist/release-docs/licenses/LICENSE-com.sun.jersey.txt
new file mode 100644
index 00000000000..d1e3c0bea37
--- /dev/null
+++ b/seatunnel-dist/release-docs/licenses/LICENSE-com.sun.jersey.txt
@@ -0,0 +1,93 @@
+COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0 1.
+
+Definitions.
+
+1.1. Contributor means each individual or entity that creates or contributes to the creation of Modifications.
+
+1.2. Contributor Version means the combination of the Original Software, prior Modifications used by a Contributor (if any), and the Modifications made by that particular Contributor.
+
+1.3. Covered Software means (a) the Original Software, or (b) Modifications, or (c) the combination of files containing Original Software with files containing Modifications, in each case including portions thereof.
+
+1.4. Executable means the Covered Software in any form other than Source Code.
+
+1.5. Initial Developer means the individual or entity that first makes Original Software available under this License.
+
+1.6. Larger Work means a work which combines Covered Software or portions thereof with code not governed by the terms of this License.
+
+1.7. License means this document.
+
+1.8. Licensable means having the right to grant, to the maximum extent possible, whether at the time of the initial grant or subsequently acquired, any and all of the rights conveyed herein.
+
+1.9. Modifications means the Source Code and Executable form of any of the following: A. Any file that results from an addition to, deletion from or modification of the contents of a file containing Original Software or previous Modifications; B. Any new file that contains any part of the Original Software or previous Modification; or C. Any new file that is contributed or otherwise made available under the terms of this License.
+
+1.10. Original Software means the Source Code and Executable form of computer software code that is originally released under this License.
+
+1.11. Patent Claims means any patent claim(s), now owned or hereafter acquired, including without limitation, method, process, and apparatus claims, in any patent Licensable by grantor.
+
+1.12. Source Code means (a) the common form of computer software code in which modifications are made and (b) associated documentation included in or with such code.
+
+1.13. You (or Your) means an individual or a legal entity exercising rights under, and complying with all of the terms of, this License. For legal entities, You includes any entity which controls, is controlled by, or is under common control with You. For purposes of this definition, control means (a) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (b) ownership of more than fifty percent (50%) of the outstanding shares or beneficial ownership of such entity.
+
+2. License Grants.
+
+ 2.1. The Initial Developer Grant. Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, the Initial Developer hereby grants You a world-wide, royalty-free, non-exclusive license:
+
+(a) under intellectual property rights (other than patent or trademark) Licensable by Initial Developer, to use, reproduce, modify, display, perform, sublicense and distribute the Original Software (or portions thereof), with or without Modifications, and/or as part of a Larger Work; and
+
+(b) under Patent Claims infringed by the making, using or selling of Original Software, to make, have made, use, practice, sell, and offer for sale, and/or otherwise dispose of the Original Software (or portions thereof);
+
+ (c) The licenses granted in Sections 2.1(a) and (b) are effective on the date Initial Developer first distributes or otherwise makes the Original Software available to a third party under the terms of this License;
+
+ (d) Notwithstanding Section 2.1(b) above, no patent license is granted: (1) for code that You delete from the Original Software, or (2) for infringements caused by: (i) the modification of the Original Software, or (ii) the combination of the Original Software with other software or devices.
+
+2.2. Contributor Grant. Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, each Contributor hereby grants You a world-wide, royalty-free, non-exclusive license:
+
+(a) under intellectual property rights (other than patent or trademark) Licensable by Contributor to use, reproduce, modify, display, perform, sublicense and distribute the Modifications created by such Contributor (or portions thereof), either on an unmodified basis, with other Modifications, as Covered Software and/or as part of a Larger Work; and
+
+(b) under Patent Claims infringed by the making, using, or selling of Modifications made by that Contributor either alone and/or in combination with its Contributor Version (or portions of such combination), to make, use, sell, offer for sale, have made, and/or otherwise dispose of: (1) Modifications made by that Contributor (or portions thereof); and (2) the combination of Modifications made by that Contributor with its Contributor Version (or portions of such combination).
+
+(c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first distributes or otherwise makes the Modifications available to a third party.
+
+(d) Notwithstanding Section 2.2(b) above, no patent license is granted: (1) for any code that Contributor has deleted from the Contributor Version; (2) for infringements caused by: (i) third party modifications of Contributor Version, or (ii) the combination of Modifications made by that Contributor with other software (except as part of the Contributor Version) or other devices; or (3) under Patent Claims infringed by Covered Software in the absence of Modifications made by that Contributor.
+
+3. Distribution Obligations.
+
+3.1. Availability of Source Code. Any Covered Software that You distribute or otherwise make available in Executable form must also be made available in Source Code form and that Source Code form must be distributed only under the terms of this License. You must include a copy of this License with every copy of the Source Code form of the Covered Software You distribute or otherwise make available. You must inform recipients of any such Covered Software in Executable form as to how they can obtain such Covered Software in Source Code form in a reasonable manner on or through a medium customarily used for software exchange.
+
+3.2. Modifications. The Modifications that You create or to which You contribute are governed by the terms of this License. You represent that You believe Your Modifications are Your original creation(s) and/or You have sufficient rights to grant the rights conveyed by this License.
+
+3.3. Required Notices. You must include a notice in each of Your Modifications that identifies You as the Contributor of the Modification. You may not remove or alter any copyright, patent or trademark notices contained within the Covered Software, or any notices of licensing or any descriptive text giving attribution to any Contributor or the Initial Developer.
+
+3.4. Application of Additional Terms. You may not offer or impose any terms on any Covered Software in Source Code form that alters or restricts the applicable version of this License or the recipients rights hereunder. You may choose to offer, and to charge a fee for, warranty, support, indemnity or liability obligations to one or more recipients of Covered Software. However, you may do so only on Your own behalf, and not on behalf of the Initial Developer or any Contributor. You must make it absolutely clear that any such warranty, support, indemnity or liability obligation is offered by You alone, and You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of warranty, support, indemnity or liability terms You offer.
+
+3.5. Distribution of Executable Versions. You may distribute the Executable form of the Covered Software under the terms of this License or under the terms of a license of Your choice, which may contain terms different from this License, provided that You are in compliance with the terms of this License and that the license for the Executable form does not attempt to limit or alter the recipients rights in the Source Code form from the rights set forth in this License. If You distribute the Covered Software in Executable form under a different license, You must make it absolutely clear that any terms which differ from this License are offered by You alone, not by the Initial Developer or Contributor. You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of any such terms You offer.
+
+3.6. Larger Works. You may create a Larger Work by combining Covered Software with other code not governed by the terms of this License and distribute the Larger Work as a single product. In such a case, You must make sure the requirements of this License are fulfilled for the Covered Software.
+
+4. Versions of the License.
+
+4.1. New Versions. Sun Microsystems, Inc. is the initial license steward and may publish revised and/or new versions of this License from time to time. Each version will be given a distinguishing version number. Except as provided in Section 4.3, no one other than the license steward has the right to modify this License.
+
+4.2. Effect of New Versions. You may always continue to use, distribute or otherwise make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. If the Initial Developer includes a notice in the Original Software prohibiting it from being distributed or otherwise made available under any subsequent version of the License, You must distribute and make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. Otherwise, You may also choose to use, distribute or otherwise make the Covered Software available under the terms of any subsequent version of the License published by the license steward.
+
+4.3. Modified Versions. When You are an Initial Developer and You want to create a new license for Your Original Software, You may create and use a modified version of this License if You: (a) rename the license and remove any references to the name of the license steward (except to note that the license differs from this License); and (b) otherwise make it clear that the license contains terms which differ from this License.
+
+5. DISCLAIMER OF WARRANTY. COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN AS IS BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER.
+
+6. TERMINATION.
+
+6.1. This License and the rights granted hereunder will terminate automatically if You fail to comply with terms herein and fail to cure such breach within 30 days of becoming aware of the breach. Provisions which, by their nature, must remain in effect beyond the termination of this License shall survive.
+
+6.2. If You assert a patent infringement claim (excluding declaratory judgment actions) against Initial Developer or a Contributor (the Initial Developer or Contributor against whom You assert such claim is referred to as Participant) alleging that the Participant Software (meaning the Contributor Version where the Participant is a Contributor or the Original Software where the Participant is the Initial Developer) directly or indirectly infringes any patent, then any and all rights granted directly or indirectly to You by such Participant, the Initial Developer (if the Initial Developer is not the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice from Participant terminate prospectively and automatically at the expiration of such 60 day notice period, unless if within such 60 day period You withdraw Your claim with respect to the Participant Software against such Participant either unilaterally or pursuant to a written agreement with Participant.
+
+6.3. In the event of termination under Sections 6.1 or 6.2 above, all end user licenses that have been validly granted by You or any distributor hereunder prior to termination (excluding licenses granted to You by any distributor) shall survive termination.
+
+7. LIMITATION OF LIABILITY. UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOST PROFITS, LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTYS NEGLIGENCE TO THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU.
+
+8. U.S. GOVERNMENT END USERS. The Covered Software is a commercial item, as that term is defined in 48 C.F.R. 2.101 (Oct. 1995), consisting of commercial computer software (as that term is defined at 48 C.F.R. 252.227-7014(a)(1)) and commercial computer software documentation as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software with only those rights set forth herein. This U.S. Government Rights clause is in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision that addresses Government rights in computer software under this License.
+
+9. MISCELLANEOUS. This License represents the complete agreement concerning subject matter hereof. If any provision of this License is held to be unenforceable, such provision shall be reformed only to the extent necessary to make it enforceable. This License shall be governed by the law of the jurisdiction specified in a notice contained within the Original Software (except to the extent applicable law, if any, provides otherwise), excluding such jurisdictions conflict-of-law provisions. Any litigation relating to this License shall be subject to the jurisdiction of the courts located in the jurisdiction and venue specified in a notice contained within the Original Software, with the losing party responsible for costs, including, without limitation, court costs and reasonable attorneys fees and expenses. The application of the United Nations Convention on Contracts for the International Sale of Goods is expressly excluded. Any law or regulation which provides that the language of a contract shall be construed against the drafter shall not apply to this License. You agree that You alone are responsible for compliance with the United States export administration regulations (and the export control laws and regulation of any other countries) when You use, distribute or otherwise make available any Covered Software.
+
+10. RESPONSIBILITY FOR CLAIMS. As between Initial Developer and the Contributors, each party is responsible for claims and damages arising, directly or indirectly, out of its utilization of rights under this License and You agree to work with Initial Developer and Contributors to distribute such responsibility on an equitable basis. Nothing herein is intended or shall be deemed to constitute any admission of liability.
+
+NOTICE PURSUANT TO SECTION 9 OF THE COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) The code released under the CDDL shall be governed by the laws of the State of California (excluding conflict-of-law provisions). Any litigation relating to this License shall be subject to the jurisdiction of the Federal Courts of the Northern District of California and the state courts of the State of California, with venue lying in Santa Clara County, California.
\ No newline at end of file
diff --git a/seatunnel-e2e/pom.xml b/seatunnel-e2e/pom.xml
index c74e5c31abf..cc08446f010 100644
--- a/seatunnel-e2e/pom.xml
+++ b/seatunnel-e2e/pom.xml
@@ -34,4 +34,22 @@
seatunnel-flink-sql-e2e
+
+
+ org.apache.seatunnel
+ seatunnel-connectors-v2-dist
+ ${project.version}
+
+
+ org.apache.seatunnel
+ seatunnel-connectors-spark-dist
+ ${project.version}
+
+
+ org.apache.seatunnel
+ seatunnel-connectors-flink-dist
+ ${project.version}
+
+
+
\ No newline at end of file
diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/FlinkContainer.java b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/FlinkContainer.java
index b706468c3f0..0ffa3163480 100644
--- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/FlinkContainer.java
+++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/FlinkContainer.java
@@ -164,5 +164,4 @@ private String getResource(String confFile) {
private String getConnectorPath(String fileName) {
return Paths.get(SEATUNNEL_CONNECTORS, "seatunnel", fileName).toString();
}
-
}
diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/assertion/FakeSourceToAssertIT.java b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/assertion/FakeSourceToAssertIT.java
similarity index 96%
rename from seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/assertion/FakeSourceToAssertIT.java
rename to seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/assertion/FakeSourceToAssertIT.java
index cc67b836a63..ce89154f40c 100644
--- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/assertion/FakeSourceToAssertIT.java
+++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/assertion/FakeSourceToAssertIT.java
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.seatunnel.e2e.flink.assertion;
+package org.apache.seatunnel.e2e.flink.v2.assertion;
import org.apache.seatunnel.e2e.flink.FlinkContainer;
diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/fake/FakeSourceToConsoleIT.java b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/fake/FakeSourceToConsoleIT.java
similarity index 96%
rename from seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/fake/FakeSourceToConsoleIT.java
rename to seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/fake/FakeSourceToConsoleIT.java
index 2663eb80af4..f715c4e79ad 100644
--- a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/fake/FakeSourceToConsoleIT.java
+++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/fake/FakeSourceToConsoleIT.java
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.seatunnel.e2e.flink.fake;
+package org.apache.seatunnel.e2e.flink.v2.fake;
import org.apache.seatunnel.e2e.flink.FlinkContainer;
diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/file/FakeSourceToFileIT.java b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/file/FakeSourceToFileIT.java
new file mode 100644
index 00000000000..ec52203c53b
--- /dev/null
+++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/file/FakeSourceToFileIT.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.e2e.flink.v2.file;
+
+import org.apache.seatunnel.e2e.flink.FlinkContainer;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.testcontainers.containers.Container;
+
+import java.io.IOException;
+
+public class FakeSourceToFileIT extends FlinkContainer {
+ @Test
+ public void testFakeSourceToFileSink() throws IOException, InterruptedException {
+ Container.ExecResult execResult = executeSeaTunnelFlinkJob("/file/fakesource_to_file.conf");
+ Assert.assertEquals(0, execResult.getExitCode());
+ }
+}
diff --git a/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/resources/file/fakesource_to_file.conf b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/resources/file/fakesource_to_file.conf
new file mode 100644
index 00000000000..7e01a805f30
--- /dev/null
+++ b/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/src/test/resources/file/fakesource_to_file.conf
@@ -0,0 +1,64 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+######
+###### This config file is a demonstration of streaming processing in seatunnel config
+######
+
+env {
+ # You can set flink configuration here
+ execution.parallelism = 1
+ job.mode = "BATCH"
+ #execution.checkpoint.interval = 10000
+ #execution.checkpoint.data-uri = "hdfs://localhost:9000/checkpoint"
+}
+
+source {
+ # This is a example source plugin **only for test and demonstrate the feature source plugin**
+ FakeSource {
+ result_table_name = "fake"
+ field_name = "name,age"
+ }
+
+ # If you would like to get more information about how to configure seatunnel and see full list of source plugins,
+ # please go to https://seatunnel.apache.org/docs/flink/configuration/source-plugins/Fake
+}
+
+transform {
+ # 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 {
+ LocalFile {
+ path="file:///tmp/hive/warehouse/test2"
+ field_delimiter="\t"
+ row_delimiter="\n"
+ 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,
+ # please go to https://seatunnel.apache.org/docs/flink/configuration/sink-plugins/Console
+}
\ No newline at end of file
diff --git a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/fake/FakeSourceToConsoleIT.java b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/fake/FakeSourceToConsoleIT.java
similarity index 96%
rename from seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/fake/FakeSourceToConsoleIT.java
rename to seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/fake/FakeSourceToConsoleIT.java
index 2f9f1d1d7b1..5f35135bb6d 100644
--- a/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/fake/FakeSourceToConsoleIT.java
+++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/fake/FakeSourceToConsoleIT.java
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.seatunnel.e2e.spark.fake;
+package org.apache.seatunnel.e2e.spark.v2.fake;
import org.apache.seatunnel.e2e.spark.SparkContainer;
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
new file mode 100644
index 00000000000..5ebe51d4792
--- /dev/null
+++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/file/FakeSourceToFileIT.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.e2e.spark.v2.file;
+
+import org.apache.seatunnel.e2e.spark.SparkContainer;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.testcontainers.containers.Container;
+
+import java.io.IOException;
+
+/**
+ * This test case is used to verify that the fake source is able to send data to the console.
+ * Make sure the SeaTunnel job can submit successfully on spark engine.
+ */
+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-e2e/seatunnel-spark-connector-v2-e2e/src/test/resources/file/fakesource_to_file.conf b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/resources/file/fakesource_to_file.conf
new file mode 100644
index 00000000000..e70490855c3
--- /dev/null
+++ b/seatunnel-e2e/seatunnel-spark-connector-v2-e2e/src/test/resources/file/fakesource_to_file.conf
@@ -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.
+#
+######
+###### This config file is a demonstration of streaming processing in seatunnel config
+######
+
+env {
+ # You can set flink configuration here
+ execution.parallelism = 1
+ job.mode = "BATCH"
+ #execution.checkpoint.interval = 10000
+ #execution.checkpoint.data-uri = "hdfs://localhost:9000/checkpoint"
+}
+
+source {
+ # This is a example source plugin **only for test and demonstrate the feature source plugin**
+ FakeSource {
+ result_table_name = "fake"
+ field_name = "name,age"
+ }
+
+ # If you would like to get more information about how to configure seatunnel and see full list of source plugins,
+ # please go to https://seatunnel.apache.org/docs/flink/configuration/source-plugins/Fake
+}
+
+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 {
+ LocalFile {
+ path="file:///tmp/hive/warehouse/test2"
+ field_delimiter="\t"
+ row_delimiter="\n"
+ 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,
+ # please go to https://seatunnel.apache.org/docs/flink/configuration/sink-plugins/Console
+}
\ No newline at end of file
diff --git a/seatunnel-examples/pom.xml b/seatunnel-examples/pom.xml
index 823b1404ecf..6025128c95d 100644
--- a/seatunnel-examples/pom.xml
+++ b/seatunnel-examples/pom.xml
@@ -33,8 +33,8 @@
seatunnel-flink-examples
seatunnel-spark-examples
seatunnel-flink-sql-examples
- seatunnel-flink-new-connector-example
- seatunnel-spark-new-connector-example
+ seatunnel-flink-connector-v2-example
+ seatunnel-spark-connector-v2-example
diff --git a/seatunnel-examples/seatunnel-flink-new-connector-example/pom.xml b/seatunnel-examples/seatunnel-flink-connector-v2-example/pom.xml
similarity index 98%
rename from seatunnel-examples/seatunnel-flink-new-connector-example/pom.xml
rename to seatunnel-examples/seatunnel-flink-connector-v2-example/pom.xml
index c27915ac9d4..68e0ad2e89e 100644
--- a/seatunnel-examples/seatunnel-flink-new-connector-example/pom.xml
+++ b/seatunnel-examples/seatunnel-flink-connector-v2-example/pom.xml
@@ -27,7 +27,7 @@
4.0.0
- seatunnel-flink-new-connector-example
+ seatunnel-flink-connector-v2-example
compile
diff --git a/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/java/org/apache/seatunnel/example/flink/v2/FakeToLocalFileExample.java b/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/java/org/apache/seatunnel/example/flink/v2/FakeToLocalFileExample.java
new file mode 100644
index 00000000000..fc15ff44b2d
--- /dev/null
+++ b/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/java/org/apache/seatunnel/example/flink/v2/FakeToLocalFileExample.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.example.flink.v2;
+
+import org.apache.seatunnel.core.starter.Seatunnel;
+import org.apache.seatunnel.core.starter.command.Command;
+import org.apache.seatunnel.core.starter.exception.CommandException;
+import org.apache.seatunnel.core.starter.flink.args.FlinkCommandArgs;
+import org.apache.seatunnel.core.starter.flink.command.FlinkCommandBuilder;
+
+import java.io.FileNotFoundException;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.file.Paths;
+
+public class FakeToLocalFileExample {
+
+ public static void main(String[] args) throws FileNotFoundException, URISyntaxException, CommandException {
+ String configFile = getTestConfigFile("/examples/fakesource_to_file.conf");
+ FlinkCommandArgs flinkCommandArgs = new FlinkCommandArgs();
+ flinkCommandArgs.setConfigFile(configFile);
+ flinkCommandArgs.setCheckConfig(false);
+ flinkCommandArgs.setVariables(null);
+ Command flinkCommand =
+ new FlinkCommandBuilder().buildCommand(flinkCommandArgs);
+ Seatunnel.run(flinkCommand);
+ }
+
+ public static String getTestConfigFile(String configFile) throws FileNotFoundException, URISyntaxException {
+ URL resource = FakeToLocalFileExample.class.getResource(configFile);
+ if (resource == null) {
+ throw new FileNotFoundException("Can't find config file: " + configFile);
+ }
+ return Paths.get(resource.toURI()).toString();
+ }
+}
diff --git a/seatunnel-examples/seatunnel-flink-new-connector-example/src/main/java/org/apache/seatunnel/example/flink/SeaTunnelApiExample.java b/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/java/org/apache/seatunnel/example/flink/v2/SeaTunnelApiExample.java
similarity index 97%
rename from seatunnel-examples/seatunnel-flink-new-connector-example/src/main/java/org/apache/seatunnel/example/flink/SeaTunnelApiExample.java
rename to seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/java/org/apache/seatunnel/example/flink/v2/SeaTunnelApiExample.java
index 56a2882b677..79912ae9619 100644
--- a/seatunnel-examples/seatunnel-flink-new-connector-example/src/main/java/org/apache/seatunnel/example/flink/SeaTunnelApiExample.java
+++ b/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/java/org/apache/seatunnel/example/flink/v2/SeaTunnelApiExample.java
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.seatunnel.example.flink;
+package org.apache.seatunnel.example.flink.v2;
import org.apache.seatunnel.core.starter.Seatunnel;
import org.apache.seatunnel.core.starter.command.Command;
diff --git a/seatunnel-examples/seatunnel-flink-new-connector-example/src/main/resources/examples/fake_to_console.conf b/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/resources/examples/fake_to_console.conf
similarity index 100%
rename from seatunnel-examples/seatunnel-flink-new-connector-example/src/main/resources/examples/fake_to_console.conf
rename to seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/resources/examples/fake_to_console.conf
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
new file mode 100644
index 00000000000..c1ce63055d9
--- /dev/null
+++ b/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/resources/examples/fakesource_to_file.conf
@@ -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.
+#
+######
+###### This config file is a demonstration of streaming processing in seatunnel config
+######
+
+env {
+ # You can set flink configuration here
+ execution.parallelism = 1
+ job.mode = "STREAMING"
+ execution.checkpoint.interval = 5000
+ #execution.checkpoint.data-uri = "hdfs://localhost:9000/checkpoint"
+}
+
+source {
+ # This is a example source plugin **only for test and demonstrate the feature source plugin**
+ FakeSource {
+ result_table_name = "fake"
+ field_name = "name,age"
+ }
+
+ # If you would like to get more information about how to configure seatunnel and see full list of source plugins,
+ # please go to https://seatunnel.apache.org/docs/flink/configuration/source-plugins/Fake
+}
+
+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"
+ 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,
+ # please go to https://seatunnel.apache.org/docs/flink/configuration/sink-plugins/Console
+}
\ No newline at end of file
diff --git a/seatunnel-examples/seatunnel-flink-new-connector-example/src/main/resources/log4j.properties b/seatunnel-examples/seatunnel-flink-new-connector-example/src/main/resources/log4j.properties
deleted file mode 100644
index db5d9e51220..00000000000
--- a/seatunnel-examples/seatunnel-flink-new-connector-example/src/main/resources/log4j.properties
+++ /dev/null
@@ -1,22 +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.
-#
-# Set everything to be logged to the console
-log4j.rootCategory=INFO, console
-log4j.appender.console=org.apache.log4j.ConsoleAppender
-log4j.appender.console.target=System.err
-log4j.appender.console.layout=org.apache.log4j.PatternLayout
-log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n
diff --git a/seatunnel-examples/seatunnel-spark-new-connector-example/pom.xml b/seatunnel-examples/seatunnel-spark-connector-v2-example/pom.xml
similarity index 98%
rename from seatunnel-examples/seatunnel-spark-new-connector-example/pom.xml
rename to seatunnel-examples/seatunnel-spark-connector-v2-example/pom.xml
index e75a88ce53d..be2abd0b0b6 100644
--- a/seatunnel-examples/seatunnel-spark-new-connector-example/pom.xml
+++ b/seatunnel-examples/seatunnel-spark-connector-v2-example/pom.xml
@@ -27,7 +27,7 @@
4.0.0
- seatunnel-spark-new-connector-example
+ seatunnel-spark-connector-v2-example
compile
diff --git a/seatunnel-examples/seatunnel-spark-new-connector-example/src/main/java/org/apache/seatunnel/example/spark/SeaTunnelApiExample.java b/seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/java/org/apache/seatunnel/example/spark/v2/SeaTunnelApiExample.java
similarity index 98%
rename from seatunnel-examples/seatunnel-spark-new-connector-example/src/main/java/org/apache/seatunnel/example/spark/SeaTunnelApiExample.java
rename to seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/java/org/apache/seatunnel/example/spark/v2/SeaTunnelApiExample.java
index cdb988fba55..c1e07dd7507 100644
--- a/seatunnel-examples/seatunnel-spark-new-connector-example/src/main/java/org/apache/seatunnel/example/spark/SeaTunnelApiExample.java
+++ b/seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/java/org/apache/seatunnel/example/spark/v2/SeaTunnelApiExample.java
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.seatunnel.example.spark;
+package org.apache.seatunnel.example.spark.v2;
import org.apache.seatunnel.common.config.DeployMode;
import org.apache.seatunnel.core.starter.Seatunnel;
diff --git a/seatunnel-examples/seatunnel-spark-new-connector-example/src/main/resources/examples/spark.batch.conf b/seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/resources/examples/spark.batch.conf
similarity index 100%
rename from seatunnel-examples/seatunnel-spark-new-connector-example/src/main/resources/examples/spark.batch.conf
rename to seatunnel-examples/seatunnel-spark-connector-v2-example/src/main/resources/examples/spark.batch.conf
diff --git a/seatunnel-translation/pom.xml b/seatunnel-translation/pom.xml
index 4f9359fcd74..3ada6b5f5f9 100644
--- a/seatunnel-translation/pom.xml
+++ b/seatunnel-translation/pom.xml
@@ -31,5 +31,4 @@
seatunnel-translation-flink
seatunnel-translation-spark
-
\ No newline at end of file
diff --git a/tools/dependencies/known-dependencies.txt b/tools/dependencies/known-dependencies.txt
index 683e35dd83e..3ff6a42893b 100755
--- a/tools/dependencies/known-dependencies.txt
+++ b/tools/dependencies/known-dependencies.txt
@@ -71,12 +71,12 @@ checker-qual-3.4.0.jar
chill-java-0.9.3.jar
chill_2.11-0.9.3.jar
classmate-1.1.0.jar
+clickhouse-jdbc-0.2.jar
classmate-1.3.1.jar
clickhouse-client-0.3.2-patch9.jar
clickhouse-grpc-client-0.3.2-patch9-netty.jar
clickhouse-http-client-0.3.2-patch9-shaded.jar
clickhouse-http-client-0.3.2-patch9.jar
-clickhouse-jdbc-0.2.jar
clickhouse-jdbc-0.3.2-patch9.jar
commons-beanutils-1.9.4.jar
commons-cli-1.4.jar
@@ -356,14 +356,16 @@ jackson-module-jaxb-annotations-2.7.8.jar
jackson-module-parameter-names-2.13.3.jar
jackson-xc-1.9.13.jar
jakarta.activation-api-1.2.1.jar
+jakarta.xml.bind-api-2.3.2.jar
+jakarta.activation-api-1.2.2.jar
jakarta.annotation-api-1.3.5.jar
jakarta.servlet-api-4.0.4.jar
-jakarta.activation-api-1.2.2.jar
-jakarta.ws.rs-api-2.1.6.jar
-jakarta.xml.bind-api-2.3.2.jar
jakarta.validation-api-2.0.2.jar
jakarta.websocket-api-1.1.2.jar
jakarta.xml.bind-api-2.3.3.jar
+jcip-annotations-1.0.jar
+jul-to-slf4j-1.7.25.jar
+jakarta.ws.rs-api-2.1.6.jar
jamon-runtime-2.4.1.jar
janino-3.1.6.jar
jasper-compiler-5.5.23.jar
@@ -393,7 +395,6 @@ jaxb-impl-2.2.3-1.jar
jboss-logging-3.2.1.Final.jar
jboss-logging-3.4.1.Final.jar
jcip-annotations-1.0-1.jar
-jcip-annotations-1.0.jar
jul-to-slf4j-1.7.36.jar
jcl-over-slf4j-1.7.12.jar
jcl-over-slf4j-1.7.16.jar
@@ -486,7 +487,6 @@ jsr305-1.3.9.jar
jsr305-2.0.1.jar
jsr305-3.0.0.jar
jsr311-api-1.1.1.jar
-jul-to-slf4j-1.7.25.jar
jvm-attach-api-1.5.jar
kafka-clients-2.0.0.jar
kafka-clients-2.4.1.jar
@@ -582,7 +582,9 @@ metrics-core-3.1.5.jar
metrics-core-3.2.1.jar
metrics-core-4.0.0.jar
metrics-json-3.1.0.jar
+metrics-json-3.1.5.jar
metrics-jvm-3.1.0.jar
+metrics-jvm-3.1.5.jar
minlog-1.3.0.jar
mongo-java-driver-3.4.2.jar
mongo-spark-connector_2.11-2.2.0.jar