method = getDeclaredMethod(object.getClass(), methodName, argTypes);
+ if (method.isPresent()) {
+ method.get().setAccessible(true);
+ return method.get().invoke(object, args);
+ } else {
+ throw new NoSuchMethodException(String.format("method invoke failed, no such method '%s' in '%s'",
+ methodName, object.getClass()));
+ }
} catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) {
throw new RuntimeException("method invoke failed", e);
}
diff --git a/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/SerializationException.java b/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/SerializationException.java
new file mode 100644
index 00000000000..33ca29a6f7d
--- /dev/null
+++ b/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/SerializationException.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.common.utils;
+
+public class SerializationException extends RuntimeException {
+
+ /**
+ * Required for serialization support.
+ *
+ * @see java.io.Serializable
+ */
+ private static final long serialVersionUID = 2263144814025689516L;
+
+ /**
+ * Constructs a new {@code SerializationException} without specified
+ * detail message.
+ */
+ public SerializationException() {
+ }
+
+ /**
+ * Constructs a new {@code SerializationException} with specified
+ * detail message.
+ *
+ * @param msg The error message.
+ */
+ public SerializationException(final String msg) {
+ super(msg);
+ }
+
+ /**
+ * Constructs a new {@code SerializationException} with specified
+ * nested {@code Throwable}.
+ *
+ * @param cause The {@code Exception} or {@code Error}
+ * that caused this exception to be thrown.
+ */
+ public SerializationException(final Throwable cause) {
+ super(cause);
+ }
+
+ /**
+ * Constructs a new {@code SerializationException} with specified
+ * detail message and nested {@code Throwable}.
+ *
+ * @param msg The error message.
+ * @param cause The {@code Exception} or {@code Error}
+ * that caused this exception to be thrown.
+ */
+ public SerializationException(final String msg, final Throwable cause) {
+ super(msg, cause);
+ }
+}
diff --git a/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/SerializationUtils.java b/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/SerializationUtils.java
index 43dead16767..7e721617e8e 100644
--- a/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/SerializationUtils.java
+++ b/seatunnel-common/src/main/java/org/apache/seatunnel/common/utils/SerializationUtils.java
@@ -20,30 +20,60 @@
import org.apache.commons.codec.binary.Base64;
import org.apache.commons.lang3.StringUtils;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamClass;
import java.io.Serializable;
public class SerializationUtils {
public static String objectToString(Serializable obj) {
if (obj != null) {
- return Base64.encodeBase64String(org.apache.commons.lang3.SerializationUtils.serialize(obj));
+ return Base64.encodeBase64String(serialize(obj));
}
return null;
}
public static T stringToObject(String str) {
if (StringUtils.isNotEmpty(str)) {
- return org.apache.commons.lang3.SerializationUtils.deserialize(Base64.decodeBase64(str));
+ return deserialize(Base64.decodeBase64(str));
}
return null;
}
+ @SuppressWarnings("checkstyle:MagicNumber")
public static byte[] serialize(T obj) {
- return org.apache.commons.lang3.SerializationUtils.serialize(obj);
+ try (ByteArrayOutputStream b = new ByteArrayOutputStream(512);
+ ObjectOutputStream out = new ObjectOutputStream(b)) {
+ out.writeObject(obj);
+ return b.toByteArray();
+ } catch (final IOException ex) {
+ throw new SerializationException(ex);
+ }
+
}
public static T deserialize(byte[] bytes) {
- return org.apache.commons.lang3.SerializationUtils.deserialize(bytes);
+ try (ByteArrayInputStream s = new ByteArrayInputStream(bytes);
+ ObjectInputStream in = new ObjectInputStream(s) {
+ @Override
+ protected Class> resolveClass(ObjectStreamClass desc) throws IOException, ClassNotFoundException {
+ // make sure use current thread classloader
+ ClassLoader cl = Thread.currentThread().getContextClassLoader();
+ if (cl == null) {
+ return super.resolveClass(desc);
+ }
+ return Class.forName(desc.getName(), false, cl);
+ }
+ }) {
+ @SuppressWarnings("unchecked") final T obj = (T) in.readObject();
+ return obj;
+ } catch (final ClassNotFoundException | IOException ex) {
+ throw new SerializationException(ex);
+ }
}
}
diff --git a/seatunnel-common/src/test/java/org/apache/seatunnel/common/utils/ReflectionUtilsTest.java b/seatunnel-common/src/test/java/org/apache/seatunnel/common/utils/ReflectionUtilsTest.java
new file mode 100644
index 00000000000..422c70eef4d
--- /dev/null
+++ b/seatunnel-common/src/test/java/org/apache/seatunnel/common/utils/ReflectionUtilsTest.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.common.utils;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLClassLoader;
+
+public class ReflectionUtilsTest {
+
+ @Test
+ public void testInvoke() throws MalformedURLException {
+ ReflectionUtils.invoke(new String[]{}, "toString");
+
+ URLClassLoader classLoader = new URLClassLoader(new URL[]{}, Thread.currentThread().getContextClassLoader());
+ ReflectionUtils.invoke(classLoader, "addURL", new URL("file:///test"));
+ Assert.assertArrayEquals(classLoader.getURLs(), new URL[]{new URL("file:///test")});
+ }
+
+}
diff --git a/seatunnel-common/src/test/java/org/apache/seatunnel/common/utils/SerializationUtilsTest.java b/seatunnel-common/src/test/java/org/apache/seatunnel/common/utils/SerializationUtilsTest.java
new file mode 100644
index 00000000000..ad0e80ceabb
--- /dev/null
+++ b/seatunnel-common/src/test/java/org/apache/seatunnel/common/utils/SerializationUtilsTest.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.common.utils;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+
+@SuppressWarnings("checkstyle:RegexpSingleline")
+public class SerializationUtilsTest {
+
+ @Test
+ public void testObjectToString() {
+
+ HashMap data = new HashMap<>();
+ data.put("key1", "value1");
+ data.put("seatunnelTest", "apache SeaTunnel");
+ data.put("中 文", "Apache Asia");
+ String configStr = SerializationUtils.objectToString(data);
+ Assert.assertNotNull(configStr);
+
+ HashMap dataAfter = SerializationUtils.stringToObject(configStr);
+
+ Assert.assertEquals(dataAfter, data);
+
+ data.put("key2", "");
+ Assert.assertNotEquals(dataAfter, data);
+
+ }
+
+ @Test
+ public void testByteToObject() {
+
+ HashMap data = new HashMap<>();
+ data.put("key1", "value1");
+ data.put("seatunnelTest", "apache SeaTunnel");
+ data.put("中 文", "Apache Asia");
+
+ ArrayList> array = new ArrayList<>();
+ array.add(data);
+ HashMap data2 = new HashMap<>();
+ data2.put("Apache Asia", "中 文");
+ data2.put("value1", "key1");
+ data2.put("apache SeaTunnel", "seatunnelTest");
+ array.add(data2);
+
+ byte[] result = SerializationUtils.serialize(array);
+
+ ArrayList> array2 = SerializationUtils.deserialize(result);
+
+ Assert.assertEquals(array2, array);
+
+ Assert.assertThrows(SerializationException.class, () -> SerializationUtils.deserialize(new byte[]{1, 0, 1}));
+
+ }
+
+}
diff --git a/seatunnel-common/src/test/java/org/apache/seatunnel/common/utils/VariablesSubstituteTest.java b/seatunnel-common/src/test/java/org/apache/seatunnel/common/utils/VariablesSubstituteTest.java
index 33d721ae81f..09d044a5312 100644
--- a/seatunnel-common/src/test/java/org/apache/seatunnel/common/utils/VariablesSubstituteTest.java
+++ b/seatunnel-common/src/test/java/org/apache/seatunnel/common/utils/VariablesSubstituteTest.java
@@ -26,7 +26,6 @@
public class VariablesSubstituteTest {
- @SuppressWarnings("checkstyle:MagicNumber")
@Test
public void testSubstitute() {
String timeFormat = "yyyyMMddHHmmss";
diff --git a/seatunnel-connectors-v2-dist/pom.xml b/seatunnel-connectors-v2-dist/pom.xml
index 9e349fbb99d..fe37965a1b8 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,23 @@
connector-hive
${project.version}
+
+ org.apache.seatunnel
+ connector-file-hadoop
+ ${project.version}
+
+
+ org.apache.seatunnel
+ connector-file-local
+ ${project.version}
+
+
+ org.apache.seatunnel
+ connector-hudi
+ ${project.version}
+
-
diff --git a/seatunnel-connectors-v2/README.md b/seatunnel-connectors-v2/README.md
index 7522b846eb8..b24122d941e 100644
--- a/seatunnel-connectors-v2/README.md
+++ b/seatunnel-connectors-v2/README.md
@@ -5,7 +5,7 @@ This article introduces the new interface and the new code structure on account
In order to separate from the old code, we have defined new modules for execution flow. This facilitates parallel development at the current stage, and reduces the difficulty of merging. All the relevant code at this stage is kept on the ``api-draft`` branch.
### **Example**
-We have prepared a new version of the locally executable example program in ``seatunnel-examples``, which can be directly called using ``seatunnel-flink-new-connector-example`` or ``seatunnel-spark-new-connector-example`` in ``SeaTunnelApiExample``. This is also the debugging method that is often used in the local development of Connector. The corresponding configuration files are saved in the same module ``resources/examples`` folder as before.
+We have prepared a new version of the locally executable example program in ``seatunnel-examples``, which can be directly called using ``seatunnel-flink-connector-v2-example`` or ``seatunnel-spark-connector-v2-example`` in ``SeaTunnelApiExample``. This is also the debugging method that is often used in the local development of Connector. The corresponding configuration files are saved in the same module ``resources/examples`` folder as before.
### **Startup Class**
diff --git a/seatunnel-connectors-v2/README.zh.md b/seatunnel-connectors-v2/README.zh.md
index c0aa4d6395b..017b19907a2 100644
--- a/seatunnel-connectors-v2/README.zh.md
+++ b/seatunnel-connectors-v2/README.zh.md
@@ -4,7 +4,7 @@ Because SeaTunnel design new API for connectors, 所以通过这篇文章来介
现阶段所有相关代码保存在`api-draft`分支上。
为了和老的代码分开,方便现阶段的并行开发,以及降低merge的难度。我们为新的执行流程定义了新的模块
### Example
-我们已经在`seatunnel-examples`中准备好了新版本的可本地执行Example程序,直接调用`seatunnel-flink-new-connector-example`或`seatunnel-spark-new-connector-example`中的`SeaTunnelApiExample`即可。这也是本地开发Connector经常会用到的调试方式。
+我们已经在`seatunnel-examples`中准备好了新版本的可本地执行Example程序,直接调用`seatunnel-flink-connector-v2-example`或`seatunnel-spark-connector-v2-example`中的`SeaTunnelApiExample`即可。这也是本地开发Connector经常会用到的调试方式。
对应的配置文件保存在同模块的`resources/examples`文件夹下,和以前一样。
### 启动类
和老的启动类分开,我们创建了两个新的启动类工程,分别是`seatunnel-core/seatunnel-flink-starter`和`seatunnel-core/seatunnel-spark-starter`. 可以在这里找到如何将配置文件解析为可以执行的Flink/Spark流程。
diff --git a/seatunnel-connectors-v2/connector-assert/src/test/java/org/apache/seatunnel/flink/assertion/AssertExecutorTest.java b/seatunnel-connectors-v2/connector-assert/src/test/java/org/apache/seatunnel/flink/assertion/AssertExecutorTest.java
index c5d30f3b189..fa9fee2093f 100644
--- a/seatunnel-connectors-v2/connector-assert/src/test/java/org/apache/seatunnel/flink/assertion/AssertExecutorTest.java
+++ b/seatunnel-connectors-v2/connector-assert/src/test/java/org/apache/seatunnel/flink/assertion/AssertExecutorTest.java
@@ -29,7 +29,6 @@
import java.util.List;
-@SuppressWarnings("magicnumber")
public class AssertExecutorTest extends TestCase {
SeaTunnelRow row = new SeaTunnelRow(new Object[]{"jared", 17});
SeaTunnelRowType rowType = new SeaTunnelRowType(new String[]{"name", "age"}, new SeaTunnelDataType[]{BasicType.STRING_TYPE, BasicType.INT_TYPE});
diff --git a/seatunnel-connectors-v2/connector-assert/src/test/java/org/apache/seatunnel/flink/assertion/rule/AssertRuleParserTest.java b/seatunnel-connectors-v2/connector-assert/src/test/java/org/apache/seatunnel/flink/assertion/rule/AssertRuleParserTest.java
index 9a9628dedf0..5c1bb650cc2 100644
--- a/seatunnel-connectors-v2/connector-assert/src/test/java/org/apache/seatunnel/flink/assertion/rule/AssertRuleParserTest.java
+++ b/seatunnel-connectors-v2/connector-assert/src/test/java/org/apache/seatunnel/flink/assertion/rule/AssertRuleParserTest.java
@@ -28,7 +28,6 @@
import java.util.List;
-@SuppressWarnings("magicnumber")
public class AssertRuleParserTest extends TestCase {
AssertRuleParser parser = new AssertRuleParser();
diff --git a/seatunnel-connectors-v2/connector-clickhouse/src/main/java/org/apache/seatunnel/connectors/seatunnel/clickhouse/sink/client/ClickhouseSinkWriter.java b/seatunnel-connectors-v2/connector-clickhouse/src/main/java/org/apache/seatunnel/connectors/seatunnel/clickhouse/sink/client/ClickhouseSinkWriter.java
index c8114b6379b..6e2a4ceb190 100644
--- a/seatunnel-connectors-v2/connector-clickhouse/src/main/java/org/apache/seatunnel/connectors/seatunnel/clickhouse/sink/client/ClickhouseSinkWriter.java
+++ b/seatunnel-connectors-v2/connector-clickhouse/src/main/java/org/apache/seatunnel/connectors/seatunnel/clickhouse/sink/client/ClickhouseSinkWriter.java
@@ -193,10 +193,10 @@ private Map initFieldInjectFunctionMap()
new BigDecimalInjectFunction(),
new DateInjectFunction(),
new DateTimeInjectFunction(),
+ new LongInjectFunction(),
new DoubleInjectFunction(),
new FloatInjectFunction(),
new IntInjectFunction(),
- new LongInjectFunction(),
new StringInjectFunction()
);
ClickhouseFieldInjectFunction defaultFunction = new StringInjectFunction();
diff --git a/seatunnel-connectors-v2/connector-clickhouse/src/main/java/org/apache/seatunnel/connectors/seatunnel/clickhouse/source/ClickhouseSource.java b/seatunnel-connectors-v2/connector-clickhouse/src/main/java/org/apache/seatunnel/connectors/seatunnel/clickhouse/source/ClickhouseSource.java
index a94344e72cd..4a4b638d915 100644
--- a/seatunnel-connectors-v2/connector-clickhouse/src/main/java/org/apache/seatunnel/connectors/seatunnel/clickhouse/source/ClickhouseSource.java
+++ b/seatunnel-connectors-v2/connector-clickhouse/src/main/java/org/apache/seatunnel/connectors/seatunnel/clickhouse/source/ClickhouseSource.java
@@ -24,8 +24,6 @@
import static org.apache.seatunnel.connectors.seatunnel.clickhouse.config.Config.USERNAME;
import org.apache.seatunnel.api.common.PrepareFailException;
-import org.apache.seatunnel.api.serialization.DefaultSerializer;
-import org.apache.seatunnel.api.serialization.Serializer;
import org.apache.seatunnel.api.source.Boundedness;
import org.apache.seatunnel.api.source.SeaTunnelSource;
import org.apache.seatunnel.api.source.SourceReader;
@@ -124,9 +122,4 @@ public SourceSplitEnumerator resto
return new ClickhouseSourceSplitEnumerator(enumeratorContext);
}
- @Override
- public Serializer getEnumeratorStateSerializer() {
- return new DefaultSerializer<>();
- }
-
}
diff --git a/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/source/AbstractSingleSplitSource.java b/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/source/AbstractSingleSplitSource.java
index f2968c833e3..54410962bc2 100644
--- a/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/source/AbstractSingleSplitSource.java
+++ b/seatunnel-connectors-v2/connector-common/src/main/java/org/apache/seatunnel/connectors/seatunnel/common/source/AbstractSingleSplitSource.java
@@ -29,7 +29,7 @@ public abstract class AbstractSingleSplitSource implements SeaTunnelSource createReader(SourceReader.Context readerContext) throws Exception {
- checkArgument(readerContext.getIndexOfSubtask() == 0, "Single split source allows only a single reader to be created.");
+ checkArgument(readerContext.getIndexOfSubtask() == 0, "A single split source allows only one single reader to be created.");
return createReader(new SingleSplitReaderContext(readerContext));
}
@@ -45,11 +45,6 @@ public final SourceSplitEnumerator rest
return createEnumerator(enumeratorContext);
}
- @Override
- public final Serializer getEnumeratorStateSerializer() {
- return new DefaultSerializer<>();
- }
-
@Override
public final Serializer getSplitSerializer() {
return new DefaultSerializer<>();
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/BaseTextFileConfig.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/BaseTextFileConfig.java
new file mode 100644
index 00000000000..d6fd26d1b27
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/BaseTextFileConfig.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.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 BaseTextFileConfig 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 BaseTextFileConfig(@NonNull Config config) {
+ 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);
+ }
+ checkNotNull(path);
+
+ if (config.hasPath(Constant.FILE_NAME_EXPRESSION) && !StringUtils.isBlank(config.getString(Constant.FILE_NAME_EXPRESSION))) {
+ this.fileNameExpression = config.getString(Constant.FILE_NAME_EXPRESSION);
+ }
+
+ 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));
+ }
+ }
+
+ public BaseTextFileConfig() {}
+}
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..77b72f004ca
--- /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,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.
+ */
+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 FileSinkWriterWithTransaction(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..c847ff659f2
--- /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,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;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+import java.io.Serializable;
+import java.util.List;
+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;
+
+ private Map> partitionDirAndValsMap;
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/FileCommitInfo.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/FileCommitInfo.java
new file mode 100644
index 00000000000..0fcb04a03fa
--- /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,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.sink;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+import java.io.Serializable;
+import java.util.List;
+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 Map> partitionDirAndValsMap;
+
+ private String transactionDir;
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/FileSinkAggregatedCommitter.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/FileSinkAggregatedCommitter.java
new file mode 100644
index 00000000000..3c7c8cf9c92
--- /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,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.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.apache.commons.collections4.CollectionUtils;
+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;
+import java.util.Set;
+
+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<>();
+ Map> partitionDirAndValsMap = 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());
+ Set>> entries = commitInfo.getPartitionDirAndValsMap().entrySet();
+ if (!CollectionUtils.isEmpty(entries)) {
+ partitionDirAndValsMap.putAll(commitInfo.getPartitionDirAndValsMap());
+ }
+ });
+ return new FileAggregatedCommitInfo(aggregateCommitInfo, partitionDirAndValsMap);
+ }
+
+ @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/FileSinkWriterWithTransaction.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/FileSinkWriterWithTransaction.java
new file mode 100644
index 00000000000..83e51d1bc6a
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/FileSinkWriterWithTransaction.java
@@ -0,0 +1,160 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.sink;
+
+import org.apache.seatunnel.api.sink.SinkWriter;
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.config.TextFileSinkConfig;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.SinkFileSystemPlugin;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.transaction.TransactionStateFileWriter;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.FileSinkPartitionDirNameGenerator;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.FileSinkTransactionFileNameGenerator;
+
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+
+import lombok.NonNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Optional;
+
+public class FileSinkWriterWithTransaction implements SinkWriter {
+ private static final Logger LOGGER = LoggerFactory.getLogger(FileSinkWriterWithTransaction.class);
+
+ private SeaTunnelRowType seaTunnelRowTypeInfo;
+ private Config pluginConfig;
+ private Context context;
+ private String jobId;
+
+ private TransactionStateFileWriter fileWriter;
+
+ private TextFileSinkConfig textFileSinkConfig;
+
+ public FileSinkWriterWithTransaction(@NonNull SeaTunnelRowType seaTunnelRowTypeInfo,
+ @NonNull Config pluginConfig,
+ @NonNull SinkWriter.Context context,
+ @NonNull TextFileSinkConfig textFileSinkConfig,
+ @NonNull String jobId,
+ @NonNull SinkFileSystemPlugin sinkFileSystemPlugin) {
+ this.seaTunnelRowTypeInfo = seaTunnelRowTypeInfo;
+ this.pluginConfig = pluginConfig;
+ this.context = context;
+ this.jobId = jobId;
+ this.textFileSinkConfig = textFileSinkConfig;
+
+ Optional transactionStateFileWriter = sinkFileSystemPlugin.getTransactionStateFileWriter(this.seaTunnelRowTypeInfo,
+ new FileSinkTransactionFileNameGenerator(
+ this.textFileSinkConfig.getFileFormat(),
+ this.textFileSinkConfig.getFileNameExpression(),
+ this.textFileSinkConfig.getFileNameTimeFormat()),
+ new FileSinkPartitionDirNameGenerator(
+ this.textFileSinkConfig.getPartitionFieldList(),
+ this.textFileSinkConfig.getPartitionFieldsIndexInRow(),
+ this.textFileSinkConfig.getPartitionDirExpression()),
+ this.textFileSinkConfig.getSinkColumnsIndexInRow(),
+ this.textFileSinkConfig.getTmpPath(),
+ this.textFileSinkConfig.getPath(),
+ this.jobId,
+ this.context.getIndexOfSubtask(),
+ this.textFileSinkConfig.getFieldDelimiter(),
+ this.textFileSinkConfig.getRowDelimiter(),
+ sinkFileSystemPlugin.getFileSystem().get());
+
+ if (!transactionStateFileWriter.isPresent()) {
+ throw new RuntimeException("A TransactionStateFileWriter is need");
+ }
+
+ this.fileWriter = transactionStateFileWriter.get();
+
+ fileWriter.beginTransaction(1L);
+ }
+
+ public FileSinkWriterWithTransaction(@NonNull SeaTunnelRowType seaTunnelRowTypeInfo,
+ @NonNull Config pluginConfig,
+ @NonNull SinkWriter.Context context,
+ @NonNull TextFileSinkConfig textFileSinkConfig,
+ @NonNull String jobId,
+ @NonNull List 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/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..ce94ff84769
--- /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,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.SeaTunnelRow;
+import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
+import org.apache.seatunnel.connectors.seatunnel.file.config.BaseTextFileConfig;
+import org.apache.seatunnel.connectors.seatunnel.file.config.Constant;
+import org.apache.seatunnel.connectors.seatunnel.file.config.PartitionConfig;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.FileSinkPartitionDirNameGenerator;
+
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+
+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 BaseTextFileConfig implements PartitionConfig {
+
+ private List sinkColumnList;
+
+ private List partitionFieldList;
+
+ /**
+ * default is ${k0}=${v0}/${k1}=${v1}/... {@link FileSinkPartitionDirNameGenerator#generatorPartitionDir(SeaTunnelRow)} ()}
+ */
+ 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.SINK_COLUMNS) && !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..d9a39c5df3b
--- /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,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 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 org.apache.seatunnel.connectors.seatunnel.file.sink.TransactionStateFileSinkWriter#prepareCommit()} or {@link org.apache.seatunnel.connectors.seatunnel.file.sink.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 org.apache.seatunnel.connectors.seatunnel.file.sink.TransactionStateFileSinkWriter#prepareCommit()}
+ * We should end the transaction in this method. After this method is called, the transaction will no longer accept data writing
+ *
+ * @return Return the commit information that can be commit in {@link FileSinkAggregatedCommitter#commit(List)}
+ */
+ 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..b14827b5671
--- /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,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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;
+
+ private Map> partitionDirAndValsMap;
+
+ 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) {
+ Map> dataPartitionDirAndValsMap = this.partitionDirNameGenerator.generatorPartitionDir(seaTunnelRow);
+ String beingWrittenFileKey = dataPartitionDirAndValsMap.keySet().toArray()[0].toString();
+ // 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);
+ if (!Constant.NON_PARTITION.equals(dataPartitionDirAndValsMap.keySet().toArray()[0].toString())){
+ partitionDirAndValsMap.putAll(dataPartitionDirAndValsMap);
+ }
+ 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.partitionDirAndValsMap = 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);
+
+ Map> copyMap = this.partitionDirAndValsMap.entrySet().stream()
+ .collect(Collectors.toMap(e -> e.getKey(), e -> new ArrayList(e.getValue())));
+ return Optional.of(new FileCommitInfo(commitMap, copyMap, 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..a9175409fab
--- /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,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.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.ArrayList;
+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 Map> generatorPartitionDir(SeaTunnelRow seaTunnelRow) {
+ Map> partitionDirAndValsMap = new HashMap<>(1);
+ if (CollectionUtils.isEmpty(this.partitionFieldsIndexInRow)) {
+ partitionDirAndValsMap.put(Constant.NON_PARTITION, null);
+ return partitionDirAndValsMap;
+ }
+
+ List vals = new ArrayList<>(partitionFieldsIndexInRow.size());
+ String partitionDir;
+ 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("/");
+ vals.add(seaTunnelRow.getFields()[partitionFieldsIndexInRow.get(i)].toString());
+ }
+ partitionDir = 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());
+ vals.add(seaTunnelRow.getFields()[partitionFieldsIndexInRow.get(i)].toString());
+ }
+ partitionDir = VariablesSubstitute.substitute(partitionDirExpression, valueMap);
+ }
+
+ partitionDirAndValsMap.put(partitionDir, vals);
+ return partitionDirAndValsMap;
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/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-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/file/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
similarity index 76%
rename from seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/file/writer/FileWriter.java
rename to seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/FileWriter.java
index 8ee8777a1e7..276c981fa64 100644
--- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/file/writer/FileWriter.java
+++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/FileWriter.java
@@ -15,21 +15,18 @@
* limitations under the License.
*/
-package org.apache.seatunnel.connectors.seatunnel.hive.sink.file.writer;
+package org.apache.seatunnel.connectors.seatunnel.file.sink.writer;
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
import lombok.NonNull;
-import java.util.Map;
+import java.io.Serializable;
-public interface FileWriter {
+public interface FileWriter extends Serializable {
void write(@NonNull SeaTunnelRow seaTunnelRow);
- @NonNull
- Map getNeedMoveFiles();
-
/**
* In this method we need finish write the file. The following operations are often required:
* 1. Flush memory to disk.
@@ -37,13 +34,4 @@ public interface FileWriter {
* 3. Add the mapping relationship between seatunnel file path and hive file path to needMoveFiles.
*/
void finishAndCloseWriteFile();
-
- /**
- * The writer needs to be reset after each checkpoint is completed
- *
- * @param checkpointId checkpointId
- */
- void resetFileWriter(@NonNull String checkpointId);
-
- void abort();
}
diff --git a/seatunnel-connectors-v2/connector-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..05c90256bda
--- /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,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.file.sink.writer;
+
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+
+public interface PartitionDirNameGenerator extends Serializable {
+ Map> 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..0867f104ad0
--- /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).keySet().toArray()[0].toString();
+ Assert.assertEquals("test/3", partitionDir);
+
+ partitionDirNameGenerator = new FileSinkPartitionDirNameGenerator(partitionFieldList, partitionFieldsIndexInRow, "${k0}=${v0}/${k1}=${v1}");
+ partitionDir = partitionDirNameGenerator.generatorPartitionDir(seaTunnelRow).keySet().toArray()[0].toString();
+ Assert.assertEquals("c3=test/c4=3", partitionDir);
+
+ partitionDirNameGenerator = new FileSinkPartitionDirNameGenerator(null, null, "${k0}=${v0}/${k1}=${v1}");
+ partitionDir = partitionDirNameGenerator.generatorPartitionDir(seaTunnelRow).keySet().toArray()[0].toString();
+ 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-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/file/writer/HdfsTxtFileWriter.java b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/HdfsTxtTransactionStateFileWriter.java
similarity index 57%
rename from seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/file/writer/HdfsTxtFileWriter.java
rename to seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/HdfsTxtTransactionStateFileWriter.java
index 71b26568fd7..81882c414c6 100644
--- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/file/writer/HdfsTxtFileWriter.java
+++ b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/HdfsTxtTransactionStateFileWriter.java
@@ -15,98 +15,75 @@
* limitations under the License.
*/
-package org.apache.seatunnel.connectors.seatunnel.hive.sink.file.writer;
+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.hive.sink.HiveSinkConfig;
+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.Lombok;
import lombok.NonNull;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
-import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
-import java.util.stream.IntStream;
-public class HdfsTxtFileWriter extends AbstractFileWriter {
- private static final Logger LOGGER = LoggerFactory.getLogger(HdfsTxtFileWriter.class);
+public class HdfsTxtTransactionStateFileWriter extends AbstractTransactionStateFileWriter {
+ private static final Logger LOGGER = LoggerFactory.getLogger(HdfsTxtTransactionStateFileWriter.class);
private Map beingWrittenOutputStream;
- protected final int[] sinkColumnIndexes;
- public HdfsTxtFileWriter(SeaTunnelRowType seaTunnelRowType,
- HiveSinkConfig hiveSinkConfig,
- long sinkId,
- int subTaskIndex) {
- super(seaTunnelRowType, hiveSinkConfig, sinkId, subTaskIndex);
+ 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<>();
- List sinkColumns = hiveSinkConfig.getSinkColumns();
- if (sinkColumns == null || sinkColumns.size() == 0) {
- this.sinkColumnIndexes = IntStream.range(0, seaTunnelRowType.getTotalFields()).toArray();
- } else {
- this.sinkColumnIndexes = IntStream.range(0, seaTunnelRowType.getTotalFields())
- .filter(i -> sinkColumns.contains(seaTunnelRowType.getFieldName(i)))
- .toArray();
- }
- }
-
- @Override
- @NonNull
- public String getFileSuffix() {
- return "txt";
}
@Override
- public void resetMoreFileWriter(@NonNull String checkpointId) {
+ public void beginTransaction(String transactionId) {
this.beingWrittenOutputStream = new HashMap<>();
}
@Override
- public void abortMore() {
- // delete files
- beingWrittenOutputStream.keySet().stream().forEach(file -> {
- try {
- boolean deleted = HdfsUtils.deleteFile(file);
- if (!deleted) {
- LOGGER.error("delete file {} error", file);
- throw new IOException(String.format("delete file {} error", file));
- }
- } catch (IOException e) {
- LOGGER.error("delete file {} error", file);
- throw new RuntimeException(e);
- }
- });
-
+ public void abortTransaction(String transactionId) {
this.beingWrittenOutputStream = new HashMap<>();
}
@Override
public void write(@NonNull SeaTunnelRow seaTunnelRow) {
- Lombok.checkNotNull(seaTunnelRow, "seaTunnelRow is null");
String filePath = getOrCreateFilePathBeingWritten(seaTunnelRow);
FSDataOutputStream fsDataOutputStream = getOrCreateOutputStream(filePath);
String line = transformRowToLine(seaTunnelRow);
try {
fsDataOutputStream.write(line.getBytes());
- fsDataOutputStream.write(hiveSinkConfig.getHiveTxtFileLineDelimiter().getBytes());
+ fsDataOutputStream.write(rowDelimiter.getBytes());
} catch (IOException e) {
LOGGER.error("write data to file {} error", filePath);
throw new RuntimeException(e);
}
}
- @NonNull
- @Override
- public Map getNeedMoveFiles() {
- return this.needMoveFiles;
- }
-
@Override
public void finishAndCloseWriteFile() {
beingWrittenOutputStream.entrySet().forEach(entry -> {
@@ -123,7 +100,7 @@ public void finishAndCloseWriteFile() {
}
}
- needMoveFiles.put(entry.getKey(), getHiveLocation(entry.getKey()));
+ needMoveFiles.put(entry.getKey(), getTargetLocation(entry.getKey()));
});
}
@@ -142,10 +119,6 @@ private FSDataOutputStream getOrCreateOutputStream(@NonNull String filePath) {
}
private String transformRowToLine(@NonNull SeaTunnelRow seaTunnelRow) {
- return Arrays.stream(sinkColumnIndexes)
- .boxed()
- .map(seaTunnelRow::getField)
- .map(value -> value == null ? "" : value.toString())
- .collect(Collectors.joining(hiveSinkConfig.getHiveTxtFileFieldDelimiter()));
+ 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-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/file/writer/HdfsUtils.java b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/HdfsUtils.java
similarity index 60%
rename from seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/file/writer/HdfsUtils.java
rename to seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/HdfsUtils.java
index 23b1e5843f2..421c7f7ebbd 100644
--- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/file/writer/HdfsUtils.java
+++ b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/hdfs/HdfsUtils.java
@@ -15,17 +15,22 @@
* limitations under the License.
*/
-package org.apache.seatunnel.connectors.seatunnel.hive.sink.file.writer;
+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);
@@ -35,9 +40,11 @@ public class HdfsUtils {
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");
- conf.set("fs.defaultFs", path);
- return FileSystem.get(conf);
+ return FileSystem.get(URI.create(path), conf);
}
public static FSDataOutputStream getOutputStream(@NonNull String outFilePath) throws IOException {
@@ -47,9 +54,19 @@ public static FSDataOutputStream getOutputStream(@NonNull String outFilePath) th
return fsDataOutputStream;
}
- public static boolean deleteFile(@NonNull String file) throws IOException {
+ 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);
- return hdfsFs.delete(new Path(file), true);
+ if (!hdfsFs.delete(new Path(file), true)) {
+ throw new IOException("delete file " + file + " error");
+ }
}
/**
@@ -74,17 +91,22 @@ public static void renameFile(@NonNull String oldName, @NonNull String newName,
if (!fileExist(newName.substring(0, newName.lastIndexOf("/")))) {
createDir(newName.substring(0, newName.lastIndexOf("/")));
}
- LOGGER.info("rename file :[" + oldPath + "] to [" + newPath + "] finish");
- hdfsFs.rename(oldPath, newPath);
+ 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 boolean createDir(@NonNull String filePath)
+ public static void createDir(@NonNull String filePath)
throws IOException {
FileSystem hdfsFs = getHdfsFs(filePath);
Path dfs = new Path(filePath);
- return hdfsFs.mkdirs(dfs);
+ if (!hdfsFs.mkdirs(dfs)) {
+ throw new IOException("create dir " + filePath + " error");
+ }
}
public static boolean fileExist(@NonNull String filePath)
@@ -93,4 +115,23 @@ public static boolean fileExist(@NonNull String 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..3262ccd1cb2
--- /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,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.stream.Collectors;
+
+public class FileSinkAggregatedCommitterTest {
+ @SuppressWarnings("checkstyle:UnnecessaryParentheses")
+ @Test
+ public void testCommit() throws Exception {
+ FileSinkAggregatedCommitter fileSinkAggregatedCommitter = new FileSinkAggregatedCommitter(new HdfsFileSystemCommitter());
+ 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);
+
+ Map> partitionDirAndVals = new HashMap<>();
+ partitionDirAndVals.put("/c3=4/c4=rrr", Arrays.stream((new String[]{"4", "rrr"})).collect(Collectors.toList()));
+ partitionDirAndVals.put("/c3=4/c4=bbb", Arrays.stream((new String[]{"4", "bbb"})).collect(Collectors.toList()));
+
+ FileAggregatedCommitInfo fileAggregatedCommitInfo = new FileAggregatedCommitInfo(transactionFiles, partitionDirAndVals);
+ List 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:UnnecessaryParentheses")
+ @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=3/c4=rrr/test1.txt", targetDir + "/c3=3/c4=rrr/test1.txt");
+ needMoveFiles.put(transactionDir + "/c3=4/c4=bbb/test1.txt", targetDir + "/c3=4/c4=bbb/test1.txt");
+ Map> partitionDirAndVals = new HashMap<>();
+ partitionDirAndVals.put("/c3=3/c4=rrr", Arrays.stream((new String[]{"3", "rrr"})).collect(Collectors.toList()));
+ partitionDirAndVals.put("/c3=4/c4=bbb", Arrays.stream((new String[]{"4", "bbb"})).collect(Collectors.toList()));
+ FileCommitInfo fileCommitInfo = new FileCommitInfo(needMoveFiles, partitionDirAndVals, transactionDir);
+ HdfsUtils.createFile(transactionDir + "/c3=3/c4=rrr/test1.txt");
+ HdfsUtils.createFile(transactionDir + "/c3=4/c4=bbb/test1.txt");
+
+ Map 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");
+ Map> partitionDirAndVals1 = new HashMap<>();
+ partitionDirAndVals.put("/c3=4/c4=rrr", Arrays.stream((new String[]{"4", "rrr"})).collect(Collectors.toList()));
+ partitionDirAndVals.put("/c3=4/c4=bbb", Arrays.stream((new String[]{"4", "bbb"})).collect(Collectors.toList()));
+ FileCommitInfo fileCommitInfo1 = new FileCommitInfo(needMoveFiles1, partitionDirAndVals1, transactionDir);
+ List 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=3/c4=rrr/test1.txt", combine.getTransactionMap().get(transactionDir).get(transactionDir + "/c3=3/c4=rrr/test1.txt"));
+ Assert.assertEquals(targetDir + "/c3=4/c4=bbb/test1.txt", combine.getTransactionMap().get(transactionDir).get(transactionDir + "/c3=4/c4=bbb/test1.txt"));
+ Assert.assertEquals(targetDir + "/c3=4/c4=rrr/test2.txt", combine.getTransactionMap().get(transactionDir).get(transactionDir + "/c3=4/c4=rrr/test2.txt"));
+ Assert.assertEquals(targetDir + "/c3=4/c4=bbb/test2.txt", combine.getTransactionMap().get(transactionDir).get(transactionDir + "/c3=4/c4=bbb/test2.txt"));
+ Assert.assertEquals(3, combine.getPartitionDirAndValsMap().keySet().size());
+ }
+
+ @SuppressWarnings("checkstyle:UnnecessaryParentheses")
+ @Test
+ public void testAbort() throws Exception {
+ FileSinkAggregatedCommitter fileSinkAggregatedCommitter = new FileSinkAggregatedCommitter(new HdfsFileSystemCommitter());
+ 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");
+ Map> partitionDirAndVals = new HashMap<>();
+ partitionDirAndVals.put("/c3=4/c4=rrr", Arrays.stream((new String[]{"4", "rrr"})).collect(Collectors.toList()));
+ partitionDirAndVals.put("/c3=4/c4=bbb", Arrays.stream((new String[]{"4", "bbb"})).collect(Collectors.toList()));
+ HdfsUtils.createFile(transactionDir + "/c3=4/c4=rrr/test1.txt");
+ HdfsUtils.createFile(transactionDir + "/c3=4/c4=bbb/test1.txt");
+
+ transactionFiles.put(transactionDir, needMoveFiles);
+ FileAggregatedCommitInfo fileAggregatedCommitInfo = new FileAggregatedCommitInfo(transactionFiles, partitionDirAndVals);
+ 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..d8633e86d16
--- /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,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.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"));
+
+ Map> partitionDirAndValsMap = fileCommitInfo.getPartitionDirAndValsMap();
+ Assert.assertEquals(2, partitionDirAndValsMap.size());
+ Assert.assertTrue(partitionDirAndValsMap.keySet().contains("c3=str1/c4=str2"));
+ Assert.assertTrue(partitionDirAndValsMap.keySet().contains("c3=str1/c4=str3"));
+ Assert.assertTrue(partitionDirAndValsMap.get("c3=str1/c4=str2").size() == 2);
+ Assert.assertEquals("str1", partitionDirAndValsMap.get("c3=str1/c4=str2").get(0));
+ Assert.assertEquals("str2", partitionDirAndValsMap.get("c3=str1/c4=str2").get(1));
+ Assert.assertEquals("str1", partitionDirAndValsMap.get("c3=str1/c4=str3").get(0));
+ Assert.assertEquals("str3", partitionDirAndValsMap.get("c3=str1/c4=str3").get(1));
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-file/connector-file-local/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..720ad3eba78
--- /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,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.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.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.stream.Collectors;
+
+public class FileSinkAggregatedCommitterTest {
+ @SuppressWarnings("checkstyle:UnnecessaryParentheses")
+ 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);
+
+ Map> partitionDirAndVals = new HashMap<>();
+ partitionDirAndVals.put("/c3=4/c4=rrr", Arrays.stream((new String[]{"4", "rrr"})).collect(Collectors.toList()));
+ partitionDirAndVals.put("/c3=4/c4=bbb", Arrays.stream((new String[]{"4", "bbb"})).collect(Collectors.toList()));
+
+ FileAggregatedCommitInfo fileAggregatedCommitInfo = new FileAggregatedCommitInfo(transactionFiles, partitionDirAndVals);
+ List 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:UnnecessaryParentheses")
+ @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=3/c4=rrr/test1.txt", targetDir + "/c3=3/c4=rrr/test1.txt");
+ needMoveFiles.put(transactionDir + "/c3=4/c4=bbb/test1.txt", targetDir + "/c3=4/c4=bbb/test1.txt");
+ Map> partitionDirAndVals = new HashMap<>();
+ partitionDirAndVals.put("/c3=3/c4=rrr", Arrays.stream((new String[]{"3", "rrr"})).collect(Collectors.toList()));
+ partitionDirAndVals.put("/c3=4/c4=bbb", Arrays.stream((new String[]{"4", "bbb"})).collect(Collectors.toList()));
+ FileCommitInfo fileCommitInfo = new FileCommitInfo(needMoveFiles, partitionDirAndVals, transactionDir);
+ FileUtils.createFile(transactionDir + "/c3=3/c4=rrr/test1.txt");
+ FileUtils.createFile(transactionDir + "/c3=4/c4=bbb/test1.txt");
+
+ Map 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");
+ Map> partitionDirAndVals1 = new HashMap<>();
+ partitionDirAndVals.put("/c3=4/c4=rrr", Arrays.stream((new String[]{"4", "rrr"})).collect(Collectors.toList()));
+ partitionDirAndVals.put("/c3=4/c4=bbb", Arrays.stream((new String[]{"4", "bbb"})).collect(Collectors.toList()));
+ FileCommitInfo fileCommitInfo1 = new FileCommitInfo(needMoveFiles1, partitionDirAndVals1, transactionDir);
+ List 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=3/c4=rrr/test1.txt", combine.getTransactionMap().get(transactionDir).get(transactionDir + "/c3=3/c4=rrr/test1.txt"));
+ Assert.assertEquals(targetDir + "/c3=4/c4=bbb/test1.txt", combine.getTransactionMap().get(transactionDir).get(transactionDir + "/c3=4/c4=bbb/test1.txt"));
+ Assert.assertEquals(targetDir + "/c3=4/c4=rrr/test2.txt", combine.getTransactionMap().get(transactionDir).get(transactionDir + "/c3=4/c4=rrr/test2.txt"));
+ Assert.assertEquals(targetDir + "/c3=4/c4=bbb/test2.txt", combine.getTransactionMap().get(transactionDir).get(transactionDir + "/c3=4/c4=bbb/test2.txt"));
+ Assert.assertEquals(3, combine.getPartitionDirAndValsMap().keySet().size());
+ }
+
+ @SuppressWarnings("checkstyle:UnnecessaryParentheses")
+ @Test
+ public void testAbort() throws Exception {
+ FileSinkAggregatedCommitter fileSinkAggregatedCommitter = new FileSinkAggregatedCommitter(new LocalFileSystemCommitter());
+ 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");
+ Map> partitionDirAndVals = new HashMap<>();
+ partitionDirAndVals.put("/c3=4/c4=rrr", Arrays.stream((new String[]{"4", "rrr"})).collect(Collectors.toList()));
+ partitionDirAndVals.put("/c3=4/c4=bbb", Arrays.stream((new String[]{"4", "bbb"})).collect(Collectors.toList()));
+ FileUtils.createFile(transactionDir + "/c3=4/c4=rrr/test1.txt");
+ FileUtils.createFile(transactionDir + "/c3=4/c4=bbb/test1.txt");
+
+ transactionFiles.put(transactionDir, needMoveFiles);
+ FileAggregatedCommitInfo fileAggregatedCommitInfo = new FileAggregatedCommitInfo(transactionFiles, partitionDirAndVals);
+ 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));
+
+ 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..d739ac41c2d
--- /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,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.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"));
+
+ Map> partitionDirAndValsMap = fileCommitInfo.getPartitionDirAndValsMap();
+ Assert.assertEquals(2, partitionDirAndValsMap.size());
+ Assert.assertTrue(partitionDirAndValsMap.keySet().contains("c3=str1/c4=str2"));
+ Assert.assertTrue(partitionDirAndValsMap.keySet().contains("c3=str1/c4=str3"));
+ Assert.assertTrue(partitionDirAndValsMap.get("c3=str1/c4=str2").size() == 2);
+ Assert.assertEquals("str1", partitionDirAndValsMap.get("c3=str1/c4=str2").get(0));
+ Assert.assertEquals("str2", partitionDirAndValsMap.get("c3=str1/c4=str2").get(1));
+ Assert.assertEquals("str1", partitionDirAndValsMap.get("c3=str1/c4=str3").get(0));
+ Assert.assertEquals("str3", partitionDirAndValsMap.get("c3=str1/c4=str3").get(1));
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-file/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 eb2ffa797de..a4842e0ed6b 100644
--- a/seatunnel-connectors-v2/connector-hive/pom.xml
+++ b/seatunnel-connectors-v2/connector-hive/pom.xml
@@ -30,6 +30,12 @@
connector-hive
+
+ org.apache.hive
+ hive-exec
+ provided
+
+
org.apache.seatunnel
seatunnel-api
@@ -43,14 +49,62 @@
provided
+
+ org.apache.seatunnel
+ seatunnel-core-base
+ ${project.version}
+ test
+
+
org.apache.commons
commons-lang3
+
+ org.apache.orc
+ orc-core
+
+
+ org.apache.hadoop
+ hadoop-common
+
+
+ org.apache.hadoop
+ hadoop-hdfs
+
+
+
+
+ org.apache.commons
+ commons-collections4
+
+
+
+ org.apache.seatunnel
+ connector-file-hadoop
+ ${project.version}
+
+
+
+ org.codehaus.janino
+ janino
+ ${janino.version}
+
+
junit
junit
+
+ org.powermock
+ powermock-module-junit4
+ test
+
+
+ org.powermock
+ powermock-api-mockito2
+ test
+
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/Constant.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/Constant.java
new file mode 100644
index 00000000000..dd0b2ab5465
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/Constant.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.hive.config;
+
+public class Constant {
+ public static final String HIVE_RESULT_TABLE_NAME = "hive_table_name";
+ public static final String HIVE_METASTORE_URIS = "hive_metastore_uris";
+}
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/SourceConfig.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/SourceConfig.java
new file mode 100644
index 00000000000..7054403164b
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/SourceConfig.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.hive.config;
+
+import static org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY;
+
+public class SourceConfig {
+
+ public static final String FILE_TYPE = "file.type";
+
+ public static final String DEFAULT_FS = FS_DEFAULT_NAME_KEY;
+
+ public static final String FILE_PATH = "file.path";
+}
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/exception/HivePluginException.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/exception/HivePluginException.java
new file mode 100644
index 00000000000..8e1c22548e3
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/exception/HivePluginException.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.hive.exception;
+
+public class HivePluginException extends Exception{
+
+ public HivePluginException(String message) {
+ super(message);
+ }
+
+ public HivePluginException(String message, Throwable cause) {
+ super(message, cause);
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveAggregatedCommitInfo.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveAggregatedCommitInfo.java
index 025fbefbfa2..6259389c4c6 100644
--- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveAggregatedCommitInfo.java
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveAggregatedCommitInfo.java
@@ -17,20 +17,18 @@
package org.apache.seatunnel.connectors.seatunnel.hive.sink;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.FileAggregatedCommitInfo;
+
import lombok.AllArgsConstructor;
import lombok.Data;
+import org.apache.hadoop.hive.metastore.api.Table;
import java.io.Serializable;
-import java.util.Map;
@Data
@AllArgsConstructor
public class HiveAggregatedCommitInfo implements Serializable {
-
- /**
- * Storage the commit info in map.
- * K is the file path need to be moved to hive data dir.
- * V is the target file path of the data file.
- */
- private Map needMoveFiles;
+ private FileAggregatedCommitInfo fileAggregatedCommitInfo;
+ private String hiveMetastoreUris;
+ private Table table;
}
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveCommitInfo.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveCommitInfo.java
index 0dd58f8f49f..002beea322f 100644
--- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveCommitInfo.java
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveCommitInfo.java
@@ -17,20 +17,22 @@
package org.apache.seatunnel.connectors.seatunnel.hive.sink;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.FileCommitInfo;
+
import lombok.AllArgsConstructor;
import lombok.Data;
+import org.apache.hadoop.hive.metastore.api.Table;
import java.io.Serializable;
-import java.util.Map;
@Data
@AllArgsConstructor
public class HiveCommitInfo implements Serializable {
- /**
- * Storage the commit info in map.
- * K is the file path need to be moved to hive data dir.
- * V is the target file path of the data file.
- */
- private Map needMoveFiles;
+ private FileCommitInfo fileCommitInfo;
+
+ private String hiveMetastoreUris;
+
+ private Table table;
+
}
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSink.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSink.java
index f1ba12edd95..4df91b1a535 100644
--- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSink.java
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSink.java
@@ -18,6 +18,7 @@
package org.apache.seatunnel.connectors.seatunnel.hive.sink;
import org.apache.seatunnel.api.common.PrepareFailException;
+import org.apache.seatunnel.api.common.SeaTunnelContext;
import org.apache.seatunnel.api.serialization.DefaultSerializer;
import org.apache.seatunnel.api.serialization.Serializer;
import org.apache.seatunnel.api.sink.SeaTunnelSink;
@@ -26,6 +27,8 @@
import org.apache.seatunnel.api.table.type.SeaTunnelDataType;
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
+import org.apache.seatunnel.common.constants.JobMode;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.config.SaveMode;
import org.apache.seatunnel.shade.com.typesafe.config.Config;
@@ -43,8 +46,11 @@
public class HiveSink implements SeaTunnelSink {
private Config config;
- private long jobId;
- private SeaTunnelRowType seaTunnelRowType;
+ private String jobId;
+ private Long checkpointId;
+ private SeaTunnelRowType seaTunnelRowTypeInfo;
+ private SeaTunnelContext seaTunnelContext;
+ private HiveSinkConfig hiveSinkConfig;
@Override
public String getPluginName() {
@@ -52,34 +58,47 @@ public String getPluginName() {
}
@Override
- public void setTypeInfo(SeaTunnelRowType seaTunnelRowType) {
- this.seaTunnelRowType = seaTunnelRowType;
+ public void setTypeInfo(SeaTunnelRowType seaTunnelRowTypeInfo) {
+ this.seaTunnelRowTypeInfo = seaTunnelRowTypeInfo;
+ this.hiveSinkConfig = new HiveSinkConfig(config, seaTunnelRowTypeInfo);
}
@Override
public SeaTunnelDataType getConsumedType() {
- return this.seaTunnelRowType;
+ return this.seaTunnelRowTypeInfo;
}
@Override
public void prepare(Config pluginConfig) throws PrepareFailException {
this.config = pluginConfig;
- this.jobId = System.currentTimeMillis();
+ this.checkpointId = 1L;
}
@Override
public SinkWriter createWriter(SinkWriter.Context context) throws IOException {
- return new HiveSinkWriter(seaTunnelRowType, config, context, System.currentTimeMillis());
+ if (!seaTunnelContext.getJobMode().equals(JobMode.BATCH) && hiveSinkConfig.getTextFileSinkConfig().getSaveMode().equals(SaveMode.OVERWRITE)) {
+ throw new RuntimeException("only batch job can overwrite hive table");
+ }
+
+ if (!this.getSinkConfig().getTextFileSinkConfig().isEnableTransaction()) {
+ throw new RuntimeException("Hive Sink Connector only support transaction now");
+ }
+ return new HiveSinkWriter(seaTunnelRowTypeInfo,
+ config,
+ context,
+ getSinkConfig(),
+ jobId);
}
@Override
public SinkWriter restoreWriter(SinkWriter.Context context, List states) throws IOException {
- return new HiveSinkWriter(seaTunnelRowType, config, context, System.currentTimeMillis());
+ return new HiveSinkWriter(seaTunnelRowTypeInfo, config, context, hiveSinkConfig, jobId, states);
}
@Override
- public Optional> getCommitInfoSerializer() {
- return Optional.of(new DefaultSerializer<>());
+ public void setSeaTunnelContext(SeaTunnelContext seaTunnelContext) {
+ this.seaTunnelContext = seaTunnelContext;
+ this.jobId = seaTunnelContext.getJobId();
}
@Override
@@ -87,8 +106,25 @@ 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 HiveSinkConfig getSinkConfig() {
+ if (this.hiveSinkConfig == null && (this.seaTunnelRowTypeInfo != null && this.config != null)) {
+ this.hiveSinkConfig = new HiveSinkConfig(config, seaTunnelRowTypeInfo);
+ }
+ return this.hiveSinkConfig;
+ }
}
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkAggregatedCommitter.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkAggregatedCommitter.java
index 673923a51b1..3a0448179a7 100644
--- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkAggregatedCommitter.java
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkAggregatedCommitter.java
@@ -18,8 +18,14 @@
package org.apache.seatunnel.connectors.seatunnel.hive.sink;
import org.apache.seatunnel.api.sink.SinkAggregatedCommitter;
-import org.apache.seatunnel.connectors.seatunnel.hive.sink.file.writer.HdfsUtils;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.FileAggregatedCommitInfo;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.hdfs.HdfsUtils;
+import org.apache.seatunnel.connectors.seatunnel.hive.utils.HiveMetaStoreProxy;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.thrift.TException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -28,41 +34,82 @@
import java.util.HashMap;
import java.util.List;
import java.util.Map;
+import java.util.Set;
public class HiveSinkAggregatedCommitter implements SinkAggregatedCommitter {
private static final Logger LOGGER = LoggerFactory.getLogger(HiveSinkAggregatedCommitter.class);
@Override
public List commit(List aggregatedCommitInfoList) throws IOException {
- if (aggregatedCommitInfoList == null || aggregatedCommitInfoList.size() == 0) {
+ LOGGER.info("=============================agg commit=================================");
+ if (CollectionUtils.isEmpty(aggregatedCommitInfoList)) {
return null;
}
List errorAggregatedCommitInfoList = new ArrayList();
- aggregatedCommitInfoList.stream().forEach(aggregateCommitInfo -> {
- try {
- Map needMoveFiles = aggregateCommitInfo.getNeedMoveFiles();
- for (Map.Entry entry : needMoveFiles.entrySet()) {
- HdfsUtils.renameFile(entry.getKey(), entry.getValue(), true);
+ HiveMetaStoreProxy hiveMetaStoreProxy = new HiveMetaStoreProxy(aggregatedCommitInfoList.get(0).getHiveMetastoreUris());
+ HiveMetaStoreClient hiveMetaStoreClient = hiveMetaStoreProxy.getHiveMetaStoreClient();
+ try {
+ aggregatedCommitInfoList.stream().forEach(aggregateCommitInfo -> {
+ try {
+ for (Map.Entry> entry : aggregateCommitInfo.getFileAggregatedCommitInfo().getTransactionMap().entrySet()) {
+ // rollback the file
+ for (Map.Entry mvFileEntry : entry.getValue().entrySet()) {
+ HdfsUtils.renameFile(mvFileEntry.getKey(), mvFileEntry.getValue(), true);
+ }
+ // delete the transaction dir
+ HdfsUtils.deleteFile(entry.getKey());
+ }
+ // add hive partition
+ aggregateCommitInfo.getFileAggregatedCommitInfo().getPartitionDirAndValsMap().entrySet().forEach(entry -> {
+ Partition part = new Partition();
+ part.setDbName(aggregateCommitInfo.getTable().getDbName());
+ part.setTableName(aggregateCommitInfo.getTable().getTableName());
+ part.setValues(entry.getValue());
+ part.setParameters(new HashMap<>());
+ part.setSd(aggregateCommitInfo.getTable().getSd().deepCopy());
+ part.getSd().setSerdeInfo(aggregateCommitInfo.getTable().getSd().getSerdeInfo());
+ part.getSd().setLocation(aggregateCommitInfo.getTable().getSd().getLocation() + "/" + entry.getKey());
+ try {
+ hiveMetaStoreClient.add_partition(part);
+ } catch (TException e) {
+ throw new RuntimeException(e);
+ }
+ });
+ } catch (Exception e) {
+ LOGGER.error("commit aggregateCommitInfo error ", e);
+ errorAggregatedCommitInfoList.add(aggregateCommitInfo);
}
- } catch (IOException e) {
- LOGGER.error("commit aggregateCommitInfo error ", e);
- errorAggregatedCommitInfoList.add(aggregateCommitInfo);
- }
- });
+ });
+ } finally {
+ hiveMetaStoreClient.close();
+ }
return errorAggregatedCommitInfoList;
}
@Override
public HiveAggregatedCommitInfo combine(List commitInfos) {
- if (commitInfos == null || commitInfos.size() == 0) {
+ if (CollectionUtils.isEmpty(commitInfos)) {
return null;
}
- Map aggregateCommitInfo = new HashMap<>();
+ Map> aggregateCommitInfo = new HashMap<>();
+ Map> partitionDirAndValsMap = new HashMap<>();
commitInfos.stream().forEach(commitInfo -> {
- aggregateCommitInfo.putAll(commitInfo.getNeedMoveFiles());
+ Map needMoveFileMap = aggregateCommitInfo.get(commitInfo.getFileCommitInfo().getTransactionDir());
+ if (needMoveFileMap == null) {
+ needMoveFileMap = new HashMap<>();
+ aggregateCommitInfo.put(commitInfo.getFileCommitInfo().getTransactionDir(), needMoveFileMap);
+ }
+ needMoveFileMap.putAll(commitInfo.getFileCommitInfo().getNeedMoveFiles());
+ Set>> entries = commitInfo.getFileCommitInfo().getPartitionDirAndValsMap().entrySet();
+ if (!CollectionUtils.isEmpty(entries)) {
+ partitionDirAndValsMap.putAll(commitInfo.getFileCommitInfo().getPartitionDirAndValsMap());
+ }
});
- return new HiveAggregatedCommitInfo(aggregateCommitInfo);
+ return new HiveAggregatedCommitInfo(
+ new FileAggregatedCommitInfo(aggregateCommitInfo, partitionDirAndValsMap),
+ commitInfos.get(0).getHiveMetastoreUris(),
+ commitInfos.get(0).getTable());
}
@Override
@@ -72,9 +119,17 @@ public void abort(List aggregatedCommitInfoList) throw
}
aggregatedCommitInfoList.stream().forEach(aggregateCommitInfo -> {
try {
- Map needMoveFiles = aggregateCommitInfo.getNeedMoveFiles();
- for (Map.Entry entry : needMoveFiles.entrySet()) {
- HdfsUtils.renameFile(entry.getValue(), entry.getKey(), true);
+ for (Map.Entry> entry : aggregateCommitInfo.getFileAggregatedCommitInfo().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());
+
+ // The partitions that have been added will be preserved and will not be deleted
}
} catch (IOException e) {
LOGGER.error("abort aggregateCommitInfo error ", e);
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkConfig.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkConfig.java
index 2e08862f7b2..a37dd3a51a2 100644
--- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkConfig.java
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkConfig.java
@@ -17,90 +17,132 @@
package org.apache.seatunnel.connectors.seatunnel.hive.sink;
+import static org.apache.seatunnel.connectors.seatunnel.file.config.Constant.FIELD_DELIMITER;
+import static org.apache.seatunnel.connectors.seatunnel.file.config.Constant.FILE_FORMAT;
+import static org.apache.seatunnel.connectors.seatunnel.file.config.Constant.FILE_NAME_EXPRESSION;
+import static org.apache.seatunnel.connectors.seatunnel.file.config.Constant.IS_PARTITION_FIELD_WRITE_IN_FILE;
+import static org.apache.seatunnel.connectors.seatunnel.file.config.Constant.PATH;
+import static org.apache.seatunnel.connectors.seatunnel.file.config.Constant.ROW_DELIMITER;
+import static org.apache.seatunnel.connectors.seatunnel.file.config.Constant.SAVE_MODE;
+import static org.apache.seatunnel.connectors.seatunnel.hive.config.Constant.HIVE_METASTORE_URIS;
+import static org.apache.seatunnel.connectors.seatunnel.hive.config.Constant.HIVE_RESULT_TABLE_NAME;
+import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
+import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
+import org.apache.seatunnel.connectors.seatunnel.file.config.Constant;
+import org.apache.seatunnel.connectors.seatunnel.file.config.FileFormat;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.config.SaveMode;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.config.TextFileSinkConfig;
+import org.apache.seatunnel.connectors.seatunnel.hive.utils.HiveMetaStoreProxy;
+
import org.apache.seatunnel.shade.com.typesafe.config.Config;
+import org.apache.seatunnel.shade.com.typesafe.config.ConfigValueFactory;
import lombok.Data;
import lombok.NonNull;
import org.apache.commons.collections.CollectionUtils;
import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.thrift.TException;
+import java.io.Serializable;
+import java.util.Arrays;
import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
@Data
-public class HiveSinkConfig {
-
- private static final String HIVE_SAVE_MODE = "save_mode";
-
- private static final String HIVE_SINK_COLUMNS = "sink_columns";
-
- private static final String HIVE_PARTITION_BY = "partition_by";
-
- private static final String HIVE_RESULT_TABLE_NAME = "result_table_name";
+public class HiveSinkConfig implements Serializable {
+ private String hiveTableName;
+ private List hivePartitionFieldList;
+ private String hiveMetaUris;
- private static final String SINK_TMP_FS_ROOT_PATH = "sink_tmp_fs_root_path";
+ private String dbName;
- private static final String HIVE_TABLE_FS_PATH = "hive_table_fs_path";
+ private String tableName;
- private static final String HIVE_TXT_FILE_FIELD_DELIMITER = "hive_txt_file_field_delimiter";
+ private Table table;
- private static final String HIVE_TXT_FILE_LINE_DELIMITER = "hive_txt_file_line_delimiter";
+ private TextFileSinkConfig textFileSinkConfig;
- private SaveMode saveMode = SaveMode.APPEND;
+ public HiveSinkConfig(@NonNull Config config, @NonNull SeaTunnelRowType seaTunnelRowTypeInfo) {
+ checkArgument(!CollectionUtils.isEmpty(Arrays.asList(seaTunnelRowTypeInfo.getFieldNames())));
- private String sinkTmpFsRootPath = "/tmp/seatunnel";
+ if (config.hasPath(HIVE_RESULT_TABLE_NAME) && !StringUtils.isBlank(config.getString(HIVE_RESULT_TABLE_NAME))) {
+ this.hiveTableName = config.getString(HIVE_RESULT_TABLE_NAME);
+ }
+ checkNotNull(hiveTableName);
- private List partitionFieldNames;
+ if (config.hasPath(HIVE_METASTORE_URIS) && !StringUtils.isBlank(config.getString(HIVE_METASTORE_URIS))) {
+ this.hiveMetaUris = config.getString(HIVE_METASTORE_URIS);
+ }
+ checkNotNull(hiveMetaUris);
- private String hiveTableName;
+ String[] dbAndTableName = hiveTableName.split("\\.");
+ if (dbAndTableName == null || dbAndTableName.length != 2) {
+ throw new RuntimeException("Please config " + HIVE_RESULT_TABLE_NAME + " as db.table format");
+ }
+ this.dbName = dbAndTableName[0];
+ this.tableName = dbAndTableName[1];
+ HiveMetaStoreProxy hiveMetaStoreProxy = new HiveMetaStoreProxy(hiveMetaUris);
+ HiveMetaStoreClient hiveMetaStoreClient = hiveMetaStoreProxy.getHiveMetaStoreClient();
+
+ try {
+ table = hiveMetaStoreClient.getTable(dbName, tableName);
+ String inputFormat = table.getSd().getInputFormat();
+ if ("org.apache.hadoop.mapred.TextInputFormat".equals(inputFormat)) {
+ config = config.withValue(FILE_FORMAT, ConfigValueFactory.fromAnyRef(FileFormat.TEXT.toString()));
+ } else {
+ throw new RuntimeException("Only support text file now");
+ }
- private List sinkColumns;
+ Map parameters = table.getSd().getSerdeInfo().getParameters();
+ config = config.withValue(IS_PARTITION_FIELD_WRITE_IN_FILE, ConfigValueFactory.fromAnyRef(false))
+ .withValue(FIELD_DELIMITER, ConfigValueFactory.fromAnyRef(parameters.get("field.delim")))
+ .withValue(ROW_DELIMITER, ConfigValueFactory.fromAnyRef(parameters.get("line.delim")))
+ .withValue(FILE_NAME_EXPRESSION, ConfigValueFactory.fromAnyRef("${transactionId}"))
+ .withValue(PATH, ConfigValueFactory.fromAnyRef(table.getSd().getLocation()));
- private String hiveTableFsPath;
+ if (!config.hasPath(SAVE_MODE) || StringUtils.isBlank(config.getString(Constant.SAVE_MODE))) {
+ config = config.withValue(SAVE_MODE, ConfigValueFactory.fromAnyRef(SaveMode.APPEND.toString()));
+ }
- private String hiveTxtFileFieldDelimiter = String.valueOf('\001');
+ this.textFileSinkConfig = new TextFileSinkConfig(config, seaTunnelRowTypeInfo);
- private String hiveTxtFileLineDelimiter = "\n";
+ // --------------------Check textFileSinkConfig with the hive table info-------------------
+ List fields = hiveMetaStoreClient.getFields(dbAndTableName[0], dbAndTableName[1]);
+ List partitionKeys = table.getPartitionKeys();
- public enum SaveMode {
- APPEND(),
- OVERWRITE();
+ // Remove partitionKeys from table fields
+ List fieldNotContainPartitionKey = fields.stream().filter(filed -> !partitionKeys.contains(filed)).collect(Collectors.toList());
- public static SaveMode fromStr(String str) {
- if ("overwrite".equals(str)) {
- return OVERWRITE;
- } else {
- return APPEND;
+ // check fields size must same as sinkColumnList size
+ if (fieldNotContainPartitionKey.size() != textFileSinkConfig.getSinkColumnList().size()) {
+ throw new RuntimeException("sink columns size must same as hive table field size");
}
- }
- }
-
- public HiveSinkConfig(@NonNull Config pluginConfig) {
- checkNotNull(pluginConfig.getString(HIVE_RESULT_TABLE_NAME));
- checkNotNull(pluginConfig.getString(HIVE_TABLE_FS_PATH));
- this.hiveTableName = pluginConfig.getString(HIVE_RESULT_TABLE_NAME);
- this.hiveTableFsPath = pluginConfig.getString(HIVE_TABLE_FS_PATH);
- this.saveMode = StringUtils.isBlank(pluginConfig.getString(HIVE_SAVE_MODE)) ? SaveMode.APPEND : SaveMode.fromStr(pluginConfig.getString(HIVE_SAVE_MODE));
- if (!StringUtils.isBlank(pluginConfig.getString(SINK_TMP_FS_ROOT_PATH))) {
- this.sinkTmpFsRootPath = pluginConfig.getString(SINK_TMP_FS_ROOT_PATH);
- }
-
- this.partitionFieldNames = pluginConfig.getStringList(HIVE_PARTITION_BY);
- this.sinkColumns = pluginConfig.getStringList(HIVE_SINK_COLUMNS);
+ // check hivePartitionFieldList size must same as partitionFieldList size
+ if (partitionKeys.size() != textFileSinkConfig.getPartitionFieldList().size()) {
+ throw new RuntimeException("partition by columns size must same as hive table partition columns size");
+ }
- if (!StringUtils.isBlank(pluginConfig.getString(HIVE_TXT_FILE_FIELD_DELIMITER))) {
- this.hiveTxtFileFieldDelimiter = pluginConfig.getString(HIVE_TXT_FILE_FIELD_DELIMITER);
+ // --------------------Check textFileSinkConfig with the hive table info end----------------
+ } catch (TException e) {
+ throw new RuntimeException(e);
+ } finally {
+ hiveMetaStoreClient.close();
}
- if (!StringUtils.isBlank(pluginConfig.getString(HIVE_TXT_FILE_LINE_DELIMITER))) {
- this.hiveTxtFileLineDelimiter = pluginConfig.getString(HIVE_TXT_FILE_LINE_DELIMITER);
+ // hive only support append or overwrite
+ if (!this.textFileSinkConfig.getSaveMode().equals(SaveMode.APPEND) && !this.textFileSinkConfig.getSaveMode().equals(SaveMode.OVERWRITE)) {
+ throw new RuntimeException("hive only support append or overwrite save mode");
}
+ }
- // partition fields must in sink columns
- if (!CollectionUtils.isEmpty(this.sinkColumns) && !CollectionUtils.isEmpty(this.partitionFieldNames) && !this.sinkColumns.containsAll(this.partitionFieldNames)) {
- throw new RuntimeException("partition fields must in sink columns");
- }
+ public TextFileSinkConfig getTextFileSinkConfig() {
+ return textFileSinkConfig;
}
}
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkState.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkState.java
index 4f9f5d12e74..a104151c36a 100644
--- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkState.java
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkState.java
@@ -25,5 +25,6 @@
@Data
@AllArgsConstructor
public class HiveSinkState implements Serializable {
- private HiveSinkConfig hiveSinkConfig;
+ private String transactionId;
+ private Long checkpointId;
}
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkWriter.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkWriter.java
index e04f577290a..4bdeae47860 100644
--- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkWriter.java
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkWriter.java
@@ -20,48 +20,120 @@
import org.apache.seatunnel.api.sink.SinkWriter;
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
-import org.apache.seatunnel.connectors.seatunnel.hive.sink.file.writer.FileWriter;
-import org.apache.seatunnel.connectors.seatunnel.hive.sink.file.writer.HdfsTxtFileWriter;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.FileCommitInfo;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.FileSinkState;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.hdfs.HdfsFileSinkPlugin;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.spi.SinkFileSystemPlugin;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.transaction.TransactionStateFileWriter;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.FileSinkPartitionDirNameGenerator;
+import org.apache.seatunnel.connectors.seatunnel.file.sink.writer.FileSinkTransactionFileNameGenerator;
import org.apache.seatunnel.shade.com.typesafe.config.Config;
-import com.google.common.collect.Lists;
import lombok.NonNull;
+import org.apache.commons.collections4.CollectionUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
-import java.util.HashMap;
import java.util.List;
-import java.util.Map;
import java.util.Optional;
+import java.util.stream.Collectors;
public class HiveSinkWriter implements SinkWriter {
private static final Logger LOGGER = LoggerFactory.getLogger(HiveSinkWriter.class);
- private SeaTunnelRowType seaTunnelRowType;
+ private SeaTunnelRowType seaTunnelRowTypeInfo;
private Config pluginConfig;
- private SinkWriter.Context context;
- private long jobId;
+ private Context context;
+ private String jobId;
- private FileWriter fileWriter;
+ private TransactionStateFileWriter fileWriter;
private HiveSinkConfig hiveSinkConfig;
- public HiveSinkWriter(@NonNull SeaTunnelRowType seaTunnelRowType,
+ public HiveSinkWriter(@NonNull SeaTunnelRowType seaTunnelRowTypeInfo,
@NonNull Config pluginConfig,
@NonNull SinkWriter.Context context,
- long jobId) {
- this.seaTunnelRowType = seaTunnelRowType;
+ @NonNull HiveSinkConfig hiveSinkConfig,
+ @NonNull String jobId) {
+ this.seaTunnelRowTypeInfo = seaTunnelRowTypeInfo;
this.pluginConfig = pluginConfig;
this.context = context;
this.jobId = jobId;
+ this.hiveSinkConfig = hiveSinkConfig;
+
+ SinkFileSystemPlugin sinkFileSystemPlugin = new HdfsFileSinkPlugin();
+ Optional transactionStateFileWriter = sinkFileSystemPlugin.getTransactionStateFileWriter(this.seaTunnelRowTypeInfo,
+ new FileSinkTransactionFileNameGenerator(
+ this.hiveSinkConfig.getTextFileSinkConfig().getFileFormat(),
+ this.hiveSinkConfig.getTextFileSinkConfig().getFileNameExpression(),
+ this.hiveSinkConfig.getTextFileSinkConfig().getFileNameTimeFormat()),
+ new FileSinkPartitionDirNameGenerator(
+ this.hiveSinkConfig.getTextFileSinkConfig().getPartitionFieldList(),
+ this.hiveSinkConfig.getTextFileSinkConfig().getPartitionFieldsIndexInRow(),
+ this.hiveSinkConfig.getTextFileSinkConfig().getPartitionDirExpression()),
+ this.hiveSinkConfig.getTextFileSinkConfig().getSinkColumnsIndexInRow(),
+ this.hiveSinkConfig.getTextFileSinkConfig().getTmpPath(),
+ this.hiveSinkConfig.getTextFileSinkConfig().getPath(),
+ this.jobId,
+ this.context.getIndexOfSubtask(),
+ this.hiveSinkConfig.getTextFileSinkConfig().getFieldDelimiter(),
+ this.hiveSinkConfig.getTextFileSinkConfig().getRowDelimiter(),
+ sinkFileSystemPlugin.getFileSystem().get());
+
+ if (!transactionStateFileWriter.isPresent()) {
+ throw new RuntimeException("A TransactionStateFileWriter is need");
+ }
+
+ this.fileWriter = transactionStateFileWriter.get();
+
+ fileWriter.beginTransaction(1L);
+ }
- hiveSinkConfig = new HiveSinkConfig(this.pluginConfig);
- fileWriter = new HdfsTxtFileWriter(this.seaTunnelRowType,
- hiveSinkConfig,
+ public HiveSinkWriter(@NonNull SeaTunnelRowType seaTunnelRowTypeInfo,
+ @NonNull Config pluginConfig,
+ @NonNull SinkWriter.Context context,
+ @NonNull HiveSinkConfig hiveSinkConfig,
+ @NonNull String jobId,
+ @NonNull List hiveSinkStates) {
+ this.seaTunnelRowTypeInfo = seaTunnelRowTypeInfo;
+ this.pluginConfig = pluginConfig;
+ this.context = context;
+ this.jobId = jobId;
+ this.hiveSinkConfig = hiveSinkConfig;
+
+ SinkFileSystemPlugin sinkFileSystemPlugin = new HdfsFileSinkPlugin();
+ Optional transactionStateFileWriter = sinkFileSystemPlugin.getTransactionStateFileWriter(this.seaTunnelRowTypeInfo,
+ new FileSinkTransactionFileNameGenerator(
+ this.hiveSinkConfig.getTextFileSinkConfig().getFileFormat(),
+ this.hiveSinkConfig.getTextFileSinkConfig().getFileNameExpression(),
+ this.hiveSinkConfig.getTextFileSinkConfig().getFileNameTimeFormat()),
+ new FileSinkPartitionDirNameGenerator(
+ this.hiveSinkConfig.getTextFileSinkConfig().getPartitionFieldList(),
+ this.hiveSinkConfig.getTextFileSinkConfig().getPartitionFieldsIndexInRow(),
+ this.hiveSinkConfig.getTextFileSinkConfig().getPartitionDirExpression()),
+ this.hiveSinkConfig.getTextFileSinkConfig().getSinkColumnsIndexInRow(),
+ this.hiveSinkConfig.getTextFileSinkConfig().getTmpPath(),
+ this.hiveSinkConfig.getTextFileSinkConfig().getPath(),
this.jobId,
- this.context.getIndexOfSubtask());
+ this.context.getIndexOfSubtask(),
+ this.hiveSinkConfig.getTextFileSinkConfig().getFieldDelimiter(),
+ this.hiveSinkConfig.getTextFileSinkConfig().getRowDelimiter(),
+ sinkFileSystemPlugin.getFileSystem().get());
+
+ if (!transactionStateFileWriter.isPresent()) {
+ throw new RuntimeException("A TransactionStateFileWriter is need");
+ }
+
+ this.fileWriter = transactionStateFileWriter.get();
+
+ // Rollback dirty transaction
+ if (hiveSinkStates.size() > 0) {
+ List transactionAfter = fileWriter.getTransactionAfter(hiveSinkStates.get(0).getTransactionId());
+ fileWriter.abortTransactions(transactionAfter);
+ }
+ fileWriter.beginTransaction(hiveSinkStates.get(0).getCheckpointId() + 1);
}
@Override
@@ -71,18 +143,12 @@ public void write(SeaTunnelRow element) throws IOException {
@Override
public Optional prepareCommit() throws IOException {
- fileWriter.finishAndCloseWriteFile();
- /**
- * We will clear the needMoveFiles in {@link #snapshotState()}, So we need copy the needMoveFiles map here.
- */
- Map commitInfoMap = new HashMap<>(fileWriter.getNeedMoveFiles().size());
- commitInfoMap.putAll(fileWriter.getNeedMoveFiles());
- return Optional.of(new HiveCommitInfo(commitInfoMap));
- }
-
- @Override
- public void abortPrepare() {
- fileWriter.abort();
+ Optional fileCommitInfoOptional = fileWriter.prepareCommit();
+ if (fileCommitInfoOptional.isPresent()) {
+ FileCommitInfo fileCommitInfo = fileCommitInfoOptional.get();
+ return Optional.of(new HiveCommitInfo(fileCommitInfo, hiveSinkConfig.getHiveMetaUris(), this.hiveSinkConfig.getTable()));
+ }
+ return Optional.empty();
}
@Override
@@ -92,8 +158,17 @@ public void close() throws IOException {
@Override
public List snapshotState(long checkpointId) throws IOException {
- //reset FileWrite
- fileWriter.resetFileWriter(System.currentTimeMillis() + "");
- return Lists.newArrayList(new HiveSinkState(hiveSinkConfig));
+ List fileSinkStates = fileWriter.snapshotState(checkpointId);
+ if (!CollectionUtils.isEmpty(fileSinkStates)) {
+ return fileSinkStates.stream().map(state ->
+ new HiveSinkState(state.getTransactionId(), state.getCheckpointId()))
+ .collect(Collectors.toList());
+ }
+ return null;
+ }
+
+ @Override
+ public void abortPrepare() {
+ fileWriter.abortTransaction();
}
}
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/file/writer/AbstractFileWriter.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/file/writer/AbstractFileWriter.java
deleted file mode 100644
index 57ca3c90173..00000000000
--- a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/file/writer/AbstractFileWriter.java
+++ /dev/null
@@ -1,155 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.seatunnel.connectors.seatunnel.hive.sink.file.writer;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import org.apache.seatunnel.api.table.type.SeaTunnelRow;
-import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
-import org.apache.seatunnel.connectors.seatunnel.hive.sink.HiveSinkConfig;
-
-import lombok.NonNull;
-
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.stream.Collectors;
-import java.util.stream.IntStream;
-
-public abstract class AbstractFileWriter implements FileWriter {
- protected Map needMoveFiles;
- protected SeaTunnelRowType seaTunnelRowType;
- protected long jobId;
- protected int subTaskIndex;
- protected HiveSinkConfig hiveSinkConfig;
-
- private static final String SEATUNNEL = "seatunnel";
- private static final String NON_PARTITION = "NON_PARTITION";
-
- protected Map beingWrittenFile;
-
- protected String checkpointId;
- protected final int[] partitionKeyIndexes;
-
- public AbstractFileWriter(@NonNull SeaTunnelRowType seaTunnelRowType,
- @NonNull HiveSinkConfig hiveSinkConfig,
- long jobId,
- int subTaskIndex) {
- checkArgument(jobId > 0);
- checkArgument(subTaskIndex > -1);
-
- this.needMoveFiles = new HashMap<>();
- this.seaTunnelRowType = seaTunnelRowType;
- this.jobId = jobId;
- this.subTaskIndex = subTaskIndex;
- this.hiveSinkConfig = hiveSinkConfig;
-
- this.beingWrittenFile = new HashMap<>();
- if (this.hiveSinkConfig.getPartitionFieldNames() == null) {
- this.partitionKeyIndexes = new int[0];
- } else {
- this.partitionKeyIndexes = IntStream.range(0, seaTunnelRowType.getTotalFields())
- .filter(i -> hiveSinkConfig.getPartitionFieldNames().contains(seaTunnelRowType.getFieldName(i)))
- .toArray();
- }
- }
-
- public String getOrCreateFilePathBeingWritten(@NonNull SeaTunnelRow seaTunnelRow) {
- String beingWrittenFileKey = getBeingWrittenFileKey(seaTunnelRow);
- // get filePath from beingWrittenFile
- String beingWrittenFilePath = beingWrittenFile.get(beingWrittenFileKey);
- if (beingWrittenFilePath != null) {
- return beingWrittenFilePath;
- } else {
- StringBuilder sbf = new StringBuilder(hiveSinkConfig.getSinkTmpFsRootPath());
- sbf.append("/")
- .append(SEATUNNEL)
- .append("/")
- .append(jobId)
- .append("/")
- .append(checkpointId)
- .append("/")
- .append(hiveSinkConfig.getHiveTableName())
- .append("/")
- .append(beingWrittenFileKey)
- .append("/")
- .append(jobId)
- .append("_")
- .append(subTaskIndex)
- .append(".")
- .append(getFileSuffix());
- String newBeingWrittenFilePath = sbf.toString();
- beingWrittenFile.put(beingWrittenFileKey, newBeingWrittenFilePath);
- return newBeingWrittenFilePath;
- }
- }
-
- private String getBeingWrittenFileKey(@NonNull SeaTunnelRow seaTunnelRow) {
- if (partitionKeyIndexes.length > 0) {
- return Arrays.stream(partitionKeyIndexes)
- .boxed()
- .map(i -> seaTunnelRowType.getFieldName(i) + "=" + seaTunnelRow.getField(i))
- .collect(Collectors.joining("/"));
- } else {
- // If there is no partition field in data, We use the fixed value NON_PARTITION as the partition directory
- return NON_PARTITION;
- }
- }
-
- /**
- * FileWriter need return the file suffix. eg: tex, orc, parquet
- *
- * @return
- */
- @NonNull
- public abstract String getFileSuffix();
-
- public String getHiveLocation(@NonNull String seaTunnelFilePath) {
- StringBuilder sbf = new StringBuilder(hiveSinkConfig.getSinkTmpFsRootPath());
- sbf.append("/")
- .append(SEATUNNEL)
- .append("/")
- .append(jobId)
- .append("/")
- .append(checkpointId)
- .append("/")
- .append(hiveSinkConfig.getHiveTableName());
- String seaTunnelPath = sbf.toString();
- String tmpPath = seaTunnelFilePath.replaceAll(seaTunnelPath, hiveSinkConfig.getHiveTableFsPath());
- return tmpPath.replaceAll(NON_PARTITION + "/", "");
- }
-
- @Override
- public void resetFileWriter(@NonNull String checkpointId) {
- this.checkpointId = checkpointId;
- this.needMoveFiles = new HashMap<>();
- this.beingWrittenFile = new HashMap<>();
- this.resetMoreFileWriter(checkpointId);
- }
-
- public abstract void resetMoreFileWriter(@NonNull String checkpointId);
-
- @Override
- public void abort() {
- this.needMoveFiles = new HashMap<>();
- this.beingWrittenFile = new HashMap<>();
- this.abortMore();
- }
-
- public abstract void abortMore();
-}
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HadoopConf.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HadoopConf.java
new file mode 100644
index 00000000000..31dddf7b49a
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HadoopConf.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.hive.source;
+
+import lombok.Data;
+
+import java.io.Serializable;
+
+@Data
+public class HadoopConf implements Serializable {
+
+ private String hdfsNameKey;
+
+ private String fsHdfsImpl = "org.apache.hadoop.hdfs.DistributedFileSystem";
+
+ public HadoopConf(String hdfsNameKey) {
+ this.hdfsNameKey = hdfsNameKey;
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSource.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSource.java
new file mode 100644
index 00000000000..ebaf2c51b74
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSource.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.hive.source;
+
+import static org.apache.seatunnel.connectors.seatunnel.hive.config.SourceConfig.FILE_PATH;
+import static org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY;
+
+import org.apache.seatunnel.api.common.PrepareFailException;
+import org.apache.seatunnel.api.common.SeaTunnelContext;
+import org.apache.seatunnel.api.source.Boundedness;
+import org.apache.seatunnel.api.source.SeaTunnelSource;
+import org.apache.seatunnel.api.source.SourceReader;
+import org.apache.seatunnel.api.source.SourceSplitEnumerator;
+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.config.CheckConfigUtil;
+import org.apache.seatunnel.common.config.CheckResult;
+import org.apache.seatunnel.common.constants.PluginType;
+import org.apache.seatunnel.connectors.seatunnel.hive.config.SourceConfig;
+import org.apache.seatunnel.connectors.seatunnel.hive.exception.HivePluginException;
+import org.apache.seatunnel.connectors.seatunnel.hive.source.file.reader.format.OrcReadStrategy;
+import org.apache.seatunnel.connectors.seatunnel.hive.source.file.reader.format.ReadStrategy;
+import org.apache.seatunnel.connectors.seatunnel.hive.source.file.reader.format.TextReadStrategy;
+
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+
+import com.google.auto.service.AutoService;
+
+import java.io.IOException;
+import java.util.List;
+
+@AutoService(SeaTunnelSource.class)
+public class HiveSource implements SeaTunnelSource {
+
+ private SeaTunnelContext seaTunnelContext;
+
+ private SeaTunnelRowType typeInfo;
+
+ private ReadStrategy readStrategy;
+
+ private HadoopConf hadoopConf;
+
+ private List filesPath;
+
+ @Override
+ public String getPluginName() {
+ return "Hive";
+ }
+
+ @Override
+ public void prepare(Config pluginConfig) {
+ CheckResult result = CheckConfigUtil.checkAllExists(pluginConfig, FILE_PATH, FS_DEFAULT_NAME_KEY);
+ if (!result.isSuccess()) {
+ throw new PrepareFailException(getPluginName(), PluginType.SOURCE, result.getMsg());
+ }
+ // default filetype is text
+ if ("orc".equals(pluginConfig.getString(SourceConfig.FILE_TYPE))) {
+ readStrategy = new OrcReadStrategy();
+ } else {
+ readStrategy = new TextReadStrategy();
+ }
+ String path = pluginConfig.getString(FILE_PATH);
+ hadoopConf = new HadoopConf(pluginConfig.getString(FS_DEFAULT_NAME_KEY));
+ try {
+ filesPath = readStrategy.getFileNamesByPath(hadoopConf, path);
+ } catch (IOException e) {
+ throw new PrepareFailException(getPluginName(), PluginType.SOURCE, "Check file path fail.");
+ }
+ try {
+ // should read from config or read from hive metadata( wait catlog done)
+ this.typeInfo = readStrategy.getSeaTunnelRowTypeInfo(hadoopConf, filesPath.get(0));
+ } catch (HivePluginException e) {
+ throw new PrepareFailException(getPluginName(), PluginType.SOURCE, "Read hive file type error.", e);
+ }
+ }
+
+ @Override
+ public void setSeaTunnelContext(SeaTunnelContext seaTunnelContext) {
+ this.seaTunnelContext = seaTunnelContext;
+ }
+
+ @Override
+ public SeaTunnelDataType getProducedType() {
+ return this.typeInfo;
+ }
+
+ @Override
+ public SourceReader createReader(SourceReader.Context readerContext) throws Exception {
+ return new HiveSourceReader(this.readStrategy, this.hadoopConf, readerContext);
+ }
+
+ @Override
+ public Boundedness getBoundedness() {
+ return Boundedness.BOUNDED;
+ }
+
+ @Override
+ public SourceSplitEnumerator createEnumerator(SourceSplitEnumerator.Context enumeratorContext) throws Exception {
+ return new HiveSourceSplitEnumerator(enumeratorContext, filesPath);
+ }
+
+ @Override
+ public SourceSplitEnumerator restoreEnumerator(SourceSplitEnumerator.Context enumeratorContext, HiveSourceState checkpointState) throws Exception {
+ return new HiveSourceSplitEnumerator(enumeratorContext, filesPath, checkpointState);
+ }
+
+}
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceReader.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceReader.java
new file mode 100644
index 00000000000..570f48e35da
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceReader.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.hive.source;
+
+import org.apache.seatunnel.api.source.Collector;
+import org.apache.seatunnel.api.source.SourceReader;
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+import org.apache.seatunnel.connectors.seatunnel.hive.source.file.reader.format.ReadStrategy;
+
+import lombok.extern.slf4j.Slf4j;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+@Slf4j
+public class HiveSourceReader implements SourceReader {
+
+ private static final long THREAD_WAIT_TIME = 500L;
+
+ private ReadStrategy readStrategy;
+
+ private HadoopConf hadoopConf;
+
+ private Set sourceSplits;
+
+ private final SourceReader.Context context;
+
+ public HiveSourceReader(ReadStrategy readStrategy, HadoopConf hadoopConf, SourceReader.Context context) {
+ this.readStrategy = readStrategy;
+ this.hadoopConf = hadoopConf;
+ this.context = context;
+ this.sourceSplits = new HashSet<>();
+ }
+
+ @Override
+ public void open() {
+ readStrategy.init(hadoopConf);
+ }
+
+ @Override
+ public void close() {
+
+ }
+
+ @Override
+ public void pollNext(Collector output) throws Exception {
+ if (sourceSplits.isEmpty()) {
+ Thread.sleep(THREAD_WAIT_TIME);
+ return;
+ }
+ sourceSplits.forEach(source -> {
+ try {
+ readStrategy.read(source.splitId(), output);
+ } catch (Exception e) {
+ throw new RuntimeException("Hive source read error", e);
+ }
+
+ });
+ context.signalNoMoreElement();
+ }
+
+ @Override
+ public List snapshotState(long checkpointId) {
+ return new ArrayList<>(sourceSplits);
+ }
+
+ @Override
+ public void addSplits(List splits) {
+ sourceSplits.addAll(splits);
+ }
+
+ @Override
+ public void handleNoMoreSplits() {
+
+ }
+
+ @Override
+ public void notifyCheckpointComplete(long checkpointId) {
+
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceSplit.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceSplit.java
new file mode 100644
index 00000000000..44e062e54c3
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceSplit.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.hive.source;
+
+import org.apache.seatunnel.api.source.SourceSplit;
+
+public class HiveSourceSplit implements SourceSplit {
+
+ private static final long serialVersionUID = -1L;
+
+ private String splitId;
+
+ public HiveSourceSplit(String splitId) {
+ this.splitId = splitId;
+ }
+
+ @Override
+ public String splitId() {
+ return this.splitId;
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceSplitEnumerator.java
new file mode 100644
index 00000000000..301b1506fe6
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceSplitEnumerator.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.hive.source;
+
+import org.apache.seatunnel.api.source.SourceSplitEnumerator;
+import org.apache.seatunnel.common.config.Common;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class HiveSourceSplitEnumerator implements SourceSplitEnumerator {
+
+ private final Context context;
+ private Set pendingSplit;
+ private Set assignedSplit;
+ private List filePaths;
+
+ public HiveSourceSplitEnumerator(SourceSplitEnumerator.Context context, List filePaths) {
+ this.context = context;
+ this.filePaths = filePaths;
+ }
+
+ public HiveSourceSplitEnumerator(SourceSplitEnumerator.Context context, List filePaths,
+ HiveSourceState sourceState) {
+ this(context, filePaths);
+ this.assignedSplit = sourceState.getAssignedSplit();
+ }
+
+ @Override
+ public void open() {
+ this.assignedSplit = new HashSet<>();
+ this.pendingSplit = new HashSet<>();
+ }
+
+ @Override
+ public void run() {
+ pendingSplit = getHiveFileSplit();
+ assignSplit(context.registeredReaders());
+ }
+
+ private Set getHiveFileSplit() {
+ Set hiveSourceSplits = new HashSet<>();
+ filePaths.forEach(k -> hiveSourceSplits.add(new HiveSourceSplit(k)));
+ return hiveSourceSplits;
+
+ }
+
+ @Override
+ public void close() throws IOException {
+
+ }
+
+ @Override
+ public void addSplitsBack(List splits, int subtaskId) {
+ if (!splits.isEmpty()) {
+ pendingSplit.addAll(splits);
+ assignSplit(Collections.singletonList(subtaskId));
+ }
+ }
+
+ private void assignSplit(Collection taskIDList) {
+ Map> readySplit = new HashMap<>(Common.COLLECTION_SIZE);
+ for (int taskID : taskIDList) {
+ readySplit.computeIfAbsent(taskID, id -> new ArrayList<>());
+ }
+
+ pendingSplit.forEach(s -> readySplit.get(getSplitOwner(s.splitId(), taskIDList.size()))
+ .add(s));
+ readySplit.forEach(context::assignSplit);
+ assignedSplit.addAll(pendingSplit);
+ pendingSplit.clear();
+ }
+
+ private static int getSplitOwner(String tp, int numReaders) {
+ return tp.hashCode() % numReaders;
+ }
+
+ @Override
+ public int currentUnassignedSplitSize() {
+ return pendingSplit.size();
+ }
+
+ @Override
+ public void registerReader(int subtaskId) {
+ if (!pendingSplit.isEmpty()) {
+ assignSplit(Collections.singletonList(subtaskId));
+ }
+ }
+
+ @Override
+ public HiveSourceState snapshotState(long checkpointId) {
+ return new HiveSourceState(assignedSplit);
+ }
+
+ @Override
+ public void notifyCheckpointComplete(long checkpointId) {
+
+ }
+
+ @Override
+ public void handleSplitRequest(int subtaskId) {
+
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceState.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceState.java
new file mode 100644
index 00000000000..f982a71cb22
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceState.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.hive.source;
+
+import java.io.Serializable;
+import java.util.Set;
+
+public class HiveSourceState implements Serializable {
+
+
+ private Set assignedSplit;
+
+ public HiveSourceState(Set assignedSplit) {
+ this.assignedSplit = assignedSplit;
+ }
+
+ public Set getAssignedSplit() {
+ return assignedSplit;
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/AbstractReadStrategy.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/AbstractReadStrategy.java
new file mode 100644
index 00000000000..2df0a21f73d
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/AbstractReadStrategy.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.hive.source.file.reader.format;
+
+import static org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY;
+
+import org.apache.seatunnel.connectors.seatunnel.hive.exception.HivePluginException;
+import org.apache.seatunnel.connectors.seatunnel.hive.source.HadoopConf;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+public abstract class AbstractReadStrategy implements ReadStrategy {
+
+ HadoopConf hadoopConf;
+
+ @Override
+ public void init(HadoopConf conf) {
+ this.hadoopConf = conf;
+ }
+
+ @Override
+ public Configuration getConfiguration(HadoopConf hadoopConf) {
+ Configuration configuration = new Configuration();
+ configuration.set(FS_DEFAULT_NAME_KEY, hadoopConf.getHdfsNameKey());
+ configuration.set("fs.hdfs.impl", hadoopConf.getFsHdfsImpl());
+ return configuration;
+ }
+
+ Configuration getConfiguration() throws HivePluginException {
+ if (null == hadoopConf) {
+ throw new HivePluginException("Not init read config");
+ }
+ return getConfiguration(hadoopConf);
+ }
+
+ boolean checkFileType(String path) {
+ return true;
+ }
+
+ @Override
+ public List getFileNamesByPath(HadoopConf hadoopConf, String path) throws IOException {
+ Configuration configuration = getConfiguration(hadoopConf);
+ List fileNames = new ArrayList<>();
+ FileSystem hdfs = FileSystem.get(configuration);
+ Path listFiles = new Path(path);
+ FileStatus[] stats = hdfs.listStatus(listFiles);
+ for (FileStatus fileStatus : stats) {
+ if (fileStatus.isDirectory()) {
+ fileNames.addAll(getFileNamesByPath(hadoopConf, fileStatus.getPath().toString()));
+ continue;
+ }
+ if (fileStatus.isFile()) {
+ // filter '_SUCCESS' file
+ if (!fileStatus.getPath().getName().equals("_SUCCESS")) {
+ fileNames.add(fileStatus.getPath().toString());
+ }
+ }
+ }
+ return fileNames;
+ }
+
+}
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/OrcReadStrategy.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/OrcReadStrategy.java
new file mode 100644
index 00000000000..325c1aaccb9
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/OrcReadStrategy.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.hive.source.file.reader.format;
+
+import org.apache.seatunnel.api.source.Collector;
+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.hive.exception.HivePluginException;
+import org.apache.seatunnel.connectors.seatunnel.hive.source.HadoopConf;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.orc.OrcFile;
+import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
+import org.apache.hadoop.hive.ql.io.orc.OrcSerde;
+import org.apache.hadoop.hive.ql.io.orc.OrcStruct;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.orc.Reader;
+import org.apache.orc.TypeDescription;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Properties;
+
+@Slf4j
+public class OrcReadStrategy extends AbstractReadStrategy {
+
+ private SeaTunnelRowType seaTunnelRowTypeInfo;
+
+ @Override
+ public void read(String path, Collector output) throws Exception {
+ if (Boolean.FALSE.equals(checkFileType(path))) {
+ throw new Exception("please check file type");
+ }
+ JobConf conf = new JobConf();
+ Path filePath = new Path(path);
+ Properties p = new Properties();
+ OrcSerde serde = new OrcSerde();
+ String columns = String.join(",", seaTunnelRowTypeInfo.getFieldNames());
+ p.setProperty("columns", columns);
+ //support types
+ serde.initialize(conf, p);
+ StructObjectInspector inspector = (StructObjectInspector) serde.getObjectInspector();
+ InputFormat in = new OrcInputFormat();
+ FileInputFormat.setInputPaths(conf, filePath);
+ InputSplit[] splits = in.getSplits(conf, 1);
+
+ conf.set("hive.io.file.readcolumn.ids", "1");
+ RecordReader reader = in.getRecordReader(splits[0], conf, Reporter.NULL);
+ NullWritable key = reader.createKey();
+ OrcStruct value = reader.createValue();
+ List extends StructField> fields = inspector.getAllStructFieldRefs();
+ while (reader.next(key, value)) {
+ Object[] datas = new Object[fields.size()];
+ for (int i = 0; i < fields.size(); i++) {
+ Object data = inspector.getStructFieldData(value, fields.get(i));
+ if (null != data) {
+ datas[i] = String.valueOf(data);
+ } else {
+ datas[i] = null;
+ }
+ }
+ output.collect(new SeaTunnelRow(datas));
+ }
+ reader.close();
+ }
+
+ @Override
+ public SeaTunnelRowType getSeaTunnelRowTypeInfo(HadoopConf hadoopConf, String path) throws HivePluginException {
+
+ if (null != seaTunnelRowTypeInfo) {
+ return seaTunnelRowTypeInfo;
+ }
+ Configuration configuration = getConfiguration(hadoopConf);
+ Path dstDir = new Path(path);
+ Reader reader;
+ try {
+ reader = OrcFile.createReader(FileSystem.get(configuration), dstDir);
+ } catch (IOException e) {
+ throw new HivePluginException("Create OrcReader Fail", e);
+ }
+
+ TypeDescription schema = reader.getSchema();
+ String[] fields = new String[schema.getFieldNames().size()];
+ SeaTunnelDataType[] types = new SeaTunnelDataType[schema.getFieldNames().size()];
+
+ for (int i = 0; i < schema.getFieldNames().size(); i++) {
+ fields[i] = schema.getFieldNames().get(i);
+ types[i] = BasicType.STRING_TYPE;
+ }
+ seaTunnelRowTypeInfo = new SeaTunnelRowType(fields, types);
+ return seaTunnelRowTypeInfo;
+ }
+
+}
+
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/ReadStrategy.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/ReadStrategy.java
new file mode 100644
index 00000000000..5e73019147a
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/ReadStrategy.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.hive.source.file.reader.format;
+
+import org.apache.seatunnel.api.source.Collector;
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
+import org.apache.seatunnel.connectors.seatunnel.hive.exception.HivePluginException;
+import org.apache.seatunnel.connectors.seatunnel.hive.source.HadoopConf;
+
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.List;
+
+public interface ReadStrategy extends Serializable {
+ void init(HadoopConf conf);
+
+ Configuration getConfiguration(HadoopConf conf);
+
+ void read(String path, Collector output) throws Exception;
+
+ SeaTunnelRowType getSeaTunnelRowTypeInfo(HadoopConf hadoopConf, String path) throws HivePluginException;
+
+ List getFileNamesByPath(HadoopConf hadoopConf, String path) throws IOException;
+}
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/TextReadStrategy.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/TextReadStrategy.java
new file mode 100644
index 00000000000..6b014d73799
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/file/reader/format/TextReadStrategy.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.hive.source.file.reader.format;
+
+import org.apache.seatunnel.api.source.Collector;
+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.hive.exception.HivePluginException;
+import org.apache.seatunnel.connectors.seatunnel.hive.source.HadoopConf;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+
+public class TextReadStrategy extends AbstractReadStrategy {
+
+ private static final String TEXT_FIELD_NAME = "lines";
+
+ @Override
+ public void read(String path, Collector output) throws IOException, HivePluginException {
+ Configuration conf = getConfiguration();
+ FileSystem fs = FileSystem.get(conf);
+ Path filePath = new Path(path);
+ try (BufferedReader reader = new BufferedReader(new InputStreamReader(fs.open(filePath), StandardCharsets.UTF_8))) {
+ reader.lines().forEach(line -> output.collect(new SeaTunnelRow(new String[]{line})));
+ }
+ }
+
+ @Override
+ public SeaTunnelRowType getSeaTunnelRowTypeInfo(HadoopConf hadoopConf, String path) {
+ return new SeaTunnelRowType(new String[]{TEXT_FIELD_NAME},
+ new SeaTunnelDataType>[]{BasicType.STRING_TYPE});
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/utils/HiveMetaStoreProxy.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/utils/HiveMetaStoreProxy.java
new file mode 100644
index 00000000000..30c9a2eba5b
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/utils/HiveMetaStoreProxy.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.hive.utils;
+
+import lombok.NonNull;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.thrift.TException;
+
+public class HiveMetaStoreProxy {
+
+ private HiveMetaStoreClient hiveMetaStoreClient;
+
+ public HiveMetaStoreProxy(@NonNull String uris) {
+ HiveConf hiveConf = new HiveConf();
+ hiveConf.set("hive.metastore.uris", uris);
+ try {
+ hiveMetaStoreClient = new HiveMetaStoreClient(hiveConf);
+ } catch (MetaException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ public Table getTable(@NonNull String dbName, @NonNull String tableName) {
+ try {
+ return hiveMetaStoreClient.getTable(dbName, tableName);
+ } catch (TException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ public HiveMetaStoreClient getHiveMetaStoreClient() {
+ return hiveMetaStoreClient;
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-hive/src/test/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/TestHiveSinkConfig.java b/seatunnel-connectors-v2/connector-hive/src/test/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/TestHiveSinkConfig.java
new file mode 100644
index 00000000000..92e5a693116
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-hive/src/test/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/TestHiveSinkConfig.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.hive.sink;
+
+import org.apache.seatunnel.api.table.type.BasicType;
+import org.apache.seatunnel.api.table.type.SeaTunnelDataType;
+import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
+
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+import org.apache.seatunnel.shade.com.typesafe.config.ConfigFactory;
+import org.apache.seatunnel.shade.com.typesafe.config.ConfigResolveOptions;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.File;
+import java.util.List;
+
+@RunWith(JUnit4.class)
+public class TestHiveSinkConfig {
+
+ @Test
+ public void testCreateHiveSinkConfig() {
+ String[] fieldNames = new String[]{"name", "age"};
+ SeaTunnelDataType[] seaTunnelDataTypes = new SeaTunnelDataType[]{BasicType.STRING_TYPE, BasicType.INT_TYPE};
+ SeaTunnelRowType seaTunnelRowTypeInfo = new SeaTunnelRowType(fieldNames, seaTunnelDataTypes);
+ String configFile = "fakesource_to_hive.conf";
+ String configFilePath = System.getProperty("user.dir") + "/src/test/resources/" + configFile;
+ Config config = ConfigFactory
+ .parseFile(new File(configFilePath))
+ .resolve(ConfigResolveOptions.defaults().setAllowUnresolved(true))
+ .resolveWith(ConfigFactory.systemProperties(),
+ ConfigResolveOptions.defaults().setAllowUnresolved(true));
+ List extends Config> sink = config.getConfigList("sink");
+ HiveSinkConfig hiveSinkConfig = new HiveSinkConfig(sink.get(0), seaTunnelRowTypeInfo);
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-hive/src/test/resources/fakesource_to_hive.conf b/seatunnel-connectors-v2/connector-hive/src/test/resources/fakesource_to_hive.conf
new file mode 100644
index 00000000000..3412ea663c4
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-hive/src/test/resources/fakesource_to_hive.conf
@@ -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.
+#
+######
+###### 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 {
+ Hive {
+ hive_table_name="default.test_fake_to_hive"
+ hive_metastore_uris="thrift://localhost:9083"
+ partition_by=["age"]
+ sink_columns=["name","age"]
+ }
+
+ # 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-connectors-v2/connector-hudi/pom.xml b/seatunnel-connectors-v2/connector-hudi/pom.xml
new file mode 100644
index 00000000000..0b2ee9ce8c4
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-hudi/pom.xml
@@ -0,0 +1,61 @@
+
+
+
+
+ seatunnel-connectors-v2
+ org.apache.seatunnel
+ ${revision}
+
+ 4.0.0
+
+ connector-hudi
+
+
+
+
+ org.apache.hive
+ hive-exec
+ provided
+
+
+
+ org.apache.seatunnel
+ seatunnel-api
+ ${project.version}
+
+
+
+ org.apache.hudi
+ hudi-hadoop-mr-bundle
+
+
+
+ org.apache.commons
+ commons-lang3
+
+
+
+ junit
+ junit
+
+
+
diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/config/HudiSourceConfig.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/config/HudiSourceConfig.java
new file mode 100644
index 00000000000..9f1b86e94a2
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/config/HudiSourceConfig.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.hudi.config;
+
+public class HudiSourceConfig {
+
+ public static final String TABLE_PATH = "table.path";
+
+ public static final String TABLE_TYPE = "table.type";
+
+ public static final String CONF_FILES = "conf.files";
+
+ public static final String USE_KERBEROS = "use.kerberos";
+
+ public static final String KERBEROS_PRINCIPAL = "kerberos.principal";
+
+ public static final String KERBEROS_PRINCIPAL_FILE = "kerberos.principal.file";
+
+}
diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/exception/HudiPluginException.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/exception/HudiPluginException.java
new file mode 100644
index 00000000000..6971e029e08
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/exception/HudiPluginException.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.hudi.exception;
+
+public class HudiPluginException extends Exception{
+
+ public HudiPluginException(String message) {
+ super(message);
+ }
+
+ public HudiPluginException(String message, Throwable cause) {
+ super(message, cause);
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/source/HudiSource.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/source/HudiSource.java
new file mode 100644
index 00000000000..2ca69d784e7
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/source/HudiSource.java
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.hudi.source;
+
+import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiSourceConfig.CONF_FILES;
+import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiSourceConfig.KERBEROS_PRINCIPAL;
+import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiSourceConfig.KERBEROS_PRINCIPAL_FILE;
+import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiSourceConfig.TABLE_PATH;
+import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiSourceConfig.TABLE_TYPE;
+import static org.apache.seatunnel.connectors.seatunnel.hudi.config.HudiSourceConfig.USE_KERBEROS;
+
+import org.apache.seatunnel.api.common.PrepareFailException;
+import org.apache.seatunnel.api.common.SeaTunnelContext;
+import org.apache.seatunnel.api.source.Boundedness;
+import org.apache.seatunnel.api.source.SeaTunnelSource;
+import org.apache.seatunnel.api.source.SourceReader;
+import org.apache.seatunnel.api.source.SourceSplitEnumerator;
+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.config.CheckConfigUtil;
+import org.apache.seatunnel.common.config.CheckResult;
+import org.apache.seatunnel.common.constants.PluginType;
+import org.apache.seatunnel.connectors.seatunnel.hudi.exception.HudiPluginException;
+import org.apache.seatunnel.connectors.seatunnel.hudi.util.HudiUtil;
+
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+
+import com.google.auto.service.AutoService;
+
+import java.io.IOException;
+
+@AutoService(SeaTunnelSource.class)
+public class HudiSource implements SeaTunnelSource {
+
+ private SeaTunnelContext seaTunnelContext;
+
+ private SeaTunnelRowType typeInfo;
+
+ private String filePath;
+
+ private String tablePath;
+
+ private String confFiles;
+
+ private boolean useKerberos = false;
+
+ @Override
+ public String getPluginName() {
+ return "Hudi";
+ }
+
+ @Override
+ public void prepare(Config pluginConfig) {
+ CheckResult result = CheckConfigUtil.checkAllExists(pluginConfig, TABLE_PATH, CONF_FILES);
+ if (!result.isSuccess()) {
+ throw new PrepareFailException(getPluginName(), PluginType.SOURCE, result.getMsg());
+ }
+ // default hudi table tupe is cow
+ // TODO: support hudi mor table
+ // TODO: support Incremental Query and Read Optimized Query
+ if (!"cow".equalsIgnoreCase(pluginConfig.getString(TABLE_TYPE))) {
+ throw new PrepareFailException(getPluginName(), PluginType.SOURCE, "Do not support hudi mor table yet!");
+ }
+ try {
+ this.confFiles = pluginConfig.getString(CONF_FILES);
+ this.tablePath = pluginConfig.getString(TABLE_PATH);
+ if (CheckConfigUtil.isValidParam(pluginConfig, USE_KERBEROS)) {
+ this.useKerberos = pluginConfig.getBoolean(USE_KERBEROS);
+ if (this.useKerberos) {
+ CheckResult kerberosCheckResult = CheckConfigUtil.checkAllExists(pluginConfig, KERBEROS_PRINCIPAL, KERBEROS_PRINCIPAL_FILE);
+ if (!kerberosCheckResult.isSuccess()) {
+ throw new PrepareFailException(getPluginName(), PluginType.SOURCE, result.getMsg());
+ }
+ HudiUtil.initKerberosAuthentication(HudiUtil.getConfiguration(this.confFiles), pluginConfig.getString(KERBEROS_PRINCIPAL), pluginConfig.getString(KERBEROS_PRINCIPAL_FILE));
+ }
+ }
+ this.filePath = HudiUtil.getParquetFileByPath(this.confFiles, tablePath);
+ if (this.filePath == null) {
+ throw new HudiPluginException(String.format("%s has no parquet file, please check!", tablePath));
+ }
+ // should read from config or read from hudi metadata( wait catlog done)
+ this.typeInfo = HudiUtil.getSeaTunnelRowTypeInfo(this.confFiles, this.filePath);
+
+ } catch (HudiPluginException | IOException e) {
+ throw new PrepareFailException(getPluginName(), PluginType.SOURCE, "Prepare HudiSource error.", e);
+ }
+
+ }
+
+ @Override
+ public void setSeaTunnelContext(SeaTunnelContext seaTunnelContext) {
+ this.seaTunnelContext = seaTunnelContext;
+ }
+
+ @Override
+ public SeaTunnelDataType getProducedType() {
+ return this.typeInfo;
+ }
+
+ @Override
+ public SourceReader createReader(SourceReader.Context readerContext) throws Exception {
+ return new HudiSourceReader(this.confFiles, readerContext, typeInfo);
+ }
+
+ @Override
+ public Boundedness getBoundedness() {
+ // Only support Snapshot Query now.
+ // After support Incremental Query and Read Optimized Query, we should supoort UNBOUNDED.
+ // TODO: support UNBOUNDED
+ return Boundedness.BOUNDED;
+ }
+
+ @Override
+ public SourceSplitEnumerator createEnumerator(SourceSplitEnumerator.Context enumeratorContext) throws Exception {
+ return new HudiSourceSplitEnumerator(enumeratorContext, tablePath, this.confFiles);
+ }
+
+ @Override
+ public SourceSplitEnumerator restoreEnumerator(SourceSplitEnumerator.Context enumeratorContext, HudiSourceState checkpointState) throws Exception {
+ return new HudiSourceSplitEnumerator(enumeratorContext, tablePath, this.confFiles, checkpointState);
+ }
+
+}
diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/source/HudiSourceReader.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/source/HudiSourceReader.java
new file mode 100644
index 00000000000..15127ae5ba1
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/source/HudiSourceReader.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.hudi.source;
+
+import org.apache.seatunnel.api.source.Collector;
+import org.apache.seatunnel.api.source.SourceReader;
+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.hudi.util.HudiUtil;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hudi.hadoop.HoodieParquetInputFormat;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Properties;
+import java.util.Set;
+
+public class HudiSourceReader implements SourceReader {
+
+ private static final long THREAD_WAIT_TIME = 500L;
+
+ private String confPaths;
+
+ private Set sourceSplits;
+
+ private final SourceReader.Context context;
+
+ private SeaTunnelRowType seaTunnelRowType;
+
+ public HudiSourceReader(String confPaths, SourceReader.Context context, SeaTunnelRowType seaTunnelRowType) {
+ this.confPaths = confPaths;
+ this.context = context;
+ this.sourceSplits = new HashSet<>();
+ this.seaTunnelRowType = seaTunnelRowType;
+ }
+
+ @Override
+ public void open() {
+ }
+
+ @Override
+ public void close() {
+
+ }
+
+ @Override
+ public void pollNext(Collector output) throws Exception {
+ if (sourceSplits.isEmpty()) {
+ Thread.sleep(THREAD_WAIT_TIME);
+ return;
+ }
+ Configuration configuration = HudiUtil.getConfiguration(this.confPaths);
+ JobConf jobConf = HudiUtil.toJobConf(configuration);
+ sourceSplits.forEach(source -> {
+ try {
+ HoodieParquetInputFormat inputFormat = new HoodieParquetInputFormat();
+ RecordReader reader = inputFormat.getRecordReader(source.getInputSplit(), jobConf, Reporter.NULL);
+ ParquetHiveSerDe serde = new ParquetHiveSerDe();
+ Properties properties = new Properties();
+ List types = new ArrayList<>();
+ for (SeaTunnelDataType> type: seaTunnelRowType.getFieldTypes()) {
+ types.add(type.getSqlType().name());
+ }
+ String columns = StringUtils.join(seaTunnelRowType.getFieldNames(), ",");
+ String columnTypes = StringUtils.join(types, ",").toLowerCase(Locale.ROOT);
+ properties.setProperty("columns", columns);
+ properties.setProperty("columns.types", columnTypes);
+ serde.initialize(jobConf, properties);
+ StructObjectInspector inspector = (StructObjectInspector) serde.getObjectInspector();
+ List extends StructField> fields = inspector.getAllStructFieldRefs();
+ NullWritable key = reader.createKey();
+ ArrayWritable value = reader.createValue();
+ while (reader.next(key, value)) {
+ Object[] datas = new Object[fields.size()];
+ for (int i = 0; i < fields.size(); i++) {
+ Object data = inspector.getStructFieldData(value, fields.get(i));
+ if (null != data) {
+ datas[i] = String.valueOf(data);
+ } else {
+ datas[i] = null;
+ }
+ }
+ output.collect(new SeaTunnelRow(datas));
+ }
+ reader.close();
+ } catch (Exception e) {
+ throw new RuntimeException("Hudi source read error", e);
+ }
+
+ });
+ context.signalNoMoreElement();
+ }
+
+ @Override
+ public List snapshotState(long checkpointId) {
+ return new ArrayList<>(sourceSplits);
+ }
+
+ @Override
+ public void addSplits(List splits) {
+ sourceSplits.addAll(splits);
+ }
+
+ @Override
+ public void handleNoMoreSplits() {
+
+ }
+
+ @Override
+ public void notifyCheckpointComplete(long checkpointId) {
+
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/source/HudiSourceSplit.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/source/HudiSourceSplit.java
new file mode 100644
index 00000000000..b08f6f68e11
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/source/HudiSourceSplit.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.hudi.source;
+
+import org.apache.seatunnel.api.source.SourceSplit;
+
+import org.apache.hadoop.mapred.InputSplit;
+
+public class HudiSourceSplit implements SourceSplit {
+
+ private static final long serialVersionUID = -1L;
+
+ private String splitId;
+
+ private InputSplit inputSplit;
+
+ public HudiSourceSplit(String splitId, InputSplit inputSplit) {
+ this.splitId = splitId;
+ this.inputSplit = inputSplit;
+ }
+
+ @Override
+ public String splitId() {
+ return this.splitId;
+ }
+
+ public InputSplit getInputSplit() {
+ return this.inputSplit;
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/source/HudiSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/source/HudiSourceSplitEnumerator.java
new file mode 100644
index 00000000000..42b072af6c1
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/source/HudiSourceSplitEnumerator.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.hudi.source;
+
+import org.apache.seatunnel.api.source.SourceSplitEnumerator;
+import org.apache.seatunnel.common.config.Common;
+import org.apache.seatunnel.connectors.seatunnel.hudi.util.HudiUtil;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hudi.hadoop.HoodieParquetInputFormat;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class HudiSourceSplitEnumerator implements SourceSplitEnumerator {
+
+ private final Context context;
+ private Set pendingSplit;
+ private Set assignedSplit;
+ private String tablePath;
+ private String confPaths;
+
+ public HudiSourceSplitEnumerator(SourceSplitEnumerator.Context context, String tablePath, String confPaths) {
+ this.context = context;
+ this.tablePath = tablePath;
+ this.confPaths = confPaths;
+ }
+
+ public HudiSourceSplitEnumerator(SourceSplitEnumerator.Context context, String tablePath,
+ String confPaths,
+ HudiSourceState sourceState) {
+ this(context, tablePath, confPaths);
+ this.assignedSplit = sourceState.getAssignedSplit();
+ }
+
+ @Override
+ public void open() {
+ this.assignedSplit = new HashSet<>();
+ this.pendingSplit = new HashSet<>();
+ }
+
+ @Override
+ public void run() throws Exception {
+ pendingSplit = getHudiSplit();
+ assignSplit(context.registeredReaders());
+ }
+
+ private Set getHudiSplit() throws IOException {
+ Set hudiSourceSplits = new HashSet<>();
+ Path path = new Path(tablePath);
+ Configuration configuration = HudiUtil.getConfiguration(confPaths);
+ JobConf jobConf = HudiUtil.toJobConf(configuration);
+ FileInputFormat.setInputPaths(jobConf, path);
+ HoodieParquetInputFormat inputFormat = new HoodieParquetInputFormat();
+ inputFormat.setConf(jobConf);
+ for (InputSplit split: inputFormat.getSplits(jobConf, 0)) {
+ hudiSourceSplits.add(new HudiSourceSplit(split.toString(), split));
+ }
+ return hudiSourceSplits;
+ }
+
+ @Override
+ public void close() throws IOException {
+
+ }
+
+ @Override
+ public void addSplitsBack(List splits, int subtaskId) {
+ if (!splits.isEmpty()) {
+ pendingSplit.addAll(splits);
+ assignSplit(Collections.singletonList(subtaskId));
+ }
+ }
+
+ private void assignSplit(Collection taskIDList) {
+ Map> readySplit = new HashMap<>(Common.COLLECTION_SIZE);
+ for (int taskID : taskIDList) {
+ readySplit.computeIfAbsent(taskID, id -> new ArrayList<>());
+ }
+
+ pendingSplit.forEach(s -> readySplit.get(getSplitOwner(s.splitId(), taskIDList.size()))
+ .add(s));
+ readySplit.forEach(context::assignSplit);
+ assignedSplit.addAll(pendingSplit);
+ pendingSplit.clear();
+ }
+
+ private static int getSplitOwner(String tp, int numReaders) {
+ return tp.hashCode() % numReaders;
+ }
+
+ @Override
+ public int currentUnassignedSplitSize() {
+ return pendingSplit.size();
+ }
+
+ @Override
+ public void registerReader(int subtaskId) {
+ if (!pendingSplit.isEmpty()) {
+ assignSplit(Collections.singletonList(subtaskId));
+ }
+ }
+
+ @Override
+ public HudiSourceState snapshotState(long checkpointId) {
+ return new HudiSourceState(assignedSplit);
+ }
+
+ @Override
+ public void notifyCheckpointComplete(long checkpointId) {
+
+ }
+
+ @Override
+ public void handleSplitRequest(int subtaskId) {
+
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/source/HudiSourceState.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/source/HudiSourceState.java
new file mode 100644
index 00000000000..6235ca694ae
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/source/HudiSourceState.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.hudi.source;
+
+import java.io.Serializable;
+import java.util.Set;
+
+public class HudiSourceState implements Serializable {
+
+
+ private Set assignedSplit;
+
+ public HudiSourceState(Set assignedSplit) {
+ this.assignedSplit = assignedSplit;
+ }
+
+ public Set getAssignedSplit() {
+ return assignedSplit;
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/util/HudiUtil.java b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/util/HudiUtil.java
new file mode 100644
index 00000000000..346232b9534
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-hudi/src/main/java/org/apache/seatunnel/connectors/seatunnel/hudi/util/HudiUtil.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.hudi.util;
+
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
+
+import org.apache.seatunnel.api.table.type.BasicType;
+import org.apache.seatunnel.api.table.type.SeaTunnelDataType;
+import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
+import org.apache.seatunnel.connectors.seatunnel.hudi.exception.HudiPluginException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.schema.MessageType;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+public class HudiUtil {
+
+ public static Configuration getConfiguration(String confPaths) {
+ Configuration configuration = new Configuration();
+ Arrays.stream(confPaths.split(";")).forEach(file -> configuration.addResource(new Path(file)));
+ return configuration;
+ }
+
+ public static String getParquetFileByPath(String confPaths, String path) throws IOException {
+ Configuration configuration = getConfiguration(confPaths);
+ FileSystem hdfs = FileSystem.get(configuration);
+ Path listFiles = new Path(path);
+ FileStatus[] stats = hdfs.listStatus(listFiles);
+ for (FileStatus fileStatus : stats) {
+ if (fileStatus.isDirectory()) {
+ String filePath = getParquetFileByPath(confPaths, fileStatus.getPath().toString());
+ if (filePath == null) {
+ continue;
+ } else {
+ return filePath;
+ }
+ }
+ if (fileStatus.isFile()) {
+ if (fileStatus.getPath().toString().endsWith("parquet")) {
+ return fileStatus.getPath().toString();
+ }
+ }
+ }
+ return null;
+ }
+
+ public static SeaTunnelRowType getSeaTunnelRowTypeInfo(String confPaths, String path) throws HudiPluginException {
+ Configuration configuration = getConfiguration(confPaths);
+ Path dstDir = new Path(path);
+ ParquetMetadata footer;
+ try {
+ footer = ParquetFileReader.readFooter(configuration, dstDir, NO_FILTER);
+ } catch (IOException e) {
+ throw new HudiPluginException("Create ParquetMetadata Fail!", e);
+ }
+ MessageType schema = footer.getFileMetaData().getSchema();
+ String[] fields = new String[schema.getFields().size()];
+ SeaTunnelDataType[] types = new SeaTunnelDataType[schema.getFields().size()];
+
+ for (int i = 0; i < schema.getFields().size(); i++) {
+ fields[i] = schema.getFields().get(i).getName();
+ types[i] = BasicType.STRING_TYPE;
+ }
+ return new SeaTunnelRowType(fields, types);
+ }
+
+ public static JobConf toJobConf(Configuration conf) {
+ if (conf instanceof JobConf) {
+ return (JobConf) conf;
+ }
+ return new JobConf(conf);
+ }
+
+ public static void initKerberosAuthentication(Configuration conf, String principal, String principalFile) throws HudiPluginException {
+ try {
+ UserGroupInformation.setConfiguration(conf);
+ UserGroupInformation.loginUserFromKeytab(principal, principalFile);
+ } catch (IOException e) {
+ throw new HudiPluginException("Kerberos Authorized Fail!", e);
+ }
+
+ }
+
+}
diff --git a/seatunnel-connectors-v2/connector-jdbc/pom.xml b/seatunnel-connectors-v2/connector-jdbc/pom.xml
index 1bb972ac96c..39839eba700 100644
--- a/seatunnel-connectors-v2/connector-jdbc/pom.xml
+++ b/seatunnel-connectors-v2/connector-jdbc/pom.xml
@@ -46,6 +46,7 @@
org.postgresql
postgresql
+ provided
diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/MySqlCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/MySqlCatalog.java
index 4f6e61ce573..c3aa84ae45c 100644
--- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/MySqlCatalog.java
+++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/MySqlCatalog.java
@@ -130,7 +130,7 @@ public CatalogTable getTable(TablePath tablePath) throws CatalogException, Table
getPrimaryKey(metaData, tablePath.getDatabaseName(), tablePath.getTableName());
PreparedStatement ps =
- conn.prepareStatement(String.format("SELECT * FROM %s LIMIT 1;", tablePath.getFullName()));
+ conn.prepareStatement(String.format("SELECT * FROM %s WHERE 1 = 0;", tablePath.getFullName()));
ResultSetMetaData tableMetaData = ps.getMetaData();
diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/xa/SemanticXidGenerator.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/xa/SemanticXidGenerator.java
index 98825a57a59..3d2a82b3d13 100644
--- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/xa/SemanticXidGenerator.java
+++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/xa/SemanticXidGenerator.java
@@ -80,8 +80,7 @@ public boolean belongsToSubtask(Xid xid, SeaTunnelContext context, SinkWriter.Co
return false;
}
int subtaskIndex = readNumber(xid.getGlobalTransactionId(), JOB_ID_BYTES, Integer.BYTES);
- if (subtaskIndex != sinkContext.getIndexOfSubtask()
- && subtaskIndex <= sinkContext.getNumberOfParallelSubtasks() - 1) {
+ if (subtaskIndex != sinkContext.getIndexOfSubtask()) {
return false;
}
byte[] jobIdBytes = new byte[JOB_ID_BYTES];
diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkAggregatedCommitter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkAggregatedCommitter.java
index f0ee2671eda..d142e95bbe2 100644
--- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkAggregatedCommitter.java
+++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkAggregatedCommitter.java
@@ -83,7 +83,9 @@ public void abort(List aggregatedCommitInfo) throws IO
public void close()
throws IOException {
try {
- xaFacade.close();
+ if (xaFacade.isOpen()) {
+ xaFacade.close();
+ }
} catch (Exception e) {
ExceptionUtils.rethrowIOException(e);
}
diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSource.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSource.java
index 73a61b3a386..2717436f194 100644
--- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSource.java
+++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSource.java
@@ -19,7 +19,6 @@
import org.apache.seatunnel.api.common.PrepareFailException;
import org.apache.seatunnel.api.common.SeaTunnelContext;
-import org.apache.seatunnel.api.serialization.DefaultSerializer;
import org.apache.seatunnel.api.serialization.Serializer;
import org.apache.seatunnel.api.source.Boundedness;
import org.apache.seatunnel.api.source.SeaTunnelSource;
@@ -141,11 +140,6 @@ public SourceSplitEnumerator restoreEnumerator
return new JdbcSourceSplitEnumerator(enumeratorContext, jdbcSourceOptions, partitionParameter);
}
- @Override
- public Serializer getEnumeratorStateSerializer() {
- return new DefaultSerializer<>();
- }
-
private SeaTunnelRowType initTableField(Connection conn) {
JdbcDialectTypeMapper jdbcDialectTypeMapper = jdbcDialect.getJdbcDialectTypeMapper();
ArrayList> seaTunnelDataTypes = new ArrayList<>();
diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java
index b5a3d42d6b4..a4d534e6775 100644
--- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java
+++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java
@@ -25,8 +25,6 @@
import org.apache.seatunnel.api.common.PrepareFailException;
import org.apache.seatunnel.api.common.SeaTunnelContext;
-import org.apache.seatunnel.api.serialization.DefaultSerializer;
-import org.apache.seatunnel.api.serialization.Serializer;
import org.apache.seatunnel.api.source.Boundedness;
import org.apache.seatunnel.api.source.SeaTunnelSource;
import org.apache.seatunnel.api.source.SourceReader;
@@ -120,11 +118,6 @@ public SourceSplitEnumerator restoreEnumerat
return new KafkaSourceSplitEnumerator(this.metadata, enumeratorContext, checkpointState);
}
- @Override
- public Serializer getEnumeratorStateSerializer() {
- return new DefaultSerializer<>();
- }
-
@Override
public void setSeaTunnelContext(SeaTunnelContext seaTunnelContext) {
this.seaTunnelContext = seaTunnelContext;
diff --git a/seatunnel-connectors-v2/connector-kudu/pom.xml b/seatunnel-connectors-v2/connector-kudu/pom.xml
new file mode 100644
index 00000000000..c8d3591abb1
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-kudu/pom.xml
@@ -0,0 +1,30 @@
+
+
+
+ seatunnel-connectors-v2
+ org.apache.seatunnel
+ ${revision}
+
+ 4.0.0
+
+ connector-kudu
+
+
+
+ org.apache.seatunnel
+ seatunnel-api
+ ${project.version}
+
+
+ org.apache.kudu
+ kudu-client
+
+
+
+ org.apache.commons
+ commons-lang3
+
+
+
\ No newline at end of file
diff --git a/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/config/KuduSinkConfig.java b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/config/KuduSinkConfig.java
new file mode 100644
index 00000000000..8f4f3ae44bf
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/config/KuduSinkConfig.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.kudu.config;
+
+import lombok.Data;
+import lombok.NonNull;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+
+
+
+@Data
+public class KuduSinkConfig {
+
+ private static final String KUDU_SAVE_MODE = "save_mode";
+ private static final String KUDU_MASTER = "kudu_master";
+ private static final String KUDU_TABLE_NAME = "kudu_table";
+
+ private SaveMode saveMode = SaveMode.APPEND;
+
+ private String kuduMaster;
+
+ /**
+ * Specifies the name of the table
+ */
+ private String kuduTableName;
+
+ public enum SaveMode {
+ APPEND(),
+ OVERWRITE();
+
+ public static SaveMode fromStr(String str) {
+ if ("overwrite".equals(str)) {
+ return OVERWRITE;
+ } else {
+ return APPEND;
+ }
+ }
+ }
+
+ public KuduSinkConfig(@NonNull Config pluginConfig) {
+
+ this.saveMode = StringUtils.isBlank(pluginConfig.getString(KUDU_SAVE_MODE)) ? SaveMode.APPEND : SaveMode.fromStr(pluginConfig.getString(KUDU_SAVE_MODE));
+
+ this.kuduMaster = pluginConfig.getString(KUDU_MASTER);
+ this.kuduTableName = pluginConfig.getString(KUDU_TABLE_NAME);
+
+
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/config/KuduSourceConfig.java b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/config/KuduSourceConfig.java
new file mode 100644
index 00000000000..9b148140c12
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/config/KuduSourceConfig.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.kudu.config;
+
+
+
+import java.io.Serializable;
+
+public class KuduSourceConfig implements Serializable {
+ //kudu master ip
+ public static final String kuduMaster = "kudu_master";
+
+ public static final String tableName = "kudu_table";
+
+ public static final String columnsList = "columnsList";
+
+
+
+}
diff --git a/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/kuduclient/KuduInputFormat.java b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/kuduclient/KuduInputFormat.java
new file mode 100644
index 00000000000..6c9a2e4629c
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/kuduclient/KuduInputFormat.java
@@ -0,0 +1,173 @@
+package org.apache.seatunnel.connectors.seatunnel.kudu.kuduclient;
+
+import org.apache.kudu.ColumnSchema;
+import org.apache.kudu.Schema;
+import org.apache.kudu.client.*;
+import org.apache.seatunnel.api.common.PrepareFailException;
+import org.apache.seatunnel.api.table.type.*;
+import org.apache.seatunnel.common.constants.PluginType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.sql.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+public class KuduInputFormat implements Serializable {
+ private static final Logger logger = LoggerFactory.getLogger(KuduInputFormat.class);
+
+ public KuduInputFormat(String kuduMaster,String tableName,String columnsList){
+ this.kuduMaster=kuduMaster;
+ this.columnsList=Arrays.asList(columnsList.split(","));
+ this.tableName=tableName;
+ // openInputFormat();
+ }
+ /**
+ * Declare the global variable KuduClient and use it to manipulate the Kudu table
+ */
+ public KuduClient kuduClient;
+
+ /**
+ * Specify kuduMaster address
+ */
+ public String kuduMaster;
+ public List columnsList;
+ public Schema schema;
+ public String keyColumn;
+
+ /**
+ * Specifies the name of the table
+ */
+ public String tableName;
+ public List getColumnsSchemas(){
+ List columns = null;
+ try {
+ schema = kuduClient.openTable(tableName).getSchema();
+ keyColumn = schema.getPrimaryKeyColumns().get(0).getName();
+ columns =schema.getColumns();
+ } catch (KuduException e) {
+ e.printStackTrace();
+ }
+ return columns;
+ }
+
+ public static SeaTunnelRow getSeaTunnelRowData(RowResult rs, SeaTunnelRowType typeInfo) throws SQLException {
+
+ List fields = new ArrayList<>();
+ SeaTunnelDataType>[] seaTunnelDataTypes = typeInfo.getFieldTypes();
+
+ for (int i = 0; i < seaTunnelDataTypes.length; i++) {
+ Object seatunnelField;
+ SeaTunnelDataType> seaTunnelDataType = seaTunnelDataTypes[i];
+ if (null == rs.getObject(i)) {
+ seatunnelField = null;
+ } else if (BasicType.BOOLEAN_TYPE.equals(seaTunnelDataType)) {
+ seatunnelField = rs.getBoolean(i);
+ } else if (BasicType.BYTE_TYPE.equals(seaTunnelDataType)) {
+ seatunnelField = rs.getByte(i);
+ } else if (BasicType.SHORT_TYPE.equals(seaTunnelDataType)) {
+ seatunnelField = rs.getShort(i);
+ } else if (BasicType.INT_TYPE.equals(seaTunnelDataType)) {
+ seatunnelField = rs.getInt(i);
+ } else if (BasicType.LONG_TYPE.equals(seaTunnelDataType)) {
+ seatunnelField = rs.getLong(i);
+ } else if (seaTunnelDataType instanceof DecimalType) {
+ Object value = rs.getObject(i);
+ seatunnelField = value instanceof BigInteger ?
+ new BigDecimal((BigInteger) value, 0)
+ : value;
+ } else if (BasicType.FLOAT_TYPE.equals(seaTunnelDataType)) {
+ seatunnelField = rs.getFloat(i);
+ } else if (BasicType.DOUBLE_TYPE.equals(seaTunnelDataType)) {
+ seatunnelField = rs.getDouble(i);
+ } else if (BasicType.STRING_TYPE.equals(seaTunnelDataType)) {
+ seatunnelField = rs.getString(i);
+ } else {
+ throw new IllegalStateException("Unexpected value: " + seaTunnelDataType);
+ }
+ fields.add(seatunnelField);
+ }
+
+ return new SeaTunnelRow(fields.toArray());
+ }
+
+ public SeaTunnelRowType getSeaTunnelRowType(List columnSchemaList) {
+
+ ArrayList> seaTunnelDataTypes = new ArrayList<>();
+ ArrayList fieldNames = new ArrayList<>();
+ try {
+
+ for (int i = 0; i < columnSchemaList.size(); i++) {
+ fieldNames.add(columnSchemaList.get(i).getName());
+ seaTunnelDataTypes.add(KuduTypeMapper.mapping(columnSchemaList, i));
+ }
+ } catch (Exception e) {
+ logger .warn("get row type info exception", e);
+ throw new PrepareFailException("kudu", PluginType.SOURCE, e.toString());
+ }
+ return new SeaTunnelRowType(fieldNames.toArray(new String[fieldNames.size()]), seaTunnelDataTypes.toArray(new SeaTunnelDataType>[seaTunnelDataTypes.size()]));
+ }
+
+ public void openInputFormat() {
+
+ KuduClient.KuduClientBuilder kuduClientBuilder = new
+ KuduClient.KuduClientBuilder(kuduMaster);
+ kuduClientBuilder.defaultOperationTimeoutMs(1800000);
+
+ kuduClient = kuduClientBuilder.build();
+
+ logger.info("The Kudu client is successfully initialized", kuduMaster, kuduClient);
+
+ }
+
+
+ /**
+ *
+ * @param lowerBound The beginning of each slice
+ * @param upperBound End of each slice
+ * @return Get the kuduScanner object for each slice
+ */
+ public KuduScanner getKuduBuildSplit(int lowerBound,int upperBound){
+ KuduScanner kuduScanner = null;
+ try {
+ KuduScanner.KuduScannerBuilder kuduScannerBuilder =
+ kuduClient.newScannerBuilder(kuduClient.openTable(tableName));
+
+ kuduScannerBuilder.setProjectedColumnNames(columnsList);
+
+ KuduPredicate lowerPred = KuduPredicate.newComparisonPredicate(
+ schema.getColumn(""+keyColumn),
+ KuduPredicate.ComparisonOp.GREATER_EQUAL,
+ lowerBound);
+
+ KuduPredicate upperPred = KuduPredicate.newComparisonPredicate(
+ schema.getColumn(""+keyColumn),
+ KuduPredicate.ComparisonOp.LESS,
+ upperBound);
+
+ kuduScanner = kuduScannerBuilder.addPredicate(lowerPred)
+ .addPredicate(upperPred).build();
+ } catch (KuduException e) {
+ e.printStackTrace();
+ logger .warn("get the Kuduscan object for each splice exception", e);
+ }
+ return kuduScanner;
+ }
+
+ public void closeInputFormat() {
+ if (kuduClient != null) {
+ try {
+ kuduClient.close();
+ } catch ( KuduException e) {
+ logger.warn("Kudu Client close failed.", e);
+ } finally {
+ kuduClient = null;
+ }
+ }
+
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/kuduclient/KuduOutputFormat.java b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/kuduclient/KuduOutputFormat.java
new file mode 100644
index 00000000000..40cd7f1a4b9
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/kuduclient/KuduOutputFormat.java
@@ -0,0 +1,165 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.kudu.kuduclient;
+
+
+import org.apache.kudu.ColumnSchema;
+import org.apache.kudu.Schema;
+import org.apache.kudu.client.*;
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+import org.apache.seatunnel.connectors.seatunnel.kudu.config.KuduSinkConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.sql.Timestamp;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * A Kudu outputFormat
+ */
+public class KuduOutputFormat
+ implements Serializable {
+ private static final Logger logger = LoggerFactory.getLogger(KuduOutputFormat.class);
+
+ private String kuduMaster;
+ private String kuduTableName;
+ private KuduClient kuduClient;
+ private KuduSession kuduSession;
+ private KuduTable kuduTable;
+
+
+ public KuduOutputFormat(KuduSinkConfig kuduSinkConfig) {
+ this.kuduMaster = kuduSinkConfig.getKuduMaster();
+ this.kuduTableName = kuduSinkConfig.getKuduTableName();
+ init();
+ }
+
+ public void write(SeaTunnelRow element) {
+
+ Insert insert = kuduTable.newInsert();
+ Schema schema = kuduTable.getSchema();
+
+ int columnCount = schema.getColumnCount();
+ PartialRow row = insert.getRow();
+ for (int columnIndex = 0; columnIndex < columnCount; columnIndex++) {
+ ColumnSchema col = schema.getColumnByIndex(columnIndex);
+ try {
+ switch (col.getType()) {
+ case BOOL:
+ row.addBoolean(columnIndex, (Boolean) element.getField(columnIndex));
+ break;
+ case INT8:
+ row.addByte(columnIndex, (Byte) element.getField(columnIndex));
+ break;
+ case INT16:
+ row.addShort(columnIndex, (Short) element.getField(columnIndex));
+ break;
+ case INT32:
+ row.addInt(columnIndex, (Integer) element.getField(columnIndex));
+ break;
+ case INT64:
+ row.addLong(columnIndex, (Long) element.getField(columnIndex));
+ break;
+ case UNIXTIME_MICROS:
+ if (element.getField(columnIndex) instanceof Timestamp) {
+ row.addTimestamp(columnIndex, (Timestamp) element.getField(columnIndex));
+ } else {
+ row.addLong(columnIndex, (Long) element.getField(columnIndex));
+ }
+ break;
+ case FLOAT:
+ row.addFloat(columnIndex, (Float) element.getField(columnIndex));
+ break;
+ case DOUBLE:
+ row.addDouble(columnIndex, (Double) element.getField(columnIndex));
+ break;
+ case STRING:
+ row.addString(columnIndex, element.getField(columnIndex).toString());
+ break;
+ case BINARY:
+ if (element.getField(columnIndex) instanceof byte[]) {
+ row.addBinary(columnIndex, (byte[]) element.getField(columnIndex));
+ } else {
+ row.addBinary(columnIndex, (ByteBuffer) element.getField(columnIndex));
+ }
+ break;
+ case DECIMAL:
+ row.addDecimal(columnIndex, (BigDecimal) element.getField(columnIndex));
+ break;
+ default:
+ throw new IllegalArgumentException("Unsupported column type: " + col.getType());
+ }
+ } catch (ClassCastException e) {
+ e.printStackTrace();
+ throw new IllegalArgumentException(
+ "Value type does not match column type " + col.getType() +
+ " for column " + col.getName());
+ }
+
+ }
+
+ try {
+ kuduSession.apply(insert);
+
+ } catch (KuduException e) {
+ e.printStackTrace();
+ }
+
+ }
+
+ public void init() {
+
+
+ KuduClient.KuduClientBuilder kuduClientBuilder = new
+ KuduClient.KuduClientBuilder(kuduMaster);
+ kuduClientBuilder.defaultOperationTimeoutMs(1800000);
+
+ this.kuduClient = kuduClientBuilder.build();
+ this.kuduSession = kuduClient.newSession();
+ this.kuduSession.setTimeoutMillis(100000);
+ this.kuduSession.setFlushMode(SessionConfiguration.FlushMode.AUTO_FLUSH_SYNC);
+
+ try {
+ kuduTable = kuduClient.openTable(kuduTableName);
+ } catch (KuduException e) {
+ e.printStackTrace();
+ }
+
+
+ logger.info("The Kudu client is successfully initialized", kuduMaster, kuduClient);
+ }
+
+
+ public void closeOutputFormat() {
+ if (kuduClient != null) {
+ try {
+ kuduClient.close();
+ kuduSession.close();
+ } catch ( KuduException e) {
+ logger.warn("Kudu Client close failed.", e);
+ } finally {
+ kuduClient = null;
+ kuduSession = null;
+ }
+ }
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/kuduclient/KuduTypeMapper.java b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/kuduclient/KuduTypeMapper.java
new file mode 100644
index 00000000000..4f63c61359d
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/kuduclient/KuduTypeMapper.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.kudu.kuduclient;
+
+import org.apache.kudu.ColumnSchema;
+import org.apache.seatunnel.api.table.type.*;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.util.List;
+
+public class KuduTypeMapper {
+
+ private static final Logger LOG = LoggerFactory.getLogger(KuduTypeMapper.class);
+
+ // ============================data types=====================
+
+ private static final String KUDU_UNKNOWN = "UNKNOWN";
+ private static final String KUDU_BIT = "BOOL";
+
+ // -------------------------number----------------------------
+ private static final String KUDU_TINYINT = "INT8";
+ private static final String KUDU_MEDIUMINT = "INT32";
+ private static final String KUDU_INT = "INT16";
+ private static final String KUDU_BIGINT = "INT64";
+
+ private static final String KUDU_FLOAT = "FLOAT";
+
+ private static final String KUDU_DOUBLE = "DOUBLE";
+ private static final String KUDU_DECIMAL = "DECIMAL32";
+
+
+ // -------------------------string----------------------------
+
+ private static final String KUDU_VARCHAR = "STRING";
+
+
+ // ------------------------------time-------------------------
+
+ private static final String KUDU_UNIXTIME_MICROS = "UNIXTIME_MICROS";
+
+
+ // ------------------------------blob-------------------------
+
+ private static final String KUDU_BINARY = "BINARY";
+
+
+
+
+ public static SeaTunnelDataType> mapping(List columnSchemaList, int colIndex) throws SQLException {
+ String KUDUType = columnSchemaList.get(colIndex).getType().getName().toUpperCase();
+
+ switch (KUDUType) {
+ case KUDU_BIT:
+ return BasicType.BOOLEAN_TYPE;
+ case KUDU_TINYINT:
+ case KUDU_MEDIUMINT:
+ case KUDU_INT:
+ return BasicType.INT_TYPE;
+ case KUDU_BIGINT:
+ return BasicType.LONG_TYPE;
+ case KUDU_DECIMAL:
+ return new DecimalType(20, 0);
+ case KUDU_FLOAT:
+ return BasicType.FLOAT_TYPE;
+ case KUDU_DOUBLE:
+ return BasicType.DOUBLE_TYPE;
+
+ case KUDU_VARCHAR:
+ return BasicType.STRING_TYPE;
+ case KUDU_UNIXTIME_MICROS:
+ return LocalTimeType.LOCAL_DATE_TIME_TYPE;
+ case KUDU_BINARY:
+ return PrimitiveByteArrayType.INSTANCE;
+
+ //Doesn't support yet
+
+ case KUDU_UNKNOWN:
+ default:
+ throw new UnsupportedOperationException(
+ String.format(
+ "Doesn't support KUDU type '%s' .",
+ KUDUType));
+ }
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/sink/KuduAggregatedCommitInfo.java b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/sink/KuduAggregatedCommitInfo.java
new file mode 100644
index 00000000000..054bdf8f112
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/sink/KuduAggregatedCommitInfo.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.kudu.sink;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+import java.io.Serializable;
+import java.util.Map;
+
+@Data
+@AllArgsConstructor
+public class KuduAggregatedCommitInfo implements Serializable {
+
+
+}
diff --git a/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/sink/KuduCommitInfo.java b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/sink/KuduCommitInfo.java
new file mode 100644
index 00000000000..271139fe88e
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/sink/KuduCommitInfo.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.kudu.sink;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+import java.io.Serializable;
+import java.util.Map;
+
+@Data
+@AllArgsConstructor
+public class KuduCommitInfo implements Serializable {
+
+
+
+}
diff --git a/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/sink/KuduSink.java b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/sink/KuduSink.java
new file mode 100644
index 00000000000..6c40252f885
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/sink/KuduSink.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.kudu.sink;
+
+import com.google.auto.service.AutoService;
+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.shade.com.typesafe.config.Config;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * Kudu Sink implementation by using SeaTunnel sink API.
+ * This class contains the method to create {@link KuduSinkWriter} and {@link KuduSinkAggregatedCommitter}.
+ */
+@AutoService(SeaTunnelSink.class)
+public class KuduSink implements SeaTunnelSink {
+
+ private Config config;
+ private SeaTunnelRowType seaTunnelRowType;
+
+ @Override
+ public String getPluginName() {
+ return "kuduSink";
+ }
+
+ @Override
+ public void setTypeInfo(SeaTunnelRowType seaTunnelRowType) {
+ this.seaTunnelRowType = seaTunnelRowType;
+ }
+
+ @Override
+ public SeaTunnelDataType getConsumedType() {
+ return this.seaTunnelRowType;
+ }
+
+ @Override
+ public void prepare(Config pluginConfig) throws PrepareFailException {
+ this.config = pluginConfig;
+ }
+
+ @Override
+ public SinkWriter createWriter(SinkWriter.Context context) throws IOException {
+ return new KuduSinkWriter(seaTunnelRowType, config, context, System.currentTimeMillis());
+ }
+
+ @Override
+ public SinkWriter restoreWriter(SinkWriter.Context context, List states) throws IOException {
+ return new KuduSinkWriter(seaTunnelRowType, config, context, System.currentTimeMillis());
+ }
+
+ @Override
+ public Optional> getCommitInfoSerializer() {
+ return Optional.of(new DefaultSerializer<>());
+ }
+
+ @Override
+ public Optional> createAggregatedCommitter() throws IOException {
+ return Optional.of(new KuduSinkAggregatedCommitter());
+ }
+
+ @Override
+ public Optional> getAggregatedCommitInfoSerializer() {
+ return Optional.of(new DefaultSerializer<>());
+ }
+
+
+}
diff --git a/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/sink/KuduSinkAggregatedCommitter.java b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/sink/KuduSinkAggregatedCommitter.java
new file mode 100644
index 00000000000..f06d5e89919
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/sink/KuduSinkAggregatedCommitter.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.kudu.sink;
+
+import org.apache.seatunnel.api.sink.SinkAggregatedCommitter;
+
+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 KuduSinkAggregatedCommitter implements SinkAggregatedCommitter {
+ private static final Logger LOGGER = LoggerFactory.getLogger(KuduSinkAggregatedCommitter.class);
+
+
+ @Override
+ public List commit(List aggregatedCommitInfo) throws IOException {
+ return null;
+ }
+
+ @Override
+ public KuduAggregatedCommitInfo combine(List commitInfos) {
+ return null;
+ }
+
+ @Override
+ public void abort(List aggregatedCommitInfo) throws Exception {
+
+ }
+
+ @Override
+ public void close() throws IOException {
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/sink/KuduSinkState.java b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/sink/KuduSinkState.java
new file mode 100644
index 00000000000..9d32b00354e
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/sink/KuduSinkState.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.kudu.sink;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+import java.io.Serializable;
+
+@Data
+@AllArgsConstructor
+public class KuduSinkState implements Serializable {
+
+}
diff --git a/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/sink/KuduSinkWriter.java b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/sink/KuduSinkWriter.java
new file mode 100644
index 00000000000..1fd9e560aec
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/sink/KuduSinkWriter.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.kudu.sink;
+
+import lombok.NonNull;
+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.kudu.config.KuduSinkConfig;
+import org.apache.seatunnel.connectors.seatunnel.kudu.kuduclient.KuduOutputFormat;
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Optional;
+
+public class KuduSinkWriter implements SinkWriter {
+ private static final Logger LOGGER = LoggerFactory.getLogger(KuduSinkWriter.class);
+
+ private SeaTunnelRowType seaTunnelRowType;
+ private Config pluginConfig;
+ private Context context;
+ private long jobId;
+
+ private KuduOutputFormat fileWriter;
+
+ private KuduSinkConfig kuduSinkConfig;
+
+ public KuduSinkWriter(@NonNull SeaTunnelRowType seaTunnelRowType,
+ @NonNull Config pluginConfig,
+ @NonNull Context context,
+ long jobId) {
+ this.seaTunnelRowType = seaTunnelRowType;
+ this.pluginConfig = pluginConfig;
+ this.context = context;
+ this.jobId = jobId;
+
+ kuduSinkConfig = new KuduSinkConfig(this.pluginConfig);
+ fileWriter = new KuduOutputFormat(kuduSinkConfig);
+
+ }
+
+ @Override
+ public void write(SeaTunnelRow element) throws IOException {
+ fileWriter.write(element);
+ }
+
+ @Override
+ public Optional prepareCommit() throws IOException {
+ return Optional.empty();
+ }
+
+ @Override
+ public void abortPrepare() {
+
+ }
+
+
+ @Override
+ public void close() throws IOException {
+ fileWriter.closeOutputFormat();
+ }
+
+
+}
diff --git a/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/source/KuduSource.java b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/source/KuduSource.java
new file mode 100644
index 00000000000..0d6f014a444
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/source/KuduSource.java
@@ -0,0 +1,193 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.kudu.source;
+
+import com.google.auto.service.AutoService;
+import org.apache.kudu.ColumnSchema;
+import org.apache.kudu.client.*;
+import org.apache.seatunnel.api.common.PrepareFailException;
+import org.apache.seatunnel.api.common.SeaTunnelContext;
+import org.apache.seatunnel.api.serialization.DefaultSerializer;
+import org.apache.seatunnel.api.serialization.Serializer;
+import org.apache.seatunnel.api.source.Boundedness;
+import org.apache.seatunnel.api.source.SeaTunnelSource;
+import org.apache.seatunnel.api.source.SourceReader;
+import org.apache.seatunnel.api.source.SourceSplitEnumerator;
+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.PluginType;
+import org.apache.seatunnel.connectors.seatunnel.kudu.config.KuduSourceConfig;
+import org.apache.seatunnel.connectors.seatunnel.kudu.kuduclient.KuduInputFormat;
+import org.apache.seatunnel.connectors.seatunnel.kudu.kuduclient.KuduTypeMapper;
+import org.apache.seatunnel.connectors.seatunnel.kudu.state.KuduSinkState;
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+@AutoService(SeaTunnelSource.class)
+public class KuduSource implements SeaTunnelSource {
+ private static final Logger LOGGER = LoggerFactory.getLogger(KuduSource.class);
+
+ private Config pluginConfig;
+ private SeaTunnelContext seaTunnelContext;
+ private SeaTunnelRowType rowTypeInfo;
+ private KuduInputFormat kuduInputFormat;
+ private PartitionParameter partitionParameter;
+
+ @Override
+ public Boundedness getBoundedness() {
+ return Boundedness.BOUNDED;
+ }
+
+ @Override
+ public SeaTunnelRowType getProducedType() {
+ return this.rowTypeInfo;
+ }
+
+ @Override
+ public SourceReader createReader(SourceReader.Context readerContext) {
+ return new KuduSourceReader(kuduInputFormat,readerContext);
+ }
+
+ @Override
+ public Serializer getSplitSerializer() {
+ return SeaTunnelSource.super.getSplitSerializer();
+ }
+
+ @Override
+ public SourceSplitEnumerator createEnumerator(
+ SourceSplitEnumerator.Context enumeratorContext) {
+ return new KuduSourceSplitEnumerator(enumeratorContext,partitionParameter);
+ }
+
+ @Override
+ public SourceSplitEnumerator restoreEnumerator(
+ SourceSplitEnumerator.Context enumeratorContext, KuduSinkState checkpointState) {
+ // todo:
+ return new KuduSourceSplitEnumerator(enumeratorContext,partitionParameter);
+ }
+
+ @Override
+ public Serializer getEnumeratorStateSerializer() {
+ return new DefaultSerializer<>();
+ }
+
+ @Override
+ public String getPluginName() {
+ return "KuduSource";
+ }
+
+ @Override
+ public void prepare(Config config) {
+
+ String kudumaster = config.getString(KuduSourceConfig.kuduMaster);
+ String tableName = config.getString(KuduSourceConfig.tableName);
+ String columnslist = config.getString(KuduSourceConfig.columnsList);
+ kuduInputFormat=new KuduInputFormat(kudumaster,tableName,columnslist);
+ try {
+ KuduClient.KuduClientBuilder kuduClientBuilder = new
+ KuduClient.KuduClientBuilder(kudumaster);
+ kuduClientBuilder.defaultOperationTimeoutMs(1800000);
+
+ KuduClient kuduClient = kuduClientBuilder.build();
+ partitionParameter = initPartitionParameter(kuduClient,tableName);
+ SeaTunnelRowType seaTunnelRowType =getSeaTunnelRowType(kuduClient.openTable(tableName).getSchema().getColumns());
+ rowTypeInfo=seaTunnelRowType;
+ } catch (KuduException e) {
+ e.printStackTrace();
+ }
+ }
+
+ private PartitionParameter initPartitionParameter(KuduClient kuduClient,String tableName) {
+ String keyColumn = null;
+ int maxKey=0;
+ int minKey=0;
+ boolean flag=true;
+ try {
+ KuduScanner.KuduScannerBuilder kuduScannerBuilder =
+ kuduClient.newScannerBuilder(kuduClient.openTable(tableName));
+ ArrayList columnsList = new ArrayList();
+ keyColumn = kuduClient.openTable(tableName).getSchema().getPrimaryKeyColumns().get(0).getName();
+ columnsList.add(""+keyColumn);
+ kuduScannerBuilder.setProjectedColumnNames(columnsList);
+ KuduScanner kuduScanner = kuduScannerBuilder.build();
+
+
+ while (kuduScanner.hasMoreRows()) {
+ RowResultIterator rowResults = kuduScanner.nextRows();
+ while (rowResults.hasNext()) {
+ RowResult row = rowResults.next();
+ int id = row.getInt(""+keyColumn);
+ if (flag){
+ maxKey=id;
+ minKey=id;
+ flag=false;
+ }else {
+ if (id>=maxKey){
+ maxKey=id;
+ }
+ if (id<=minKey){
+ minKey=id;
+ }
+ }
+ }
+ }
+ } catch (KuduException e) {
+ e.printStackTrace();
+ }
+
+
+ return new PartitionParameter(keyColumn, Long.parseLong(minKey+""), Long.parseLong(maxKey+""));
+ }
+
+
+ /* @Override
+ public SeaTunnelContext getSeaTunnelContext() {
+ return seaTunnelContext;
+ }*/
+
+ @Override
+ public void setSeaTunnelContext(SeaTunnelContext seaTunnelContext) {
+ this.seaTunnelContext = seaTunnelContext;
+ }
+
+
+ public SeaTunnelRowType getSeaTunnelRowType(List columnSchemaList) {
+
+ ArrayList> seaTunnelDataTypes = new ArrayList<>();
+ ArrayList fieldNames = new ArrayList<>();
+ try {
+
+ for (int i = 0; i < columnSchemaList.size(); i++) {
+ fieldNames.add(columnSchemaList.get(i).getName());
+ seaTunnelDataTypes.add(KuduTypeMapper.mapping(columnSchemaList, i));
+ }
+
+ } catch (Exception e) {
+ LOGGER.warn("get row type info exception", e);
+ throw new PrepareFailException("jdbc", PluginType.SOURCE, e.toString());
+ }
+ return new SeaTunnelRowType(fieldNames.toArray(new String[fieldNames.size()]), seaTunnelDataTypes.toArray(new SeaTunnelDataType>[seaTunnelDataTypes.size()]));
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/source/KuduSourceReader.java b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/source/KuduSourceReader.java
new file mode 100644
index 00000000000..b235744f216
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/source/KuduSourceReader.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.kudu.source;
+
+import org.apache.kudu.ColumnSchema;
+import org.apache.kudu.client.KuduException;
+import org.apache.kudu.client.KuduScanner;
+import org.apache.kudu.client.RowResult;
+import org.apache.kudu.client.RowResultIterator;
+import org.apache.seatunnel.api.source.Boundedness;
+import org.apache.seatunnel.api.source.Collector;
+import org.apache.seatunnel.api.source.SourceReader;
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+import org.apache.seatunnel.connectors.seatunnel.kudu.kuduclient.KuduInputFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.sql.SQLException;
+import java.util.*;
+import java.util.concurrent.ThreadLocalRandom;
+
+public class KuduSourceReader implements SourceReader {
+
+ private static final Logger LOGGER = LoggerFactory.getLogger(KuduSourceReader.class);
+
+ private final Context context;
+
+ private final KuduInputFormat kuduInputFormat;
+ Deque splits = new LinkedList<>();
+
+ boolean noMoreSplit;
+
+ public KuduSourceReader(KuduInputFormat kuduInputFormat, Context context) {
+ this.context = context;
+ this.kuduInputFormat = kuduInputFormat;
+ }
+
+ @Override
+ public void open() {
+ kuduInputFormat.openInputFormat();
+ }
+
+ @Override
+ public void close() {
+ kuduInputFormat.closeInputFormat();
+ }
+
+ @Override
+ @SuppressWarnings("magicnumber")
+ public void pollNext(Collector output) throws InterruptedException, KuduException, SQLException {
+ KuduSourceSplit split = splits.poll();
+ Object[] parameterValues = split.parameterValues;
+
+ int lowerBound=Integer.parseInt(parameterValues[0].toString());
+
+ int upperBound=Integer.parseInt(parameterValues[1].toString());
+
+
+ List columnSchemaList = kuduInputFormat.getColumnsSchemas();
+ KuduScanner kuduScanner = kuduInputFormat.getKuduBuildSplit( lowerBound, upperBound);
+ //
+ while (kuduScanner.hasMoreRows()) {
+ RowResultIterator rowResults = kuduScanner.nextRows();
+ while (rowResults.hasNext()) {
+ RowResult rowResult = rowResults.next();
+ SeaTunnelRow seaTunnelRow = KuduInputFormat.getSeaTunnelRowData(rowResult, kuduInputFormat.getSeaTunnelRowType(columnSchemaList));
+ output.collect(seaTunnelRow);
+ }
+ }
+
+
+ // Generate a random number of rows to emit.
+
+ if (Boundedness.BOUNDED.equals(context.getBoundedness())) {
+ // signal to the source that we have reached the end of the data.
+ LOGGER.info("Closed the bounded fake source");
+ context.signalNoMoreElement();
+ }
+
+ }
+
+ @Override
+ public List snapshotState(long checkpointId) {
+ return null;
+ }
+
+ @Override
+ public void addSplits(List splits) {
+ this.splits.addAll(splits);
+ }
+
+ @Override
+ public void handleNoMoreSplits() {
+ noMoreSplit = true;
+ }
+
+ @Override
+ public void notifyCheckpointComplete(long checkpointId) {
+
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/source/KuduSourceSplit.java b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/source/KuduSourceSplit.java
new file mode 100644
index 00000000000..2c92fb13fbf
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/source/KuduSourceSplit.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.kudu.source;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import org.apache.seatunnel.api.source.SourceSplit;
+@Data
+@AllArgsConstructor
+public class KuduSourceSplit implements SourceSplit {
+
+ private static final long serialVersionUID = -1L;
+
+ Object[] parameterValues;
+ public final Integer splitId;
+
+ @Override
+ public String splitId() {
+ return splitId.toString();
+ }
+
+}
diff --git a/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/source/KuduSourceSplitEnumerator.java b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/source/KuduSourceSplitEnumerator.java
new file mode 100644
index 00000000000..66c8f92dbce
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/source/KuduSourceSplitEnumerator.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.kudu.source;
+
+import org.apache.seatunnel.api.source.SourceSplitEnumerator;
+
+import org.apache.seatunnel.connectors.seatunnel.kudu.state.KuduSinkState;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+public class KuduSourceSplitEnumerator implements SourceSplitEnumerator {
+
+ private final Context enumeratorContext;
+ private PartitionParameter partitionParameter;
+ List allSplit = new ArrayList<>();
+ private Long maxVal;
+ private Long minVal;
+ private Long batchSize;
+ private Integer batchNum;
+
+ public KuduSourceSplitEnumerator(Context enumeratorContext,PartitionParameter partitionParameter) {
+ this.enumeratorContext = enumeratorContext;
+ this.partitionParameter=partitionParameter;
+ }
+
+ @Override
+ public void open() {
+
+ }
+
+ @Override
+ public void run() {
+
+ }
+
+ @Override
+ public void close() throws IOException {
+
+ }
+
+ @Override
+ public void addSplitsBack(List splits, int subtaskId) {
+
+ }
+
+ @Override
+ public int currentUnassignedSplitSize() {
+ return 0;
+ }
+
+ @Override
+ public void handleSplitRequest(int subtaskId) {
+
+ }
+
+ @Override
+ public void registerReader(int subtaskId) {
+ int parallelism = enumeratorContext.currentParallelism();
+ if (allSplit.isEmpty()) {
+ if (null != partitionParameter) {
+ Serializable[][] parameterValues = getParameterValues(partitionParameter.minValue, partitionParameter.maxValue,parallelism);
+ for (int i = 0; i < parameterValues.length; i++) {
+ allSplit.add(new KuduSourceSplit(parameterValues[i], i));
+ }
+ } else {
+ allSplit.add(new KuduSourceSplit(null, 0));
+ }
+ }
+ // Filter the split that the current task needs to run
+ List splits = allSplit.stream().filter(p -> p.splitId % parallelism == subtaskId).collect(Collectors.toList());
+ enumeratorContext.assignSplit(subtaskId, splits);
+ enumeratorContext.signalNoMoreSplits(subtaskId);
+ }
+
+ private Serializable[][] getParameterValues(Long minVal, Long maxVal, int parallelism) {
+ this.maxVal=maxVal;
+ this.minVal=minVal;
+ long maxElemCount = (maxVal - minVal) + 1;
+ batchNum=parallelism;
+ getBatchSizeAndBatchNum(parallelism);
+ long bigBatchNum = maxElemCount - (batchSize - 1) * batchNum;
+
+ Serializable[][] parameters = new Serializable[batchNum][2];
+ long start = minVal;
+ for (int i = 0; i < batchNum; i++) {
+ long end = start + batchSize - 1 - (i >= bigBatchNum ? 1 : 0);
+ parameters[i] = new Long[] {start, end};
+ start = end + 1;
+ }
+ return parameters;
+
+ }
+
+ private void getBatchSizeAndBatchNum(int parallelism) {
+ batchNum=parallelism;
+ long maxElemCount = (maxVal - minVal) + 1;
+ if (batchNum > maxElemCount) {
+ batchNum = (int) maxElemCount;
+ }
+ this.batchNum = batchNum;
+ this.batchSize = new Double(Math.ceil((double) maxElemCount / batchNum)).longValue();
+ }
+
+ @Override
+ public KuduSinkState snapshotState(long checkpointId) throws Exception {
+ return null;
+ }
+
+ @Override
+ public void notifyCheckpointComplete(long checkpointId) throws Exception {
+
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/source/PartitionParameter.java b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/source/PartitionParameter.java
new file mode 100644
index 00000000000..e791164667c
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/source/PartitionParameter.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.kudu.source;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+import java.io.Serializable;
+
+@Data
+@AllArgsConstructor
+public class PartitionParameter implements Serializable {
+
+ String partitionColumnName;
+ Long minValue;
+ Long maxValue;
+}
diff --git a/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/state/KuduSinkState.java b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/state/KuduSinkState.java
new file mode 100644
index 00000000000..f50416ee29a
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-kudu/src/main/java/org/apache/seatunnel/connectors/seatunnel/kudu/state/KuduSinkState.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.kudu.state;
+
+import java.io.Serializable;
+
+public class KuduSinkState implements Serializable {
+}
diff --git a/seatunnel-connectors-v2/connector-kudu/src/main/resources/kudu_to_kudu_flink.conf b/seatunnel-connectors-v2/connector-kudu/src/main/resources/kudu_to_kudu_flink.conf
new file mode 100644
index 00000000000..b04aeae6898
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-kudu/src/main/resources/kudu_to_kudu_flink.conf
@@ -0,0 +1,60 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+######
+###### This config file is a demonstration of streaming processing in seatunnel config
+######
+
+env {
+ # You can set flink configuration here
+ execution.parallelism = 2
+ #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**
+ KuduSource {
+ result_table_name = "studentlyh2"
+ kudu_master = "192.168.88.110:7051"
+ kudu_table = "studentlyh2"
+ columnsList = "id,name,age,sex"
+ }
+
+ # 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 id,name,age,sex from studentlyh2"
+ }
+
+ # 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 {
+ kuduSink {
+ kudu_master = "192.168.88.110:7051"
+ kudu_table = "studentlyhresultflink"
+ save_mode="append"
+ }
+
+ # 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-connectors-v2/connector-kudu/src/main/resources/kudu_to_kudu_spark.conf b/seatunnel-connectors-v2/connector-kudu/src/main/resources/kudu_to_kudu_spark.conf
new file mode 100644
index 00000000000..cb4ecbaa8e5
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-kudu/src/main/resources/kudu_to_kudu_spark.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
+ spark.app.name = "SeaTunnel"
+ spark.executor.instances = 2
+ spark.executor.cores = 2
+ spark.executor.memory = "1g"
+ spark.master = local
+ #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**
+ KuduSource {
+ result_table_name = "studentlyh2"
+ kudu_master = "192.168.88.110:7051"
+ kudu_table = "studentlyh2"
+ columnsList = "id,name,age,sex"
+ }
+
+ # 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 id,name,age,sex from studentlyh2"
+ }
+
+ # 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 {
+ kuduSink {
+ kudu_master = "192.168.88.110:7051"
+ kudu_table = "studentlyhresult"
+ save_mode="append"
+ }
+
+ # 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-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/config/SourceProperties.java b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/config/SourceProperties.java
index e8692322059..92453efee4c 100644
--- a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/config/SourceProperties.java
+++ b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/config/SourceProperties.java
@@ -19,61 +19,48 @@
public class SourceProperties {
- // Pulsar client API config prefix.
- public static final String CLIENT_CONFIG_PREFIX = "pulsar.client.";
- // Pulsar admin API config prefix.
- public static final String ADMIN_CONFIG_PREFIX = "pulsar.admin.";
-
// --------------------------------------------------------------------------------------------
// The configuration for ClientConfigurationData part.
- // All the configuration listed below should have the pulsar.client prefix.
// --------------------------------------------------------------------------------------------
- public static final String PULSAR_SERVICE_URL = CLIENT_CONFIG_PREFIX + "serviceUrl";
- public static final String PULSAR_AUTH_PLUGIN_CLASS_NAME = CLIENT_CONFIG_PREFIX + "authPluginClassName";
- public static final String PULSAR_AUTH_PARAMS = CLIENT_CONFIG_PREFIX + "authParams";
+ public static final String CLIENT_SERVICE_URL = "client.service-url";
+ public static final String AUTH_PLUGIN_CLASS = "auth.plugin-class";
+ public static final String AUTH_PARAMS = "auth.params";
// --------------------------------------------------------------------------------------------
// The configuration for ClientConfigurationData part.
// All the configuration listed below should have the pulsar.client prefix.
// --------------------------------------------------------------------------------------------
- public static final String PULSAR_ADMIN_URL = ADMIN_CONFIG_PREFIX + "adminUrl";
-
- // Pulsar source connector config prefix.
- public static final String SOURCE_CONFIG_PREFIX = "pulsar.source.";
- // Pulsar consumer API config prefix.
- public static final String CONSUMER_CONFIG_PREFIX = "pulsar.consumer.";
+ public static final String ADMIN_SERVICE_URL = "admin.service-url";
// --------------------------------------------------------------------------------------------
// The configuration for ConsumerConfigurationData part.
- // All the configuration listed below should have the pulsar.consumer prefix.
// --------------------------------------------------------------------------------------------
- public static final String PULSAR_SUBSCRIPTION_NAME = CONSUMER_CONFIG_PREFIX + "subscriptionName";
- public static final String PULSAR_SUBSCRIPTION_TYPE = CONSUMER_CONFIG_PREFIX + "subscriptionType";
- public static final String PULSAR_SUBSCRIPTION_MODE = CONSUMER_CONFIG_PREFIX + "subscriptionMode";
+ public static final String SUBSCRIPTION_NAME = "subscription.name";
+ public static final String SUBSCRIPTION_TYPE = "subscription.type";
+ public static final String SUBSCRIPTION_MODE = "subscription.mode";
// --------------------------------------------------------------------------------------------
// The configuration for pulsar source part.
- // All the configuration listed below should have the pulsar.source prefix.
// --------------------------------------------------------------------------------------------
- public static final String PULSAR_PARTITION_DISCOVERY_INTERVAL_MS = SOURCE_CONFIG_PREFIX + "partitionDiscoveryIntervalMs";
- public static final String PULSAR_TOPIC = SOURCE_CONFIG_PREFIX + "topic";
- public static final String PULSAR_TOPIC_PATTERN = SOURCE_CONFIG_PREFIX + "topic.pattern";
- public static final String PULSAR_POLL_TIMEOUT = SOURCE_CONFIG_PREFIX + "poll.timeout";
- public static final String PULSAR_POLL_INTERVAL = SOURCE_CONFIG_PREFIX + "poll.interval";
- public static final String PULSAR_BATCH_SIZE = SOURCE_CONFIG_PREFIX + "batch.size";
- public static final String PULSAR_CURSOR_START_MODE = SOURCE_CONFIG_PREFIX + "scan.cursor.start.mode";
- public static final String PULSAR_CURSOR_START_RESET_MODE = SOURCE_CONFIG_PREFIX + "scan.cursor.start.reset.mode";
- public static final String PULSAR_CURSOR_START_TIMESTAMP = SOURCE_CONFIG_PREFIX + "scan.cursor.start.timestamp";
- public static final String PULSAR_CURSOR_START_ID = SOURCE_CONFIG_PREFIX + "scan.cursor.start.id";
- public static final String PULSAR_CURSOR_STOP_MODE = SOURCE_CONFIG_PREFIX + "scan.cursor.stop.mode";
- public static final String PULSAR_CURSOR_STOP_TIMESTAMP = SOURCE_CONFIG_PREFIX + "scan.cursor.stop.timestamp";
+ public static final String TOPIC_DISCOVERY_INTERVAL = "topic-discovery.interval";
+ public static final String TOPIC = "topic";
+ public static final String TOPIC_PATTERN = "topic-pattern";
+ public static final String POLL_TIMEOUT = "poll.timeout";
+ public static final String POLL_INTERVAL = "poll.interval";
+ public static final String POLL_BATCH_SIZE = "poll.batch.size";
+ public static final String CURSOR_STARTUP_MODE = "cursor.startup.mode";
+ public static final String CURSOR_RESET_MODE = "cursor.reset.mode";
+ public static final String CURSOR_STARTUP_TIMESTAMP = "cursor.startup.timestamp";
+ public static final String CURSOR_STARTUP_ID = "cursor.startup.id";
+ public static final String CURSOR_STOP_MODE = "cursor.stop.mode";
+ public static final String CURSOR_STOP_TIMESTAMP = "cursor.stop.timestamp";
/**
- * Startup mode for the Kafka consumer, see {@link #PULSAR_CURSOR_START_MODE}.
+ * Startup mode for the pulsar consumer, see {@link #CURSOR_STARTUP_MODE}.
*/
public enum StartMode {
/**
@@ -99,7 +86,7 @@ public enum StartMode {
}
/**
- * Startup mode for the Kafka consumer, see {@link #PULSAR_CURSOR_START_MODE}.
+ * Startup mode for the pulsar consumer, see {@link #CURSOR_STARTUP_MODE}.
*/
public enum StopMode {
/**
diff --git a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java
index 20028efb071..88c36d81631 100644
--- a/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java
+++ b/seatunnel-connectors-v2/connector-pulsar/src/main/java/org/apache/seatunnel/connectors/seatunnel/pulsar/source/PulsarSource.java
@@ -19,30 +19,28 @@
import static org.apache.seatunnel.common.PropertiesUtil.getEnum;
import static org.apache.seatunnel.common.PropertiesUtil.setOption;
-import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.PULSAR_ADMIN_URL;
-import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.PULSAR_AUTH_PARAMS;
-import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.PULSAR_AUTH_PLUGIN_CLASS_NAME;
-import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.PULSAR_BATCH_SIZE;
-import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.PULSAR_CURSOR_START_MODE;
-import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.PULSAR_CURSOR_START_RESET_MODE;
-import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.PULSAR_CURSOR_START_TIMESTAMP;
-import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.PULSAR_CURSOR_STOP_MODE;
-import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.PULSAR_CURSOR_STOP_TIMESTAMP;
-import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS;
-import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.PULSAR_POLL_INTERVAL;
-import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.PULSAR_POLL_TIMEOUT;
-import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.PULSAR_SERVICE_URL;
-import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.PULSAR_SUBSCRIPTION_NAME;
-import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.PULSAR_TOPIC;
-import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.PULSAR_TOPIC_PATTERN;
+import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.ADMIN_SERVICE_URL;
+import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.AUTH_PARAMS;
+import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.AUTH_PLUGIN_CLASS;
+import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.CLIENT_SERVICE_URL;
+import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.CURSOR_RESET_MODE;
+import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.CURSOR_STARTUP_MODE;
+import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.CURSOR_STARTUP_TIMESTAMP;
+import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.CURSOR_STOP_MODE;
+import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.CURSOR_STOP_TIMESTAMP;
+import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.POLL_BATCH_SIZE;
+import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.POLL_INTERVAL;
+import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.POLL_TIMEOUT;
+import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.SUBSCRIPTION_NAME;
import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.StartMode;
import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.StartMode.LATEST;
import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.StopMode.NEVER;
+import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.TOPIC;
+import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.TOPIC_DISCOVERY_INTERVAL;
+import static org.apache.seatunnel.connectors.seatunnel.pulsar.config.SourceProperties.TOPIC_PATTERN;
import org.apache.seatunnel.api.common.PrepareFailException;
-import org.apache.seatunnel.api.serialization.DefaultSerializer;
import org.apache.seatunnel.api.serialization.DeserializationSchema;
-import org.apache.seatunnel.api.serialization.Serializer;
import org.apache.seatunnel.api.source.Boundedness;
import org.apache.seatunnel.api.source.SeaTunnelSource;
import org.apache.seatunnel.api.source.SourceReader;
@@ -100,48 +98,48 @@ public String getPluginName() {
@SuppressWarnings("checkstyle:MagicNumber")
@Override
public void prepare(Config config) throws PrepareFailException {
- CheckResult result = CheckConfigUtil.checkAllExists(config, PULSAR_SUBSCRIPTION_NAME, PULSAR_SERVICE_URL, PULSAR_ADMIN_URL);
+ CheckResult result = CheckConfigUtil.checkAllExists(config, SUBSCRIPTION_NAME, CLIENT_SERVICE_URL, ADMIN_SERVICE_URL);
if (!result.isSuccess()) {
throw new PrepareFailException(getPluginName(), PluginType.SOURCE, result.getMsg());
}
// admin config
PulsarAdminConfig.Builder adminConfigBuilder = PulsarAdminConfig.builder()
- .adminUrl(config.getString(PULSAR_ADMIN_URL));
- setOption(config, PULSAR_AUTH_PLUGIN_CLASS_NAME, config::getString, adminConfigBuilder::authPluginClassName);
- setOption(config, PULSAR_AUTH_PARAMS, config::getString, adminConfigBuilder::authParams);
+ .adminUrl(config.getString(ADMIN_SERVICE_URL));
+ setOption(config, AUTH_PLUGIN_CLASS, config::getString, adminConfigBuilder::authPluginClassName);
+ setOption(config, AUTH_PARAMS, config::getString, adminConfigBuilder::authParams);
this.adminConfig = adminConfigBuilder.build();
// client config
PulsarClientConfig.Builder clientConfigBuilder = PulsarClientConfig.builder()
- .serviceUrl(config.getString(PULSAR_SERVICE_URL));
- setOption(config, PULSAR_AUTH_PLUGIN_CLASS_NAME, config::getString, clientConfigBuilder::authPluginClassName);
- setOption(config, PULSAR_AUTH_PARAMS, config::getString, clientConfigBuilder::authParams);
+ .serviceUrl(config.getString(CLIENT_SERVICE_URL));
+ setOption(config, AUTH_PLUGIN_CLASS, config::getString, clientConfigBuilder::authPluginClassName);
+ setOption(config, AUTH_PARAMS, config::getString, clientConfigBuilder::authParams);
this.clientConfig = clientConfigBuilder.build();
// consumer config
PulsarConsumerConfig.Builder consumerConfigBuilder = PulsarConsumerConfig.builder()
- .subscriptionName(config.getString(PULSAR_SERVICE_URL));
+ .subscriptionName(config.getString(SUBSCRIPTION_NAME));
this.consumerConfig = consumerConfigBuilder.build();
// source properties
setOption(config,
- PULSAR_PARTITION_DISCOVERY_INTERVAL_MS,
+ TOPIC_DISCOVERY_INTERVAL,
30000L,
config::getLong,
v -> this.partitionDiscoveryIntervalMs = v);
setOption(config,
- PULSAR_POLL_TIMEOUT,
+ POLL_TIMEOUT,
100,
config::getInt,
v -> this.pollTimeout = v);
setOption(config,
- PULSAR_POLL_INTERVAL,
+ POLL_INTERVAL,
50L,
config::getLong,
v -> this.pollInterval = v);
setOption(config,
- PULSAR_BATCH_SIZE,
+ POLL_BATCH_SIZE,
500,
config::getInt,
v -> this.batchSize = v);
@@ -159,7 +157,7 @@ public void prepare(Config config) throws PrepareFailException {
}
private void setStartCursor(Config config) {
- StartMode startMode = getEnum(config, PULSAR_CURSOR_START_MODE, StartMode.class, LATEST);
+ StartMode startMode = getEnum(config, CURSOR_STARTUP_MODE, StartMode.class, LATEST);
switch (startMode) {
case EARLIEST:
this.startCursor = StartCursor.earliest();
@@ -169,16 +167,16 @@ private void setStartCursor(Config config) {
break;
case SUBSCRIPTION:
SubscriptionStartCursor.CursorResetStrategy resetStrategy = getEnum(config,
- PULSAR_CURSOR_START_RESET_MODE,
+ CURSOR_RESET_MODE,
SubscriptionStartCursor.CursorResetStrategy.class,
SubscriptionStartCursor.CursorResetStrategy.LATEST);
this.startCursor = StartCursor.subscription(resetStrategy);
break;
case TIMESTAMP:
- if (StringUtils.isBlank(config.getString(PULSAR_CURSOR_START_TIMESTAMP))) {
- throw new IllegalArgumentException(String.format("The '%s' property is required when the '%s' is 'timestamp'.", PULSAR_CURSOR_START_TIMESTAMP, PULSAR_CURSOR_START_MODE));
+ if (StringUtils.isBlank(config.getString(CURSOR_STARTUP_TIMESTAMP))) {
+ throw new IllegalArgumentException(String.format("The '%s' property is required when the '%s' is 'timestamp'.", CURSOR_STARTUP_TIMESTAMP, CURSOR_STARTUP_MODE));
}
- setOption(config, PULSAR_CURSOR_START_TIMESTAMP, config::getLong, timestamp -> this.startCursor = StartCursor.timestamp(timestamp));
+ setOption(config, CURSOR_STARTUP_TIMESTAMP, config::getLong, timestamp -> this.startCursor = StartCursor.timestamp(timestamp));
break;
default:
throw new IllegalArgumentException(String.format("The %s mode is not supported.", startMode));
@@ -186,7 +184,7 @@ private void setStartCursor(Config config) {
}
private void setStopCursor(Config config) {
- SourceProperties.StopMode stopMode = getEnum(config, PULSAR_CURSOR_STOP_MODE, SourceProperties.StopMode.class, NEVER);
+ SourceProperties.StopMode stopMode = getEnum(config, CURSOR_STOP_MODE, SourceProperties.StopMode.class, NEVER);
switch (stopMode) {
case LATEST:
this.stopCursor = StopCursor.latest();
@@ -195,10 +193,10 @@ private void setStopCursor(Config config) {
this.stopCursor = StopCursor.never();
break;
case TIMESTAMP:
- if (StringUtils.isBlank(config.getString(PULSAR_CURSOR_STOP_TIMESTAMP))) {
- throw new IllegalArgumentException(String.format("The '%s' property is required when the '%s' is 'timestamp'.", PULSAR_CURSOR_STOP_TIMESTAMP, PULSAR_CURSOR_STOP_MODE));
+ if (StringUtils.isBlank(config.getString(CURSOR_STOP_TIMESTAMP))) {
+ throw new IllegalArgumentException(String.format("The '%s' property is required when the '%s' is 'timestamp'.", CURSOR_STOP_TIMESTAMP, CURSOR_STOP_MODE));
}
- setOption(config, PULSAR_CURSOR_START_TIMESTAMP, config::getLong, timestamp -> this.stopCursor = StopCursor.timestamp(timestamp));
+ setOption(config, CURSOR_STARTUP_TIMESTAMP, config::getLong, timestamp -> this.stopCursor = StopCursor.timestamp(timestamp));
break;
default:
throw new IllegalArgumentException(String.format("The %s mode is not supported.", stopMode));
@@ -206,19 +204,19 @@ private void setStopCursor(Config config) {
}
private void setPartitionDiscoverer(Config config) {
- String topic = config.getString(PULSAR_TOPIC);
+ String topic = config.getString(TOPIC);
if (StringUtils.isNotBlank(topic)) {
this.partitionDiscoverer = new TopicListDiscoverer(Arrays.asList(StringUtils.split(topic, ",")));
}
- String topicPattern = config.getString(PULSAR_TOPIC_PATTERN);
+ String topicPattern = config.getString(TOPIC_PATTERN);
if (StringUtils.isNotBlank(topicPattern)) {
if (this.partitionDiscoverer != null) {
- throw new IllegalArgumentException(String.format("The properties '%s' and '%s' is exclusive.", PULSAR_TOPIC, PULSAR_TOPIC_PATTERN));
+ throw new IllegalArgumentException(String.format("The properties '%s' and '%s' is exclusive.", TOPIC, TOPIC_PATTERN));
}
this.partitionDiscoverer = new TopicPatternDiscoverer(Pattern.compile(topicPattern));
}
if (this.partitionDiscoverer == null) {
- throw new IllegalArgumentException(String.format("The properties '%s' or '%s' is required.", PULSAR_TOPIC, PULSAR_TOPIC_PATTERN));
+ throw new IllegalArgumentException(String.format("The properties '%s' or '%s' is required.", TOPIC, TOPIC_PATTERN));
}
}
@@ -274,8 +272,4 @@ public SourceSplitEnumerator r
checkpointState.assignedPartitions());
}
- @Override
- public Serializer getEnumeratorStateSerializer() {
- return new DefaultSerializer<>();
- }
}
diff --git a/seatunnel-connectors-v2/pom.xml b/seatunnel-connectors-v2/pom.xml
index be62464f81b..56483f8acc3 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,13 +35,16 @@
connector-clickhouse
connector-console
connector-fake
- connector-hive
connector-http
connector-jdbc
connector-kafka
connector-pulsar
connector-socket
+ connector-hive
+ connector-file
+ connector-hudi
connector-assert
+ connector-kudu
@@ -55,5 +57,16 @@
junit-jupiter-params
+
+
+
+ org.apache.maven.plugins
+ maven-dependency-plugin
+
+ true
+
+
+
+
-
\ No newline at end of file
+
diff --git a/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-assert/src/test/java/org/apache/seatunnel/flink/assertion/AssertExecutorTest.java b/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-assert/src/test/java/org/apache/seatunnel/flink/assertion/AssertExecutorTest.java
index 34c533cfc8b..1ddd866b90c 100644
--- a/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-assert/src/test/java/org/apache/seatunnel/flink/assertion/AssertExecutorTest.java
+++ b/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-assert/src/test/java/org/apache/seatunnel/flink/assertion/AssertExecutorTest.java
@@ -26,7 +26,6 @@
import java.util.List;
-@SuppressWarnings("magicnumber")
public class AssertExecutorTest extends TestCase {
Row row = Row.withNames();
AssertExecutor assertExecutor = new AssertExecutor();
diff --git a/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-assert/src/test/java/org/apache/seatunnel/flink/assertion/rule/AssertRuleParserTest.java b/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-assert/src/test/java/org/apache/seatunnel/flink/assertion/rule/AssertRuleParserTest.java
index fad5ed322c6..1df875e9d30 100644
--- a/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-assert/src/test/java/org/apache/seatunnel/flink/assertion/rule/AssertRuleParserTest.java
+++ b/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-assert/src/test/java/org/apache/seatunnel/flink/assertion/rule/AssertRuleParserTest.java
@@ -25,7 +25,6 @@
import java.util.List;
-@SuppressWarnings("magicnumber")
public class AssertRuleParserTest extends TestCase {
AssertRuleParser parser = new AssertRuleParser();
diff --git a/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-clickhouse/src/main/java/org/apache/seatunnel/flink/clickhouse/pojo/Shard.java b/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-clickhouse/src/main/java/org/apache/seatunnel/flink/clickhouse/pojo/Shard.java
index 20f34be1e82..e2d1394df22 100644
--- a/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-clickhouse/src/main/java/org/apache/seatunnel/flink/clickhouse/pojo/Shard.java
+++ b/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-clickhouse/src/main/java/org/apache/seatunnel/flink/clickhouse/pojo/Shard.java
@@ -114,7 +114,6 @@ public boolean equals(Object o) {
}
@Override
- @SuppressWarnings("magicnumber")
public int hashCode() {
if (hashCode != -1) {
return hashCode;
diff --git a/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-clickhouse/src/main/java/org/apache/seatunnel/flink/clickhouse/pojo/ShardMetadata.java b/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-clickhouse/src/main/java/org/apache/seatunnel/flink/clickhouse/pojo/ShardMetadata.java
index 96e27336462..033c99d9048 100644
--- a/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-clickhouse/src/main/java/org/apache/seatunnel/flink/clickhouse/pojo/ShardMetadata.java
+++ b/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-clickhouse/src/main/java/org/apache/seatunnel/flink/clickhouse/pojo/ShardMetadata.java
@@ -122,7 +122,6 @@ public boolean equals(Object o) {
}
@Override
- @SuppressWarnings("magicnumber")
public int hashCode() {
int result = shardKey.hashCode();
result = 31 * result + shardKeyType.hashCode();
diff --git a/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-doris/src/main/java/org/apache/seatunnel/flink/doris/sink/RespContent.java b/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-doris/src/main/java/org/apache/seatunnel/flink/doris/sink/RespContent.java
index 2f18861172b..67bf3781e2a 100644
--- a/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-doris/src/main/java/org/apache/seatunnel/flink/doris/sink/RespContent.java
+++ b/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-doris/src/main/java/org/apache/seatunnel/flink/doris/sink/RespContent.java
@@ -38,6 +38,9 @@ public class RespContent implements Serializable {
*/
@JsonProperty("Label")
private String label;
+
+ @JsonProperty("TwoPhaseCommit")
+ private String twoPhaseCommit;
/**
* Import complete status.
* "Success": Indicates that the import was successful.
@@ -140,6 +143,14 @@ public void setLabel(String label) {
this.label = label;
}
+ public String getTwoPhaseCommit() {
+ return twoPhaseCommit;
+ }
+
+ public void setTwoPhaseCommit(String twoPhaseCommit) {
+ this.twoPhaseCommit = twoPhaseCommit;
+ }
+
public String getStatus() {
return status;
}
@@ -265,6 +276,7 @@ public String toString() {
return "RespContent{" +
"txnId=" + txnId +
", label='" + label + '\'' +
+ ", twoPhaseCommit='" + twoPhaseCommit + '\'' +
", status='" + status + '\'' +
", existingJobStatus='" + existingJobStatus + '\'' +
", message='" + message + '\'' +
diff --git a/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-fake/src/main/java/org/apache/seatunnel/flink/fake/source/MockSchema.java b/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-fake/src/main/java/org/apache/seatunnel/flink/fake/source/MockSchema.java
index b8f0428d1a8..37f1b33726f 100644
--- a/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-fake/src/main/java/org/apache/seatunnel/flink/fake/source/MockSchema.java
+++ b/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-fake/src/main/java/org/apache/seatunnel/flink/fake/source/MockSchema.java
@@ -53,7 +53,6 @@
import java.io.ByteArrayInputStream;
import java.io.Serializable;
import java.math.BigDecimal;
-import java.math.BigInteger;
import java.nio.charset.StandardCharsets;
import java.sql.Timestamp;
import java.util.ArrayList;
@@ -132,6 +131,7 @@ public TypeInformation> typeInformation() {
dataType = BasicTypeInfo.SHORT_TYPE_INFO;
break;
case "long":
+ case "bigint":
dataType = BasicTypeInfo.LONG_TYPE_INFO;
break;
case "float":
@@ -150,10 +150,6 @@ public TypeInformation> typeInformation() {
case "bigdecimal":
dataType = BasicTypeInfo.BIG_DEC_TYPE_INFO;
break;
- case "bigint":
- case "biginteger":
- dataType = BasicTypeInfo.BIG_INT_TYPE_INFO;
- break;
case "int[]":
dataType = PrimitiveArrayTypeInfo.INT_PRIMITIVE_ARRAY_TYPE_INFO;
break;
@@ -193,11 +189,11 @@ public TypeInformation> typeInformation() {
return dataType;
}
- public Object mockData(){
+ public Object mockData() {
Object mockData;
MockConfig mockConfig = new MockConfig();
resolve(mockConfig);
- switch (this.type.trim().toLowerCase()){
+ switch (this.type.trim().toLowerCase()) {
case "int":
case "integer":
mockData = JMockData.mock(int.class, mockConfig);
@@ -216,6 +212,7 @@ public Object mockData(){
mockData = JMockData.mock(short.class, mockConfig);
break;
case "long":
+ case "bigint":
mockData = JMockData.mock(long.class, mockConfig);
break;
case "float":
@@ -234,10 +231,6 @@ public Object mockData(){
case "bigdecimal":
mockData = JMockData.mock(BigDecimal.class, mockConfig);
break;
- case "bigint":
- case "biginteger":
- mockData = JMockData.mock(BigInteger.class, mockConfig);
- break;
case "int[]":
mockData = JMockData.mock(int[].class, mockConfig);
break;
@@ -426,7 +419,7 @@ public static RowTypeInfo mockRowTypeInfo(List mockDataSchema) {
return new RowTypeInfo(types, fieldNames);
}
- public static Row mockRowData(List mockDataSchema){
+ public static Row mockRowData(List mockDataSchema) {
Object[] fieldByPosition = new Object[mockDataSchema.size()];
for (int index = 0; index < mockDataSchema.size(); index++) {
MockSchema schema = mockDataSchema.get(index);
@@ -470,7 +463,7 @@ public static Row mockRowData(List mockDataSchema){
DEFAULT_MOCK_SCHEMAS.add(ageSchema);
}
- public static List resolveConfig(Config config){
+ public static List resolveConfig(Config config) {
if (config.hasPath(MOCK_DATA_SCHEMA)) {
return config.getConfigList(MOCK_DATA_SCHEMA)
.stream()
diff --git a/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-kafka/src/main/java/org/apache/seatunnel/flink/kafka/source/KafkaTableStream.java b/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-kafka/src/main/java/org/apache/seatunnel/flink/kafka/source/KafkaTableStream.java
index 5038241315d..dbb60c7cf06 100644
--- a/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-kafka/src/main/java/org/apache/seatunnel/flink/kafka/source/KafkaTableStream.java
+++ b/seatunnel-connectors/seatunnel-connectors-flink/seatunnel-connector-flink-kafka/src/main/java/org/apache/seatunnel/flink/kafka/source/KafkaTableStream.java
@@ -111,7 +111,7 @@ public void prepare(FlinkEnvironment env) {
}
String schemaContent = config.getString(SCHEMA);
format = FormatType.from(config.getString(SOURCE_FORMAT).trim().toLowerCase());
- schemaInfo = JsonUtils.parseObject(schemaContent);
+ schemaInfo = JsonUtils.parseArray(schemaContent);
}
@Override
diff --git a/seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-hudi/src/main/scala/org/apache/seatunnel/spark/hudi/source/Hudi.scala b/seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-hudi/src/main/scala/org/apache/seatunnel/spark/hudi/source/Hudi.scala
index 82e55f09a7f..a1d42862cf2 100644
--- a/seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-hudi/src/main/scala/org/apache/seatunnel/spark/hudi/source/Hudi.scala
+++ b/seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-hudi/src/main/scala/org/apache/seatunnel/spark/hudi/source/Hudi.scala
@@ -38,7 +38,7 @@ class Hudi extends SparkBatchSource {
reader.option(e.getKey, String.valueOf(e.getValue.unwrapped()))
}
- reader.load(config.getString(HOODIE_DATASTORE_READ_PATHS))
+ reader.load()
}
override def getPluginName: String = "Hudi"
diff --git a/seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-jdbc/src/main/scala/org/apache/seatunnel/spark/jdbc/source/Jdbc.scala b/seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-jdbc/src/main/scala/org/apache/seatunnel/spark/jdbc/source/Jdbc.scala
index 6648e019c99..6a31beefd66 100644
--- a/seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-jdbc/src/main/scala/org/apache/seatunnel/spark/jdbc/source/Jdbc.scala
+++ b/seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-jdbc/src/main/scala/org/apache/seatunnel/spark/jdbc/source/Jdbc.scala
@@ -18,11 +18,12 @@ package org.apache.seatunnel.spark.jdbc.source
import scala.collection.JavaConversions._
import scala.util.{Failure, Success, Try}
-
import org.apache.seatunnel.common.config.{CheckResult, TypesafeConfigUtils}
import org.apache.seatunnel.common.config.CheckConfigUtil.checkAllExists
import org.apache.seatunnel.spark.SparkEnvironment
import org.apache.seatunnel.spark.batch.SparkBatchSource
+import org.apache.seatunnel.spark.jdbc.source.util.HiveDialect
+import org.apache.spark.sql.jdbc.JdbcDialects
import org.apache.spark.sql.{DataFrameReader, Dataset, Row, SparkSession}
class Jdbc extends SparkBatchSource {
@@ -58,6 +59,10 @@ class Jdbc extends SparkBatchSource {
case Failure(_) => // do nothing
}
+ if (config.getString("url").startsWith("jdbc:hive2")) {
+ JdbcDialects.registerDialect(new HiveDialect)
+ }
+
reader
}
diff --git a/seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-jdbc/src/main/scala/org/apache/seatunnel/spark/jdbc/source/util/HiveDialect.scala b/seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-jdbc/src/main/scala/org/apache/seatunnel/spark/jdbc/source/util/HiveDialect.scala
new file mode 100644
index 00000000000..81eb48bf60d
--- /dev/null
+++ b/seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-jdbc/src/main/scala/org/apache/seatunnel/spark/jdbc/source/util/HiveDialect.scala
@@ -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.spark.jdbc.source.util
+
+import org.apache.spark.sql.jdbc.JdbcDialect
+
+class HiveDialect extends JdbcDialect {
+ override def canHandle(url: String): Boolean = {
+ url.startsWith("jdbc:hive2")
+ }
+
+ override def quoteIdentifier(colName: String): String = {
+ if (colName.contains(".")) {
+ val colName1 = colName.substring(colName.indexOf(".") + 1)
+ s"`$colName1`"
+ } else {
+ s"`$colName`"
+ }
+ }
+}
diff --git a/seatunnel-core/seatunnel-core-base/src/main/java/org/apache/seatunnel/core/base/command/AbstractCommandArgs.java b/seatunnel-core/seatunnel-core-base/src/main/java/org/apache/seatunnel/core/base/command/AbstractCommandArgs.java
index a19fe285f1b..9042b2c140e 100644
--- a/seatunnel-core/seatunnel-core-base/src/main/java/org/apache/seatunnel/core/base/command/AbstractCommandArgs.java
+++ b/seatunnel-core/seatunnel-core-base/src/main/java/org/apache/seatunnel/core/base/command/AbstractCommandArgs.java
@@ -38,7 +38,7 @@ public abstract class AbstractCommandArgs implements CommandArgs {
private List variables = Collections.emptyList();
// todo: use command type enum
- @Parameter(names = {"-t", "--check"},
+ @Parameter(names = {"-ck", "--check"},
description = "check config")
private boolean checkConfig = false;
diff --git a/seatunnel-core/seatunnel-core-flink-sql/src/main/bin/start-seatunnel-sql.sh b/seatunnel-core/seatunnel-core-flink-sql/src/main/bin/start-seatunnel-sql.sh
index 41890ad91a0..6f55aa5baca 100755
--- a/seatunnel-core/seatunnel-core-flink-sql/src/main/bin/start-seatunnel-sql.sh
+++ b/seatunnel-core/seatunnel-core-flink-sql/src/main/bin/start-seatunnel-sql.sh
@@ -17,9 +17,27 @@
#
set -eu
-APP_DIR=$(cd $(dirname ${0})/../;pwd)
+# resolve links - $0 may be a softlink
+PRG="$0"
+
+while [ -h "$PRG" ] ; do
+ # shellcheck disable=SC2006
+ ls=`ls -ld "$PRG"`
+ # shellcheck disable=SC2006
+ link=`expr "$ls" : '.*-> \(.*\)$'`
+ if expr "$link" : '/.*' > /dev/null; then
+ PRG="$link"
+ else
+ # shellcheck disable=SC2006
+ PRG=`dirname "$PRG"`/"$link"
+ fi
+done
+
+PRG_DIR=`dirname "$PRG"`
+APP_DIR=`cd "$PRG_DIR/.." >/dev/null; pwd`
CONF_DIR=${APP_DIR}/config
APP_JAR=${APP_DIR}/lib/seatunnel-core-flink-sql.jar
+APP_MAIN="org.apache.seatunnel.core.sql.FlinkSqlStarter"
if [ -f "${CONF_DIR}/seatunnel-env.sh" ]; then
. "${CONF_DIR}/seatunnel-env.sh"
@@ -33,7 +51,7 @@ else
fi
-CMD=$(java -cp ${APP_JAR} org.apache.seatunnel.core.sql.FlinkSqlStarter ${args}) && EXIT_CODE=$? || EXIT_CODE=$?
+CMD=$(java -cp ${APP_JAR} ${APP_MAIN} ${args}) && EXIT_CODE=$? || EXIT_CODE=$?
if [ ${EXIT_CODE} -eq 234 ]; then
# print usage
echo "${CMD}"
diff --git a/seatunnel-core/seatunnel-core-flink-sql/src/main/java/org/apache/seatunnel/core/sql/FlinkSqlStarter.java b/seatunnel-core/seatunnel-core-flink-sql/src/main/java/org/apache/seatunnel/core/sql/FlinkSqlStarter.java
index e8795ecc760..141a2fd92bd 100644
--- a/seatunnel-core/seatunnel-core-flink-sql/src/main/java/org/apache/seatunnel/core/sql/FlinkSqlStarter.java
+++ b/seatunnel-core/seatunnel-core-flink-sql/src/main/java/org/apache/seatunnel/core/sql/FlinkSqlStarter.java
@@ -39,6 +39,7 @@ public class FlinkSqlStarter implements Starter {
FlinkSqlStarter(String[] args) {
this.flinkCommandArgs = CommandLineUtils.parseCommandArgs(args, FlinkJobType.SQL);
// set the deployment mode, used to get the job jar path.
+ Common.setStarter(true);
Common.setDeployMode(flinkCommandArgs.getDeployMode());
this.appJar = Common.appLibDir().resolve(APP_JAR_NAME).toString();
}
diff --git a/seatunnel-core/seatunnel-core-flink/src/main/bin/start-seatunnel-flink.sh b/seatunnel-core/seatunnel-core-flink/src/main/bin/start-seatunnel-flink.sh
index d135ae7deb4..4e5906841d5 100755
--- a/seatunnel-core/seatunnel-core-flink/src/main/bin/start-seatunnel-flink.sh
+++ b/seatunnel-core/seatunnel-core-flink/src/main/bin/start-seatunnel-flink.sh
@@ -17,9 +17,28 @@
#
set -eu
-APP_DIR=$(cd $(dirname ${0})/../;pwd)
+# resolve links - $0 may be a softlink
+PRG="$0"
+
+while [ -h "$PRG" ] ; do
+ # shellcheck disable=SC2006
+ ls=`ls -ld "$PRG"`
+ # shellcheck disable=SC2006
+ link=`expr "$ls" : '.*-> \(.*\)$'`
+ if expr "$link" : '/.*' > /dev/null; then
+ PRG="$link"
+ else
+ # shellcheck disable=SC2006
+ PRG=`dirname "$PRG"`/"$link"
+ fi
+done
+
+PRG_DIR=`dirname "$PRG"`
+APP_DIR=`cd "$PRG_DIR/.." >/dev/null; pwd`
CONF_DIR=${APP_DIR}/config
APP_JAR=${APP_DIR}/lib/seatunnel-core-flink.jar
+ENV_PARAMETERS_MAIN="org.apache.seatunnel.core.flink.FlinkEnvParameterParser"
+APP_MAIN="org.apache.seatunnel.core.flink.FlinkStarter"
if [ -f "${CONF_DIR}/seatunnel-env.sh" ]; then
. "${CONF_DIR}/seatunnel-env.sh"
@@ -32,7 +51,7 @@ else
args=$@
fi
-ENV_PARAMETERS_OR_ERROR=$(java -cp ${APP_JAR} org.apache.seatunnel.core.flink.FlinkEnvParameterParser ${args}) && EXIT_CODE=$? || EXIT_CODE=$?
+ENV_PARAMETERS_OR_ERROR=$(java -cp ${APP_JAR} ${ENV_PARAMETERS_MAIN} ${args}) && EXIT_CODE=$? || EXIT_CODE=$?
if [ ${EXIT_CODE} -eq 0 ]; then
echo "Export JVM_ARGS: ${ENV_PARAMETERS_OR_ERROR}"
export JVM_ARGS="${ENV_PARAMETERS_OR_ERROR}"
@@ -41,7 +60,7 @@ else
exit ${EXIT_CODE}
fi
-CMD=$(java -cp ${APP_JAR} org.apache.seatunnel.core.flink.FlinkStarter ${args}) && EXIT_CODE=$? || EXIT_CODE=$?
+CMD=$(java -cp ${APP_JAR} ${APP_MAIN} ${args}) && EXIT_CODE=$? || EXIT_CODE=$?
if [ ${EXIT_CODE} -eq 234 ]; then
# print usage
echo "${CMD}"
diff --git a/seatunnel-core/seatunnel-core-flink/src/main/java/org/apache/seatunnel/core/flink/FlinkStarter.java b/seatunnel-core/seatunnel-core-flink/src/main/java/org/apache/seatunnel/core/flink/FlinkStarter.java
index 20ea886632b..cd4a2be7a7a 100644
--- a/seatunnel-core/seatunnel-core-flink/src/main/java/org/apache/seatunnel/core/flink/FlinkStarter.java
+++ b/seatunnel-core/seatunnel-core-flink/src/main/java/org/apache/seatunnel/core/flink/FlinkStarter.java
@@ -47,6 +47,7 @@ public class FlinkStarter implements Starter {
this.flinkCommandArgs = CommandLineUtils.parseCommandArgs(args, FlinkJobType.JAR);
// set the deployment mode, used to get the job jar path.
Common.setDeployMode(flinkCommandArgs.getDeployMode());
+ Common.setStarter(true);
this.appJar = Common.appLibDir().resolve(APP_JAR_NAME).toString();
}
diff --git a/seatunnel-core/seatunnel-core-flink/src/main/java/org/apache/seatunnel/core/flink/config/FlinkExecutionContext.java b/seatunnel-core/seatunnel-core-flink/src/main/java/org/apache/seatunnel/core/flink/config/FlinkExecutionContext.java
index 773feb870b1..2acb8369934 100644
--- a/seatunnel-core/seatunnel-core-flink/src/main/java/org/apache/seatunnel/core/flink/config/FlinkExecutionContext.java
+++ b/seatunnel-core/seatunnel-core-flink/src/main/java/org/apache/seatunnel/core/flink/config/FlinkExecutionContext.java
@@ -33,7 +33,9 @@
import java.net.URL;
import java.util.ArrayList;
+import java.util.HashSet;
import java.util.List;
+import java.util.Set;
import java.util.stream.Collectors;
public class FlinkExecutionContext extends AbstractExecutionContext {
@@ -47,12 +49,12 @@ public FlinkExecutionContext(Config config, EngineType engine) {
this.flinkSourcePluginDiscovery = new FlinkSourcePluginDiscovery();
this.flinkTransformPluginDiscovery = new FlinkTransformPluginDiscovery();
this.flinkSinkPluginDiscovery = new FlinkSinkPluginDiscovery();
- List pluginJars = new ArrayList<>();
+ Set pluginJars = new HashSet<>();
// since we didn't split the transform plugin jars, we just need to register the source/sink plugin jars
pluginJars.addAll(flinkSourcePluginDiscovery.getPluginJarPaths(getPluginIdentifiers(PluginType.SOURCE)));
pluginJars.addAll(flinkSinkPluginDiscovery.getPluginJarPaths(getPluginIdentifiers(PluginType.SINK)));
- this.pluginJars = pluginJars;
- this.getEnvironment().registerPlugin(pluginJars);
+ this.pluginJars = new ArrayList<>(pluginJars);
+ this.getEnvironment().registerPlugin(this.pluginJars);
}
@Override
@@ -63,7 +65,7 @@ public List> getSources() {
return configList.stream()
.map(pluginConfig -> {
PluginIdentifier pluginIdentifier = PluginIdentifier.of(engineType, pluginType, pluginConfig.getString("plugin_name"));
- BaseSource pluginInstance = flinkSourcePluginDiscovery.getPluginInstance(pluginIdentifier);
+ BaseSource pluginInstance = flinkSourcePluginDiscovery.createPluginInstance(pluginIdentifier);
pluginInstance.setConfig(pluginConfig);
return pluginInstance;
}).collect(Collectors.toList());
@@ -77,7 +79,7 @@ public List> getTransforms() {
return configList.stream()
.map(pluginConfig -> {
PluginIdentifier pluginIdentifier = PluginIdentifier.of(engineType, pluginType, pluginConfig.getString("plugin_name"));
- BaseTransform pluginInstance = flinkTransformPluginDiscovery.getPluginInstance(pluginIdentifier);
+ BaseTransform pluginInstance = flinkTransformPluginDiscovery.createPluginInstance(pluginIdentifier);
pluginInstance.setConfig(pluginConfig);
return pluginInstance;
}).collect(Collectors.toList());
@@ -91,7 +93,7 @@ public List> getSinks() {
return configList.stream()
.map(pluginConfig -> {
PluginIdentifier pluginIdentifier = PluginIdentifier.of(engineType, pluginType, pluginConfig.getString("plugin_name"));
- BaseSink pluginInstance = flinkSinkPluginDiscovery.getPluginInstance(pluginIdentifier);
+ BaseSink pluginInstance = flinkSinkPluginDiscovery.createPluginInstance(pluginIdentifier);
pluginInstance.setConfig(pluginConfig);
return pluginInstance;
}).collect(Collectors.toList());
diff --git a/seatunnel-core/seatunnel-core-flink/src/test/java/org/apache/seatunnel/core/flink/args/FlinkCommandArgsTest.java b/seatunnel-core/seatunnel-core-flink/src/test/java/org/apache/seatunnel/core/flink/args/FlinkCommandArgsTest.java
index 56f4f5e6374..a4dba1b27cc 100644
--- a/seatunnel-core/seatunnel-core-flink/src/test/java/org/apache/seatunnel/core/flink/args/FlinkCommandArgsTest.java
+++ b/seatunnel-core/seatunnel-core-flink/src/test/java/org/apache/seatunnel/core/flink/args/FlinkCommandArgsTest.java
@@ -27,7 +27,7 @@ public class FlinkCommandArgsTest {
@Test
public void testParseFlinkArgs() {
- String[] args = {"-c", "app.conf", "-t", "-i", "city=shenyang", "-i", "date=20200202"};
+ String[] args = {"-c", "app.conf", "-ck", "-i", "city=shenyang", "-i", "date=20200202"};
FlinkCommandArgs flinkArgs = new FlinkCommandArgs();
JCommander.newBuilder()
.addObject(flinkArgs)
diff --git a/seatunnel-core/seatunnel-core-flink/src/test/java/org/apache/seatunnel/core/flink/utils/CommandLineUtilsTest.java b/seatunnel-core/seatunnel-core-flink/src/test/java/org/apache/seatunnel/core/flink/utils/CommandLineUtilsTest.java
index fe206c0f595..2befcdbe877 100644
--- a/seatunnel-core/seatunnel-core-flink/src/test/java/org/apache/seatunnel/core/flink/utils/CommandLineUtilsTest.java
+++ b/seatunnel-core/seatunnel-core-flink/src/test/java/org/apache/seatunnel/core/flink/utils/CommandLineUtilsTest.java
@@ -34,14 +34,14 @@ public class CommandLineUtilsTest {
@Test
public void testParseCommandArgs() {
- String[] args = {"--detached", "-c", "app.conf", "-t", "-i", "city=shenyang", "-i", "date=20200202",
+ String[] args = {"--detached", "-c", "app.conf", "-ck", "-i", "city=shenyang", "-i", "date=20200202",
"-r", "run-application", "--unkown", "unkown-command"};
FlinkCommandArgs flinkCommandArgs = CommandLineUtils.parseCommandArgs(args, FlinkJobType.JAR);
Assert.assertEquals(flinkCommandArgs.getFlinkParams(), Arrays.asList("--detached", "--unkown", "unkown-command"));
Assert.assertEquals(flinkCommandArgs.getRunMode(), FlinkRunMode.APPLICATION_RUN);
Assert.assertEquals(flinkCommandArgs.getVariables(), Arrays.asList("city=shenyang", "date=20200202"));
- String[] args1 = {"--detached", "-c", "app.conf", "-t", "-i", "city=shenyang", "-i", "date=20200202",
+ String[] args1 = {"--detached", "-c", "app.conf", "-ck", "-i", "city=shenyang", "-i", "date=20200202",
"-r", "run-application", "--unkown", "unkown-command"};
flinkCommandArgs = CommandLineUtils.parseCommandArgs(args1, FlinkJobType.SQL);
Assert.assertEquals(flinkCommandArgs.getFlinkParams(), Arrays.asList("--detached", "--unkown", "unkown-command"));
@@ -51,7 +51,7 @@ public void testParseCommandArgs() {
@Test
public void testBuildFlinkJarCommand() throws FileNotFoundException {
- String[] args = {"--detached", "-c", APP_CONF_PATH, "-t", "-i", "city=shenyang", "-i", "date=20200202",
+ String[] args = {"--detached", "-c", APP_CONF_PATH, "-ck", "-i", "city=shenyang", "-i", "date=20200202",
"-r", "run-application", "--unkown", "unkown-command"};
FlinkCommandArgs flinkCommandArgs = CommandLineUtils.parseCommandArgs(args, FlinkJobType.JAR);
List commands = CommandLineUtils.buildFlinkCommand(flinkCommandArgs, "CLASS_NAME", "/path/to/jar");
@@ -65,14 +65,14 @@ public void testBuildFlinkJarCommand() throws FileNotFoundException {
Arrays.asList("${FLINK_HOME}/bin/flink", "run-application", "--detached", "--unkown", "unkown-command", "-c",
"CLASS_NAME", "/path/to/jar", "--config", APP_CONF_PATH, "--check", "-Dcity=shenyang", "-Ddate=20200202"));
- String[] args1 = {"--detached", "-c", "app.conf", "-t", "-i", "city=shenyang", "-i", "date=20200202",
+ String[] args1 = {"--detached", "-c", "app.conf", "-ck", "-i", "city=shenyang", "-i", "date=20200202",
"-r", "run-application", "--unkown", "unkown-command"};
}
@Test
public void testBuildFlinkSQLCommand() throws FileNotFoundException{
- String[] args = {"--detached", "-c", SQL_CONF_PATH, "-t", "-i", "city=shenyang", "-i", "date=20200202",
+ String[] args = {"--detached", "-c", SQL_CONF_PATH, "-ck", "-i", "city=shenyang", "-i", "date=20200202",
"-r", "run-application", "--unkown", "unkown-command"};
FlinkCommandArgs flinkCommandArgs = CommandLineUtils.parseCommandArgs(args, FlinkJobType.SQL);
List commands = CommandLineUtils.buildFlinkCommand(flinkCommandArgs, "CLASS_NAME", "/path/to/jar");
diff --git a/seatunnel-core/seatunnel-core-spark/pom.xml b/seatunnel-core/seatunnel-core-spark/pom.xml
index 968b99f030d..5c1264b1850 100644
--- a/seatunnel-core/seatunnel-core-spark/pom.xml
+++ b/seatunnel-core/seatunnel-core-spark/pom.xml
@@ -107,6 +107,12 @@
${project.version}
+
+ org.apache.seatunnel
+ seatunnel-transform-spark-nulltf
+ ${project.version}
+
+
org.apache.seatunnel
seatunnel-transform-spark-null-rate
diff --git a/seatunnel-core/seatunnel-core-spark/src/main/bin/start-seatunnel-spark.sh b/seatunnel-core/seatunnel-core-spark/src/main/bin/start-seatunnel-spark.sh
index 35ea18a956c..4fa08ef2dcd 100755
--- a/seatunnel-core/seatunnel-core-spark/src/main/bin/start-seatunnel-spark.sh
+++ b/seatunnel-core/seatunnel-core-spark/src/main/bin/start-seatunnel-spark.sh
@@ -16,9 +16,27 @@
# limitations under the License.
#
set -eu
-APP_DIR=$(cd $(dirname ${0})/../;pwd)
+# resolve links - $0 may be a softlink
+PRG="$0"
+
+while [ -h "$PRG" ] ; do
+ # shellcheck disable=SC2006
+ ls=`ls -ld "$PRG"`
+ # shellcheck disable=SC2006
+ link=`expr "$ls" : '.*-> \(.*\)$'`
+ if expr "$link" : '/.*' > /dev/null; then
+ PRG="$link"
+ else
+ # shellcheck disable=SC2006
+ PRG=`dirname "$PRG"`/"$link"
+ fi
+done
+
+PRG_DIR=`dirname "$PRG"`
+APP_DIR=`cd "$PRG_DIR/.." >/dev/null; pwd`
CONF_DIR=${APP_DIR}/config
APP_JAR=${APP_DIR}/lib/seatunnel-core-spark.jar
+APP_MAIN="org.apache.seatunnel.core.spark.SparkStarter"
if [ -f "${CONF_DIR}/seatunnel-env.sh" ]; then
. "${CONF_DIR}/seatunnel-env.sh"
@@ -31,7 +49,7 @@ else
args=$@
fi
-CMD=$(java -cp ${APP_JAR} org.apache.seatunnel.core.spark.SparkStarter ${args}) && EXIT_CODE=$? || EXIT_CODE=$?
+CMD=$(java -cp ${APP_JAR} ${APP_MAIN} ${args}) && EXIT_CODE=$? || EXIT_CODE=$?
if [ ${EXIT_CODE} -eq 234 ]; then
# print usage
echo "${CMD}"
diff --git a/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/SparkStarter.java b/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/SparkStarter.java
index 0fd70b629df..7eca89b4b36 100644
--- a/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/SparkStarter.java
+++ b/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/SparkStarter.java
@@ -50,9 +50,11 @@
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
+import java.util.Set;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@@ -148,8 +150,8 @@ private static SparkCommandArgs parseCommandArgs(String[] args) {
public List buildCommands() throws IOException {
setSparkConf();
Common.setDeployMode(commandArgs.getDeployMode());
+ Common.setStarter(true);
this.jars.addAll(getPluginsJarDependencies());
- this.jars.addAll(listJars(Common.appLibDir()));
this.jars.addAll(getConnectorJarDependencies());
this.appName = this.sparkConf.getOrDefault("spark.app.name", Constants.LOGO);
return buildFinal();
@@ -213,7 +215,7 @@ private List getConnectorJarDependencies() {
return Collections.emptyList();
}
Config config = new ConfigBuilder(Paths.get(commandArgs.getConfigFile())).getConfig();
- List pluginJars = new ArrayList<>();
+ Set pluginJars = new HashSet<>();
SparkSourcePluginDiscovery sparkSourcePluginDiscovery = new SparkSourcePluginDiscovery();
SparkSinkPluginDiscovery sparkSinkPluginDiscovery = new SparkSinkPluginDiscovery();
pluginJars.addAll(sparkSourcePluginDiscovery.getPluginJarPaths(getPluginIdentifiers(config, PluginType.SOURCE)));
@@ -221,18 +223,6 @@ private List getConnectorJarDependencies() {
return pluginJars.stream().map(url -> new File(url.getPath()).toPath()).collect(Collectors.toList());
}
- /**
- * list jars in given directory
- */
- private List listJars(Path dir) throws IOException {
- try (Stream stream = Files.list(dir)) {
- return stream
- .filter(it -> !Files.isDirectory(it))
- .filter(it -> it.getFileName().endsWith("jar"))
- .collect(Collectors.toList());
- }
- }
-
/**
* build final spark-submit commands
*/
@@ -405,10 +395,9 @@ private ClusterModeSparkStarter(String[] args, SparkCommandArgs commandArgs) {
@Override
public List buildCommands() throws IOException {
Common.setDeployMode(commandArgs.getDeployMode());
+ Common.setStarter(true);
Path pluginTarball = Common.pluginTarball();
- if (Files.notExists(pluginTarball)) {
- CompressionUtils.tarGzip(Common.pluginRootDir(), pluginTarball);
- }
+ CompressionUtils.tarGzip(Common.pluginRootDir(), pluginTarball);
this.files.add(pluginTarball);
this.files.add(Paths.get(commandArgs.getConfigFile()));
return super.buildCommands();
diff --git a/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/config/SparkExecutionContext.java b/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/config/SparkExecutionContext.java
index 7effd19b349..07854b83cf5 100644
--- a/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/config/SparkExecutionContext.java
+++ b/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/config/SparkExecutionContext.java
@@ -33,7 +33,9 @@
import java.net.URL;
import java.util.ArrayList;
+import java.util.HashSet;
import java.util.List;
+import java.util.Set;
import java.util.stream.Collectors;
public class SparkExecutionContext extends AbstractExecutionContext