This is an automated email from the ASF dual-hosted git repository.
kirs pushed a commit to branch dev
in repository https://gitbox.apache.org/repos/asf/incubator-seatunnel.git
The following commit(s) were added to refs/heads/dev by this push:
new 4fb6f2a21 [Feature][Connector-V2] Add hdfs file source connector
(#2420)
4fb6f2a21 is described below
commit 4fb6f2a216df0c8b13ce410ecf967b33ae0a3df1
Author: TyrantLucifer <[email protected]>
AuthorDate: Tue Aug 16 10:06:30 2022 +0800
[Feature][Connector-V2] Add hdfs file source connector (#2420)
* [Feature][Connector-V2] Setting hadoop dependency scope from compile to
provided in connector-file-hadoop
---
docs/en/connector-v2/source/HdfsFile.md | 41 ++++++++++++++
plugin-mapping.properties | 1 +
.../connector-file/connector-file-hadoop/pom.xml | 1 -
.../seatunnel/file/hdfs/source/HdfsFileSource.java | 66 ++++++++++++++++++++++
.../file/hdfs/source/config/HdfsSourceConfig.java | 26 +++++++++
5 files changed, 134 insertions(+), 1 deletion(-)
diff --git a/docs/en/connector-v2/source/HdfsFile.md
b/docs/en/connector-v2/source/HdfsFile.md
new file mode 100644
index 000000000..6f8d4df83
--- /dev/null
+++ b/docs/en/connector-v2/source/HdfsFile.md
@@ -0,0 +1,41 @@
+# HdfsFile
+
+> Hdfs file source connector
+
+## Description
+
+Read data from hdfs file system.
+
+## Options
+
+| name | type | required | default value |
+|--------------| ------ |----------|---------------|
+| path | string | yes | - |
+| type | string | yes | - |
+| fs.defaultFS | string | yes | - |
+
+### path [string]
+
+The source file path.
+
+### type [string]
+
+File type, supported as the following file types:
+
+`text` `csv` `parquet` `orc` `json`
+
+### fs.defaultFS [string]
+
+Hdfs cluster address.
+
+## Example
+
+```hcon
+
+HdfsFile {
+ path = "/apps/hive/demo/student"
+ type = "parquet"
+ fs.defaultFS = "hdfs://namenode001"
+}
+
+```
\ No newline at end of file
diff --git a/plugin-mapping.properties b/plugin-mapping.properties
index 65a3991e0..6f0fffb56 100644
--- a/plugin-mapping.properties
+++ b/plugin-mapping.properties
@@ -107,6 +107,7 @@ seatunnel.sink.Jdbc = connector-jdbc
seatunnel.source.Kudu = connector-kudu
seatunnel.sink.Kudu = connector-kudu
seatunnel.sink.Email = connector-email
+seatunnel.source.HdfsFile = connector-file-hadoop
seatunnel.sink.HdfsFile = connector-file-hadoop
seatunnel.source.LocalFile = connector-file-local
seatunnel.sink.LocalFile = connector-file-local
diff --git
a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/pom.xml
b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/pom.xml
index 54ad443e4..788265a79 100644
--- a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/pom.xml
+++ b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/pom.xml
@@ -38,7 +38,6 @@
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-shaded-hadoop-2</artifactId>
- <version>${flink-shaded-hadoop-2.version}</version>
<scope>provided</scope>
</dependency>
</dependencies>
diff --git
a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/source/HdfsFileSource.java
b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/source/HdfsFileSource.java
new file mode 100644
index 000000000..44cddd879
--- /dev/null
+++
b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/source/HdfsFileSource.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.file.hdfs.source;
+
+import org.apache.seatunnel.api.common.PrepareFailException;
+import org.apache.seatunnel.api.source.SeaTunnelSource;
+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.file.config.FileSystemType;
+import org.apache.seatunnel.connectors.seatunnel.file.config.HadoopConf;
+import
org.apache.seatunnel.connectors.seatunnel.file.exception.FilePluginException;
+import
org.apache.seatunnel.connectors.seatunnel.file.hdfs.source.config.HdfsSourceConfig;
+import org.apache.seatunnel.connectors.seatunnel.file.source.BaseFileSource;
+import
org.apache.seatunnel.connectors.seatunnel.file.source.reader.ReadStrategyFactory;
+
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+
+import com.google.auto.service.AutoService;
+
+import java.io.IOException;
+
+@AutoService(SeaTunnelSource.class)
+public class HdfsFileSource extends BaseFileSource {
+
+ @Override
+ public String getPluginName() {
+ return FileSystemType.HDFS.getFileSystemPluginName();
+ }
+
+ @Override
+ public void prepare(Config pluginConfig) throws PrepareFailException {
+ CheckResult result = CheckConfigUtil.checkAllExists(pluginConfig,
HdfsSourceConfig.FILE_PATH, HdfsSourceConfig.FILE_TYPE,
HdfsSourceConfig.DEFAULT_FS);
+ if (!result.isSuccess()) {
+ throw new PrepareFailException(getPluginName(), PluginType.SOURCE,
result.getMsg());
+ }
+ readStrategy =
ReadStrategyFactory.of(pluginConfig.getString(HdfsSourceConfig.FILE_TYPE));
+ String path = pluginConfig.getString(HdfsSourceConfig.FILE_PATH);
+ hadoopConf = new
HadoopConf(pluginConfig.getString(HdfsSourceConfig.DEFAULT_FS));
+ try {
+ filePaths = readStrategy.getFileNamesByPath(hadoopConf, path);
+ } catch (IOException e) {
+ throw new PrepareFailException(getPluginName(), PluginType.SOURCE,
"Check file path fail.");
+ }
+ try {
+ rowType = readStrategy.getSeaTunnelRowTypeInfo(hadoopConf,
filePaths.get(0));
+ } catch (FilePluginException e) {
+ throw new PrepareFailException(getPluginName(), PluginType.SOURCE,
"Read file schema error.", e);
+ }
+ }
+}
diff --git
a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/source/config/HdfsSourceConfig.java
b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/source/config/HdfsSourceConfig.java
new file mode 100644
index 000000000..07422edc4
--- /dev/null
+++
b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/source/config/HdfsSourceConfig.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.hdfs.source.config;
+
+import static
org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY;
+
+import org.apache.seatunnel.connectors.seatunnel.file.config.BaseSourceConfig;
+
+public class HdfsSourceConfig extends BaseSourceConfig {
+ public static final String DEFAULT_FS = FS_DEFAULT_NAME_KEY;
+}