You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@seatunnel.apache.org by ki...@apache.org on 2022/07/04 09:22:20 UTC
[incubator-seatunnel] branch dev updated: [Connector-V2]Hive Source (#2123)
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 ffcf3f59e [Connector-V2]Hive Source (#2123)
ffcf3f59e is described below
commit ffcf3f59e2bcbc366e018c8a9b4a324067ab62fa
Author: Kirs <ki...@apache.org>
AuthorDate: Mon Jul 4 17:22:14 2022 +0800
[Connector-V2]Hive Source (#2123)
* [Connector-V2]Hive Source
* Unified dep version
* Unified dep version
* Add License and Notice
---
plugin-mapping.properties | 1 +
pom.xml | 181 +++++++++++++-
.../seatunnel/api/common/PrepareFailException.java | 5 +
seatunnel-connectors-v2/connector-hive/pom.xml | 25 ++
.../seatunnel/hive/config/SourceConfig.java | 18 +-
.../hive/exception/HivePluginException.java | 18 +-
.../seatunnel/hive/sink/HiveSinkConfig.java | 2 +-
.../seatunnel/hive/source/HadoopConf.java | 21 +-
.../seatunnel/hive/source/HiveSource.java | 129 ++++++++++
.../seatunnel/hive/source/HiveSourceReader.java | 98 ++++++++
.../seatunnel/hive/source/HiveSourceSplit.java | 23 +-
.../hive/source/HiveSourceSplitEnumerator.java | 126 ++++++++++
.../seatunnel/hive/source/HiveSourceState.java | 22 +-
.../file/reader/format/AbstractReadStrategy.java | 82 +++++++
.../source/file/reader/format/OrcReadStrategy.java | 123 ++++++++++
.../source/file/reader/format/ReadStrategy.java | 42 ++++
.../file/reader/format/TextReadStrategy.java | 56 +++++
.../pom.xml | 37 +--
.../seatunnel-hive-shade/pom.xml | 69 ++++++
seatunnel-dist/release-docs/LICENSE | 89 +++++--
seatunnel-dist/release-docs/NOTICE | 140 +++++++++++
.../release-docs/licenses/LICENSE-ant.txt | 48 ++++
.../release-docs/licenses/LICENSE-ivy.txt | 259 +++++++++++++++++++++
.../licenses/LICENSE-parquet-hadoop-bundle.txt | 248 ++++++++++++++++++++
.../release-docs/licenses/LICENSE-st.txt | 26 +++
tools/dependencies/known-dependencies.txt | 74 ++++--
26 files changed, 1840 insertions(+), 122 deletions(-)
diff --git a/plugin-mapping.properties b/plugin-mapping.properties
index da3890a70..bd38586a9 100644
--- a/plugin-mapping.properties
+++ b/plugin-mapping.properties
@@ -96,6 +96,7 @@ seatunnel.sink.Kafka = connector-kafka
seatunnel.source.Http = connector-http
seatunnel.source.Socket = connector-socket
seatunnel.sink.Hive = connector-hive
+seatunnel.source.Hive = connector-hive
seatunnel.source.Clickhouse = connector-clickhouse
seatunnel.sink.Clickhouse = connector-clickhouse
seatunnel.sink.ClickhouseFile = connector-clickhouse
diff --git a/pom.xml b/pom.xml
index 23e15e3c4..9339d5f40 100644
--- a/pom.xml
+++ b/pom.xml
@@ -77,6 +77,7 @@
you need to open this annotation and change the dependency of config-shade to project.
<module>seatunnel-config</module>
-->
+ <module>seatunnel-dependency-shade</module>
<module>seatunnel-common</module>
<module>seatunnel-apis</module>
<module>seatunnel-core</module>
@@ -108,6 +109,17 @@
<iceberg.version>0.13.1</iceberg.version>
<flink.version>1.13.6</flink.version>
<hudi.version>0.10.0</hudi.version>
+ <orc.version>1.5.6</orc.version>
+ <hive.exec.version>2.3.9</hive.exec.version>
+ <commons.logging.version>1.2</commons.logging.version>
+ <commons.beanutils.version>1.9.3</commons.beanutils.version>
+ <commons.configuration.version>1.7</commons.configuration.version>
+ <commons.digester.version>1.8.1</commons.digester.version>
+ <curator.version>4.3.0</curator.version>
+ <codehaus.jackson.version>1.9.13</codehaus.jackson.version>
+ <janino.version>3.1.6</janino.version>
+ <jersey.version>1.19</jersey.version>
+ <javax.servlet.jap.version>2.1</javax.servlet.jap.version>
<hadoop.binary.version>2.7</hadoop.binary.version>
<jackson.version>2.12.6</jackson.version>
<lombok.version>1.18.0</lombok.version>
@@ -427,7 +439,7 @@
<artifactId>spark-streaming-kafka-0-10_${scala.binary.version}</artifactId>
<version>${spark.version}</version>
</dependency>
-
+
<dependency>
<groupId>org.projectlombok</groupId>
<artifactId>lombok</artifactId>
@@ -663,6 +675,171 @@
<version>${snappy-java.version}</version>
</dependency>
+ <dependency>
+ <groupId>org.apache.orc</groupId>
+ <artifactId>orc-core</artifactId>
+ <version>${orc.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>javax.servlet</groupId>
+ <artifactId>servlet-api</artifactId>
+ </exclusion>
+
+ <exclusion>
+ <groupId>org.apache.logging.log4j</groupId>
+ <artifactId>*</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>com.fasterxml.jackson.core</groupId>
+ <artifactId>*</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.apapche.hadoop</groupId>
+ <artifactId>*</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.apache.curator</groupId>
+ <artifactId>*</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.codehaus.jackson</groupId>
+ <artifactId>jackson-core-asl</artifactId>
+ <version>${codehaus.jackson.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.codehaus.jackson</groupId>
+ <artifactId>jackson-xc</artifactId>
+ <version>${codehaus.jackson.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.codehaus.jackson</groupId>
+ <artifactId>jackson-mapper-</artifactId>
+ <version>${codehaus.jackson.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.codehaus.jackson</groupId>
+ <artifactId>jackson-jaxrs</artifactId>
+ <version>${codehaus.jackson.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.codehaus.janino</groupId>
+ <artifactId>janino</artifactId>
+ <version>${janino.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>com.sun.jersey</groupId>
+ <artifactId>jersey-json</artifactId>
+ <version>${jersey.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hive</groupId>
+ <artifactId>hive-exec</artifactId>
+ <version>${hive.exec.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.pentaho</groupId>
+ <artifactId>pentaho-aggdesigner-algorithm</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>javax.servlet</groupId>
+ <artifactId>servlet-api</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.apache.logging.log4j</groupId>
+ <artifactId>log4j-1.2-api</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.apache.logging.log4j</groupId>
+ <artifactId>log4j-web</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>com.fasterxml.jackson.core</groupId>
+ <artifactId>*</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.apapche.hadoop</groupId>
+ <artifactId>*</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>com.github.joshelser</groupId>
+ <artifactId>dropwizard-metrics-hadoop-metrics2-reporter</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.apache.logging.log4j</groupId>
+ <artifactId>*</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.apache.zookeeper</groupId>
+ <artifactId>zookeeper</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-server-resourcemanager</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-hdfs</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>commons-logging</groupId>
+ <artifactId>commons-logging</artifactId>
+ <version>${commons.logging.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>commons-beanutils</groupId>
+ <artifactId>commons-beanutils</artifactId>
+ <version>${commons.beanutils.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>commons-configuration</groupId>
+ <artifactId>commons-configuration</artifactId>
+ <version>${commons.configuration.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>commons-digester</groupId>
+ <artifactId>commons-digester</artifactId>
+ <version>${commons.digester.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.curator</groupId>
+ <artifactId>curator-client</artifactId>
+ <version>${curator.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.curator</groupId>
+ <artifactId>curator-framework</artifactId>
+ <version>${curator.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.curator</groupId>
+ <artifactId>curator-recipes</artifactId>
+ <version>${curator.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.curator</groupId>
+ <artifactId>curator-recipes</artifactId>
+ <version>${curator.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>com.sun.jersey</groupId>
+ <artifactId>jersey-core</artifactId>
+ <version>${jersey.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>com.sun.jersey</groupId>
+ <artifactId>jersey-server</artifactId>
+ <version>${jersey.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>javax.servlet.jsp</groupId>
+ <artifactId>jsp-api</artifactId>
+ <version>${javax.servlet.jap.version}</version>
+ </dependency>
+
</dependencies>
</dependencyManagement>
@@ -907,7 +1084,7 @@
</execution>
</executions>
</plugin>
-
+
<!-- checkstyle (End) -->
<plugin>
diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/PrepareFailException.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/PrepareFailException.java
index bb43d07ff..7cb75b98f 100644
--- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/PrepareFailException.java
+++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/PrepareFailException.java
@@ -30,4 +30,9 @@ public class PrepareFailException extends RuntimeException {
super(String.format("PluginName: %s, PluginType: %s, Message: %s", pluginName, type.getType(),
message));
}
+
+ public PrepareFailException(String pluginName, PluginType type, String message, Throwable cause) {
+ super(String.format("PluginName: %s, PluginType: %s, Message: %s", pluginName, type.getType(),
+ message), cause);
+ }
}
diff --git a/seatunnel-connectors-v2/connector-hive/pom.xml b/seatunnel-connectors-v2/connector-hive/pom.xml
index eb2ffa797..db75d801e 100644
--- a/seatunnel-connectors-v2/connector-hive/pom.xml
+++ b/seatunnel-connectors-v2/connector-hive/pom.xml
@@ -30,6 +30,12 @@
<artifactId>connector-hive</artifactId>
<dependencies>
+ <dependency>
+ <groupId>org.apache.seatunnel</groupId>
+ <artifactId>seatunnel-hive-shade</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+
<dependency>
<groupId>org.apache.seatunnel</groupId>
<artifactId>seatunnel-api</artifactId>
@@ -48,6 +54,25 @@
<artifactId>commons-lang3</artifactId>
</dependency>
+ <dependency>
+ <groupId>org.apache.orc</groupId>
+ <artifactId>orc-core</artifactId>
+ <exclusions>
+ <exclusion>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-common</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-hdfs</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.commons</groupId>
+ <artifactId>commons-collections4</artifactId>
+ </dependency>
+
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/PrepareFailException.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/SourceConfig.java
similarity index 59%
copy from seatunnel-api/src/main/java/org/apache/seatunnel/api/common/PrepareFailException.java
copy to seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/SourceConfig.java
index bb43d07ff..705440316 100644
--- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/PrepareFailException.java
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/config/SourceConfig.java
@@ -15,19 +15,15 @@
* limitations under the License.
*/
-package org.apache.seatunnel.api.common;
+package org.apache.seatunnel.connectors.seatunnel.hive.config;
-import org.apache.seatunnel.common.constants.PluginType;
+import static org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY;
-import org.apache.seatunnel.shade.com.typesafe.config.Config;
+public class SourceConfig {
-/**
- * This exception will throw when {@link SeaTunnelPluginLifeCycle#prepare(Config)} failed.
- */
-public class PrepareFailException extends RuntimeException {
+ public static final String FILE_TYPE = "file.type";
+
+ public static final String DEFAULT_FS = FS_DEFAULT_NAME_KEY;
- public PrepareFailException(String pluginName, PluginType type, String message) {
- super(String.format("PluginName: %s, PluginType: %s, Message: %s", pluginName, type.getType(),
- message));
- }
+ public static final String FILE_PATH = "file.path";
}
diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/PrepareFailException.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/exception/HivePluginException.java
similarity index 60%
copy from seatunnel-api/src/main/java/org/apache/seatunnel/api/common/PrepareFailException.java
copy to seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/exception/HivePluginException.java
index bb43d07ff..8e1c22548 100644
--- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/PrepareFailException.java
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/exception/HivePluginException.java
@@ -15,19 +15,15 @@
* limitations under the License.
*/
-package org.apache.seatunnel.api.common;
+package org.apache.seatunnel.connectors.seatunnel.hive.exception;
-import org.apache.seatunnel.common.constants.PluginType;
+public class HivePluginException extends Exception{
-import org.apache.seatunnel.shade.com.typesafe.config.Config;
-
-/**
- * This exception will throw when {@link SeaTunnelPluginLifeCycle#prepare(Config)} failed.
- */
-public class PrepareFailException extends RuntimeException {
+ public HivePluginException(String message) {
+ super(message);
+ }
- public PrepareFailException(String pluginName, PluginType type, String message) {
- super(String.format("PluginName: %s, PluginType: %s, Message: %s", pluginName, type.getType(),
- 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/HiveSinkConfig.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/sink/HiveSinkConfig.java
index 2e08862f7..e4495121b 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
@@ -23,7 +23,7 @@ import org.apache.seatunnel.shade.com.typesafe.config.Config;
import lombok.Data;
import lombok.NonNull;
-import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.collections4.CollectionUtils;
import org.apache.commons.lang3.StringUtils;
import java.util.List;
diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/PrepareFailException.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HadoopConf.java
similarity index 60%
copy from seatunnel-api/src/main/java/org/apache/seatunnel/api/common/PrepareFailException.java
copy to seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HadoopConf.java
index bb43d07ff..31dddf7b4 100644
--- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/PrepareFailException.java
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HadoopConf.java
@@ -15,19 +15,20 @@
* limitations under the License.
*/
-package org.apache.seatunnel.api.common;
+package org.apache.seatunnel.connectors.seatunnel.hive.source;
-import org.apache.seatunnel.common.constants.PluginType;
+import lombok.Data;
-import org.apache.seatunnel.shade.com.typesafe.config.Config;
+import java.io.Serializable;
-/**
- * This exception will throw when {@link SeaTunnelPluginLifeCycle#prepare(Config)} failed.
- */
-public class PrepareFailException extends RuntimeException {
+@Data
+public class HadoopConf implements Serializable {
+
+ private String hdfsNameKey;
+
+ private String fsHdfsImpl = "org.apache.hadoop.hdfs.DistributedFileSystem";
- public PrepareFailException(String pluginName, PluginType type, String message) {
- super(String.format("PluginName: %s, PluginType: %s, Message: %s", pluginName, type.getType(),
- message));
+ 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 000000000..769c1d768
--- /dev/null
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSource.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.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.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.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<SeaTunnelRow, HiveSourceSplit, HiveSourceState> {
+
+ private SeaTunnelContext seaTunnelContext;
+
+ private SeaTunnelRowType typeInfo;
+
+ private ReadStrategy readStrategy;
+
+ private HadoopConf hadoopConf;
+
+ private List<String> 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<SeaTunnelRow> getProducedType() {
+ return this.typeInfo;
+ }
+
+ @Override
+ public SourceReader<SeaTunnelRow, HiveSourceSplit> createReader(SourceReader.Context readerContext) throws Exception {
+ return new HiveSourceReader(this.readStrategy, this.hadoopConf, readerContext);
+ }
+
+ @Override
+ public Boundedness getBoundedness() {
+ return Boundedness.BOUNDED;
+ }
+
+ @Override
+ public SourceSplitEnumerator<HiveSourceSplit, HiveSourceState> createEnumerator(SourceSplitEnumerator.Context<HiveSourceSplit> enumeratorContext) throws Exception {
+ return new HiveSourceSplitEnumerator(enumeratorContext, filesPath);
+ }
+
+ @Override
+ public SourceSplitEnumerator<HiveSourceSplit, HiveSourceState> restoreEnumerator(SourceSplitEnumerator.Context<HiveSourceSplit> enumeratorContext, HiveSourceState checkpointState) throws Exception {
+ return new HiveSourceSplitEnumerator(enumeratorContext, filesPath, checkpointState);
+ }
+
+ @Override
+ public Serializer<HiveSourceState> getEnumeratorStateSerializer() {
+ return new DefaultSerializer<>();
+ }
+}
diff --git a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/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 000000000..570f48e35
--- /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<SeaTunnelRow, HiveSourceSplit> {
+
+ private static final long THREAD_WAIT_TIME = 500L;
+
+ private ReadStrategy readStrategy;
+
+ private HadoopConf hadoopConf;
+
+ private Set<HiveSourceSplit> 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<SeaTunnelRow> 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<HiveSourceSplit> snapshotState(long checkpointId) {
+ return new ArrayList<>(sourceSplits);
+ }
+
+ @Override
+ public void addSplits(List<HiveSourceSplit> splits) {
+ sourceSplits.addAll(splits);
+ }
+
+ @Override
+ public void handleNoMoreSplits() {
+
+ }
+
+ @Override
+ public void notifyCheckpointComplete(long checkpointId) {
+
+ }
+}
diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/PrepareFailException.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceSplit.java
similarity index 60%
copy from seatunnel-api/src/main/java/org/apache/seatunnel/api/common/PrepareFailException.java
copy to seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceSplit.java
index bb43d07ff..44e062e54 100644
--- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/PrepareFailException.java
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceSplit.java
@@ -15,19 +15,22 @@
* limitations under the License.
*/
-package org.apache.seatunnel.api.common;
+package org.apache.seatunnel.connectors.seatunnel.hive.source;
-import org.apache.seatunnel.common.constants.PluginType;
+import org.apache.seatunnel.api.source.SourceSplit;
-import org.apache.seatunnel.shade.com.typesafe.config.Config;
+public class HiveSourceSplit implements SourceSplit {
-/**
- * This exception will throw when {@link SeaTunnelPluginLifeCycle#prepare(Config)} failed.
- */
-public class PrepareFailException extends RuntimeException {
+ private static final long serialVersionUID = -1L;
+
+ private String splitId;
+
+ public HiveSourceSplit(String splitId) {
+ this.splitId = splitId;
+ }
- public PrepareFailException(String pluginName, PluginType type, String message) {
- super(String.format("PluginName: %s, PluginType: %s, Message: %s", pluginName, type.getType(),
- message));
+ @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 000000000..301b1506f
--- /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<HiveSourceSplit, HiveSourceState> {
+
+ private final Context<HiveSourceSplit> context;
+ private Set<HiveSourceSplit> pendingSplit;
+ private Set<HiveSourceSplit> assignedSplit;
+ private List<String> filePaths;
+
+ public HiveSourceSplitEnumerator(SourceSplitEnumerator.Context<HiveSourceSplit> context, List<String> filePaths) {
+ this.context = context;
+ this.filePaths = filePaths;
+ }
+
+ public HiveSourceSplitEnumerator(SourceSplitEnumerator.Context<HiveSourceSplit> context, List<String> 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<HiveSourceSplit> getHiveFileSplit() {
+ Set<HiveSourceSplit> hiveSourceSplits = new HashSet<>();
+ filePaths.forEach(k -> hiveSourceSplits.add(new HiveSourceSplit(k)));
+ return hiveSourceSplits;
+
+ }
+
+ @Override
+ public void close() throws IOException {
+
+ }
+
+ @Override
+ public void addSplitsBack(List<HiveSourceSplit> splits, int subtaskId) {
+ if (!splits.isEmpty()) {
+ pendingSplit.addAll(splits);
+ assignSplit(Collections.singletonList(subtaskId));
+ }
+ }
+
+ private void assignSplit(Collection<Integer> taskIDList) {
+ Map<Integer, List<HiveSourceSplit>> 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-api/src/main/java/org/apache/seatunnel/api/common/PrepareFailException.java b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceState.java
similarity index 60%
copy from seatunnel-api/src/main/java/org/apache/seatunnel/api/common/PrepareFailException.java
copy to seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceState.java
index bb43d07ff..f982a71cb 100644
--- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/common/PrepareFailException.java
+++ b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/source/HiveSourceState.java
@@ -15,19 +15,21 @@
* limitations under the License.
*/
-package org.apache.seatunnel.api.common;
+package org.apache.seatunnel.connectors.seatunnel.hive.source;
-import org.apache.seatunnel.common.constants.PluginType;
+import java.io.Serializable;
+import java.util.Set;
-import org.apache.seatunnel.shade.com.typesafe.config.Config;
+public class HiveSourceState implements Serializable {
-/**
- * This exception will throw when {@link SeaTunnelPluginLifeCycle#prepare(Config)} failed.
- */
-public class PrepareFailException extends RuntimeException {
- public PrepareFailException(String pluginName, PluginType type, String message) {
- super(String.format("PluginName: %s, PluginType: %s, Message: %s", pluginName, type.getType(),
- message));
+ private Set<HiveSourceSplit> assignedSplit;
+
+ public HiveSourceState(Set<HiveSourceSplit> assignedSplit) {
+ this.assignedSplit = assignedSplit;
+ }
+
+ public Set<HiveSourceSplit> 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 000000000..50163235c
--- /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,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.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<String> getFileNamesByPath(HadoopConf hadoopConf, String path) throws IOException {
+ Configuration configuration = getConfiguration(hadoopConf);
+ List<String> 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()) {
+
+ 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 000000000..325c1aacc
--- /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<SeaTunnelRow> 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<NullWritable, OrcStruct> in = new OrcInputFormat();
+ FileInputFormat.setInputPaths(conf, filePath);
+ InputSplit[] splits = in.getSplits(conf, 1);
+
+ conf.set("hive.io.file.readcolumn.ids", "1");
+ RecordReader<NullWritable, OrcStruct> 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 000000000..5e7301914
--- /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<SeaTunnelRow> output) throws Exception;
+
+ SeaTunnelRowType getSeaTunnelRowTypeInfo(HadoopConf hadoopConf, String path) throws HivePluginException;
+
+ List<String> 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 000000000..aa899cd51
--- /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<SeaTunnelRow> 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[]{"TEXT_FIELD_NAME", 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/pom.xml b/seatunnel-dependency-shade/pom.xml
similarity index 58%
copy from seatunnel-connectors-v2/connector-hive/pom.xml
copy to seatunnel-dependency-shade/pom.xml
index eb2ffa797..1e6259991 100644
--- a/seatunnel-connectors-v2/connector-hive/pom.xml
+++ b/seatunnel-dependency-shade/pom.xml
@@ -20,37 +20,20 @@
<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
<parent>
- <artifactId>seatunnel-connectors-v2</artifactId>
+ <artifactId>seatunnel</artifactId>
<groupId>org.apache.seatunnel</groupId>
<version>${revision}</version>
+ <relativePath>../pom.xml</relativePath>
</parent>
<modelVersion>4.0.0</modelVersion>
- <artifactId>connector-hive</artifactId>
-
- <dependencies>
- <dependency>
- <groupId>org.apache.seatunnel</groupId>
- <artifactId>seatunnel-api</artifactId>
- <version>${project.version}</version>
- </dependency>
-
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-shaded-hadoop-2</artifactId>
- <version>${flink-shaded-hadoop-2.version}</version>
- <scope>provided</scope>
- </dependency>
-
- <dependency>
- <groupId>org.apache.commons</groupId>
- <artifactId>commons-lang3</artifactId>
- </dependency>
+ <artifactId>seatunnel-dependency-shade</artifactId>
+ <packaging>pom</packaging>
+
+ <modules>
+ <module>seatunnel-hive-shade</module>
+ </modules>
- <dependency>
- <groupId>junit</groupId>
- <artifactId>junit</artifactId>
- </dependency>
- </dependencies>
-</project>
+</project>
\ No newline at end of file
diff --git a/seatunnel-dependency-shade/seatunnel-hive-shade/pom.xml b/seatunnel-dependency-shade/seatunnel-hive-shade/pom.xml
new file mode 100644
index 000000000..ddab451b6
--- /dev/null
+++ b/seatunnel-dependency-shade/seatunnel-hive-shade/pom.xml
@@ -0,0 +1,69 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <parent>
+ <artifactId>seatunnel-dependency-shade</artifactId>
+ <groupId>org.apache.seatunnel</groupId>
+ <version>${revision}</version>
+ </parent>
+ <modelVersion>4.0.0</modelVersion>
+ <artifactId>seatunnel-hive-shade</artifactId>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.hive</groupId>
+ <artifactId>hive-exec</artifactId>
+ </dependency>
+ </dependencies>
+ <build>
+ <finalName>${project.artifactId}-${project.version}</finalName>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-shade-plugin</artifactId>
+ <executions>
+ <execution>
+ <phase>package</phase>
+ <goals>
+ <goal>shade</goal>
+ </goals>
+ <configuration>
+ <filters>
+ <filter>
+ <artifact>*:*</artifact>
+ <excludes>
+ <exclude>META-INF/*.SF</exclude>
+ <exclude>META-INF/*.DSA</exclude>
+ <exclude>META-INF/*.RSA</exclude>
+ <exclude>org/apache/calcite/**</exclude>
+ <exclude>org/htrace/**</exclude>
+ </excludes>
+ </filter>
+ </filters>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+
+ </plugins>
+ </build>
+</project>
\ No newline at end of file
diff --git a/seatunnel-dist/release-docs/LICENSE b/seatunnel-dist/release-docs/LICENSE
index 4e4d2f912..2a9a3c9af 100644
--- a/seatunnel-dist/release-docs/LICENSE
+++ b/seatunnel-dist/release-docs/LICENSE
@@ -556,7 +556,23 @@ The text of each license is the standard Apache 2.0 license.
(Apache License, Version 2.0) fastutil (it.unimi.dsi:fastutil:6.5.6 - http://fasutil.dsi.unimi.it/)
(Apache License, Version 2.0) fastutil (it.unimi.dsi:fastutil:7.0.13 - http://fasutil.di.unimi.it/)
(Apache License, Version 2.0) fastutil (it.unimi.dsi:fastutil:8.5.4 - http://fastutil.di.unimi.it/)
+ (Apache License, Version 2.0) hadoop-mapreduce-client-app (org.apache.hadoop:hadoop-mapreduce-client-app:2.6.5 - no url defined)
+ (Apache License, Version 2.0) hadoop-mapreduce-client-common (org.apache.hadoop:hadoop-mapreduce-client-common:2.6.5 - no url defined)
+ (Apache License, Version 2.0) hadoop-mapreduce-client-common (org.apache.hadoop:hadoop-mapreduce-client-common:3.0.0 - no url defined)
+ (Apache License, Version 2.0) hadoop-mapreduce-client-core (org.apache.hadoop:hadoop-mapreduce-client-core:2.6.5 - no url defined)
(Apache License, Version 2.0) hadoop-mapreduce-client-core (org.apache.hadoop:hadoop-mapreduce-client-core:2.7.7 - no url defined)
+ (Apache License, Version 2.0) hadoop-mapreduce-client-core (org.apache.hadoop:hadoop-mapreduce-client-core:3.0.0 - no url defined)
+ (Apache License, Version 2.0) hadoop-mapreduce-client-jobclient (org.apache.hadoop:hadoop-mapreduce-client-jobclient:2.6.5 - no url defined)
+ (Apache License, Version 2.0) hadoop-mapreduce-client-jobclient (org.apache.hadoop:hadoop-mapreduce-client-jobclient:3.0.0 - no url defined)
+ (Apache License, Version 2.0) hadoop-mapreduce-client-shuffle (org.apache.hadoop:hadoop-mapreduce-client-shuffle:2.6.5 - no url defined)
+ (Apache License, Version 2.0) hadoop-yarn-api (org.apache.hadoop:hadoop-yarn-api:2.6.5 - no url defined)
+ (Apache License, Version 2.0) hadoop-yarn-api (org.apache.hadoop:hadoop-yarn-api:3.0.0 - no url defined)
+ (Apache License, Version 2.0) hadoop-yarn-client (org.apache.hadoop:hadoop-yarn-client:2.6.5 - no url defined)
+ (Apache License, Version 2.0) hadoop-yarn-client (org.apache.hadoop:hadoop-yarn-client:3.0.0 - no url defined)
+ (Apache License, Version 2.0) hadoop-yarn-common (org.apache.hadoop:hadoop-yarn-common:2.6.5 - no url defined)
+ (Apache License, Version 2.0) hadoop-yarn-common (org.apache.hadoop:hadoop-yarn-common:2.7.7 - no url defined)
+ (Apache License, Version 2.0) hadoop-yarn-common (org.apache.hadoop:hadoop-yarn-common:3.0.0 - no url defined)
+ (Apache License, Version 2.0) hadoop-yarn-server-common (org.apache.hadoop:hadoop-yarn-server-common:2.6.5 - no url defined)
(Apache License, Version 2.0) htrace-core4 (org.apache.htrace:htrace-core4:4.2.0-incubating - http://incubator.apache.org/projects/htrace.html)
(Apache License, Version 2.0) hudi-spark-bundle_2.11 (org.apache.hudi:hudi-spark-bundle_2.11:0.10.0 - https://github.com/apache/hudi/hudi-spark-bundle_2.11)
(Apache License, Version 2.0) java-xmlbuilder (com.jamesmurty.utils:java-xmlbuilder:0.4 - http://code.google.com/p/java-xmlbuilder/)
@@ -664,24 +680,14 @@ The text of each license is the standard Apache 2.0 license.
(The Apache Software License, Version 2.0) Commons Digester (commons-digester:commons-digester:1.8.1 - http://commons.apache.org/digester/)
(The Apache Software License, Version 2.0) Commons IO (commons-io:commons-io:2.4 - http://commons.apache.org/io/)
(The Apache Software License, Version 2.0) Commons Lang (commons-lang:commons-lang:2.6 - http://commons.apache.org/lang/)
- (The Apache Software License, Version 2.0) Commons Logging (commons-logging:commons-logging:1.1.3 - http://commons.apache.org/proper/commons-logging/)
(The Apache Software License, Version 2.0) Commons Math (org.apache.commons:commons-math3:3.1.1 - http://commons.apache.org/math/)
(The Apache Software License, Version 2.0) Commons Net (commons-net:commons-net:3.1 - http://commons.apache.org/net/)
(The Apache Software License, Version 2.0) Commons Pool (commons-pool:commons-pool:1.5.4 - http://commons.apache.org/pool/)
(The Apache Software License, Version 2.0) Commons Pool (commons-pool:commons-pool:1.6 - http://commons.apache.org/pool/)
(The Apache Software License, Version 2.0) Converter: Moshi (com.squareup.retrofit2:converter-moshi:2.9.0 - https://github.com/square/retrofit)
(The Apache Software License, Version 2.0) CronScheduler (io.timeandspace:cron-scheduler:0.1 - https://github.com/TimeAndSpaceIO/CronScheduler)
- (The Apache Software License, Version 2.0) Curator Client (org.apache.curator:curator-client:2.12.0 - http://curator.apache.org/curator-client)
- (The Apache Software License, Version 2.0) Curator Client (org.apache.curator:curator-client:2.6.0 - http://curator.apache.org/curator-client)
- (The Apache Software License, Version 2.0) Curator Client (org.apache.curator:curator-client:2.7.1 - http://curator.apache.org/curator-client)
(The Apache Software License, Version 2.0) Curator Client (org.apache.curator:curator-client:4.3.0 - http://curator.apache.org/curator-client)
- (The Apache Software License, Version 2.0) Curator Framework (org.apache.curator:curator-framework:2.12.0 - http://curator.apache.org/curator-framework)
- (The Apache Software License, Version 2.0) Curator Framework (org.apache.curator:curator-framework:2.6.0 - http://curator.apache.org/curator-framework)
- (The Apache Software License, Version 2.0) Curator Framework (org.apache.curator:curator-framework:2.7.1 - http://curator.apache.org/curator-framework)
(The Apache Software License, Version 2.0) Curator Framework (org.apache.curator:curator-framework:4.3.0 - http://curator.apache.org/curator-framework)
- (The Apache Software License, Version 2.0) Curator Recipes (org.apache.curator:curator-recipes:2.12.0 - http://curator.apache.org/curator-recipes)
- (The Apache Software License, Version 2.0) Curator Recipes (org.apache.curator:curator-recipes:2.6.0 - http://curator.apache.org/curator-recipes)
- (The Apache Software License, Version 2.0) Curator Recipes (org.apache.curator:curator-recipes:2.7.1 - http://curator.apache.org/curator-recipes)
(The Apache Software License, Version 2.0) Curator Recipes (org.apache.curator:curator-recipes:4.3.0 - http://curator.apache.org/curator-recipes)
(The Apache Software License, Version 2.0) Curator Service Discovery (org.apache.curator:curator-x-discovery:4.3.0 - http://curator.apache.org/curator-x-discovery)
(The Apache Software License, Version 2.0) Data Mapper for Jackson (org.codehaus.jackson:jackson-mapper-asl:1.9.13 - http://jackson.codehaus.org)
@@ -756,7 +762,6 @@ The text of each license is the standard Apache 2.0 license.
(The Apache Software License, Version 2.0) JPam (net.sf.jpam:jpam:1.1 - http://jpam.sf.net)
(The Apache Software License, Version 2.0) JSON Small and Fast Parser (net.minidev:json-smart:2.3 - http://www.minidev.net/)
(The Apache Software License, Version 2.0) Jackson (org.codehaus.jackson:jackson-core-asl:1.9.13 - http://jackson.codehaus.org)
- (The Apache Software License, Version 2.0) Jackson (org.codehaus.jackson:jackson-core-asl:1.9.2 - http://jackson.codehaus.org)
(The Apache Software License, Version 2.0) Jackson 2 extensions to the Google HTTP Client Library for Java. (com.google.http-client:google-http-client-jackson2:1.26.0 - https://github.com/googleapis/google-http-java-client/google-http-client-jackson2)
(The Apache Software License, Version 2.0) Jackson dataformat: CBOR (com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.12.3 - http://github.com/FasterXML/jackson-dataformats-binary)
(The Apache Software License, Version 2.0) Jackson dataformat: CBOR (com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.8.10 - http://github.com/FasterXML/jackson-dataformats-binary)
@@ -870,18 +875,58 @@ The text of each license is the standard Apache 2.0 license.
(The Apache Software License, Version 2.0) secure-sm (org.elasticsearch:elasticsearch-secure-sm:6.3.1 - https://github.com/elastic/elasticsearch)
(The Apache Software License, Version 2.0) server (org.elasticsearch:elasticsearch:6.3.1 - https://github.com/elastic/elasticsearch)
(The Apache Software License, Version 2.0) server (org.elasticsearch:elasticsearch:7.5.1 - https://github.com/elastic/elasticsearch)
- (The Apache Software License, Version 2.0) snappy-java (org.xerial.snappy:snappy-java:1.1.2.6 - https://github.com/xerial/snappy-java)
- (The Apache Software License, Version 2.0) snappy-java (org.xerial.snappy:snappy-java:1.1.4 - https://github.com/xerial/snappy-java)
- (The Apache Software License, Version 2.0) snappy-java (org.xerial.snappy:snappy-java:1.1.7.1 - https://github.com/xerial/snappy-java)
- (The Apache Software License, Version 2.0) snappy-java (org.xerial.snappy:snappy-java:1.1.7.3 - https://github.com/xerial/snappy-java)
+ (The Apache Software License, Version 2.0) snappy-java (org.xerial.snappy:snappy-java:1.1.8.3 - https://github.com/xerial/snappy-java)
(The Apache Software License, Version 2.0) transport (org.elasticsearch.client:transport:6.3.1 - https://github.com/elastic/elasticsearch)
(The Apache Software License, Version 2.0) transport (org.elasticsearch.client:transport:7.5.1 - https://github.com/elastic/elasticsearch)
(The Apache Software License, Version 2.0) transport-netty4 (org.elasticsearch.plugin:transport-netty4-client:6.3.1 - https://github.com/elastic/elasticsearch)
(The Apache Software License, Version 2.0) transport-netty4 (org.elasticsearch.plugin:transport-netty4-client:7.5.1 - https://github.com/elastic/elasticsearch)
(The Apache Software License, Version 2.0) x-content (org.elasticsearch:elasticsearch-x-content:6.3.1 - https://github.com/elastic/elasticsearch)
(The Apache Software License, Version 2.0) zookeeper (org.apache.zookeeper:zookeeper:3.4.10 - no url defined)
-
-
+ (Apache License, Version 2.0) Hadoop Metrics2 Reporter for Dropwizard Metrics (com.github.joshelser:dropwizard-metrics-hadoop-metrics2-reporter:0.1.2 - https://github.com/joshelser/dropwizard-hadoop-metrics2)
+ (The Apache Software License, Version 2.0) Open JSON (com.tdunning:json:1.8 - https://github.com/tdunning/open-json)
+ (The Apache Software License, Version 2.0) EL (commons-el:commons-el:1.0 - http://jakarta.apache.org/commons/el/)
+ (Apache License 2.0) Metrics Core (io.dropwizard.metrics:metrics-core:3.1.0 - http://metrics.codahale.com/metrics-core/)
+ (Apache License 2.0) Jackson Integration for Metrics (io.dropwizard.metrics:metrics-json:3.1.0 - http://metrics.codahale.com/metrics-json/)
+ (Apache License 2.0) JVM Integration for Metrics (io.dropwizard.metrics:metrics-jvm:3.1.0 - http://metrics.codahale.com/metrics-jvm/)
+ (Apache 2) Joda-Time (joda-time:joda-time:2.8.1 - http://www.joda.org/joda-time/)
+ (Apache License, Version 2.0) eigenbase-properties (net.hydromatic:eigenbase-properties:1.1.5 - http://github.com/julianhyde/eigenbase-properties)
+ (Apache 2) opencsv (net.sf.opencsv:opencsv:2.3 - http://opencsv.sf.net)
+ (The Apache Software License, Version 2.0) Apache Ant Core (org.apache.ant:ant:1.9.1 - http://ant.apache.org/)
+ (The Apache Software License, Version 2.0) Apache Ant Launcher (org.apache.ant:ant-launcher:1.9.1 - http://ant.apache.org/)
+ (The Apache Software License, Version 2.0) Apache Commons Compress (org.apache.commons:commons-compress:1.9 - http://commons.apache.org/proper/commons-compress/)
+ (The Apache Software License, Version 2.0) Annotation 1.0 (org.apache.geronimo.specs:geronimo-annotation_1.0_spec:1.1.1 - http://geronimo.apache.org/specs/geronimo-annotation_1.0_spec)
+ (The Apache Software License, Version 2.0) Java Authentication SPI for Containers (org.apache.geronimo.specs:geronimo-jaspic_1.0_spec:1.0 - http://geronimo.apache.org/maven/specs/geronimo-jaspic_1.0_spec/1.0)
+ (The Apache Software License, Version 2.0) JTA 1.1 (org.apache.geronimo.specs:geronimo-jta_1.1_spec:1.1.1 - http://geronimo.apache.org/specs/geronimo-jta_1.1_spec)
+ (The Apache Software License, Version 2.0) Hive Common (org.apache.hive:hive-common:2.3.9 - https://hive.apache.org/hive-common)
+ (The Apache Software License, Version 2.0) Hive Query Language (org.apache.hive:hive-exec:2.3.9 - https://hive.apache.org/hive-exec)
+ (The Apache Software License, Version 2.0) Hive Llap Client (org.apache.hive:hive-llap-client:2.3.9 - https://hive.apache.org/hive-llap-client)
+ (The Apache Software License, Version 2.0) Hive Llap Common (org.apache.hive:hive-llap-common:2.3.9 - https://hive.apache.org/hive-llap-common)
+ (The Apache Software License, Version 2.0) Hive Llap Tez (org.apache.hive:hive-llap-tez:2.3.9 - https://hive.apache.org/hive-llap-tez)
+ (The Apache Software License, Version 2.0) Hive Serde (org.apache.hive:hive-serde:2.3.9 - https://hive.apache.org/hive-serde)
+ (The Apache Software License, Version 2.0) Hive Service RPC (org.apache.hive:hive-service-rpc:2.3.9 - https://hive.apache.org/hive-service-rpc)
+ (The Apache Software License, Version 2.0) Hive Shims (org.apache.hive:hive-shims:2.3.9 - https://hive.apache.org/hive-shims)
+ (The Apache Software License, Version 2.0) Hive Vector-Code-Gen Utilities (org.apache.hive:hive-vector-code-gen:2.3.9 - https://hive.apache.org/hive-vector-code-gen)
+ (The Apache Software License, Version 2.0) Hive Shims 0.23 (org.apache.hive.shims:hive-shims-0.23:2.3.9 - https://hive.apache.org/hive-shims-0.23)
+ (The Apache Software License, Version 2.0) Hive Shims Common (org.apache.hive.shims:hive-shims-common:2.3.9 - https://hive.apache.org/hive-shims-common)
+ (The Apache Software License, Version 2.0) Hive Shims Scheduler (org.apache.hive.shims:hive-shims-scheduler:2.3.9 - https://hive.apache.org/hive-shims-scheduler)
+ (The Apache Software License, Version 2.0) Apache Ivy (org.apache.ivy:ivy:2.4.0 - http://ant.apache.org/ivy/)
+ (The Apache Software License, Version 2.0) Apache Parquet Hadoop Bundle (org.apache.parquet:parquet-hadoop-bundle:1.8.1 - https://parquet.apache.org)
+ (The Apache Software License, Version 2.0) Apache Thrift (org.apache.thrift:libfb303:0.9.3 - http://thrift.apache.org)
+ (The Apache Software License, Version 2.0) Apache Thrift (org.apache.thrift:libthrift:0.9.3 - http://thrift.apache.org)
+ (The Apache Software License, Version 2.0) Apache Velocity (org.apache.velocity:velocity:1.5 - http://velocity.apache.org/engine/releases/velocity-1.5/)
+ (The Apache Software License, Version 2.0) Apache Groovy (org.codehaus.groovy:groovy-all:2.4.4 - http://groovy-lang.org)
+ (The Apache Software License, Version 2.0) Data Mapper for Jackson (org.codehaus.jackson:jackson-mapper-asl:1.8.13 - http://jackson.codehaus.org)
+ (Apache Software License - Version 2.0) Jettison (org.codehaus.jettison:jettison:1.1 - nhttps://mvnrepository.com/artifact/org.codehaus.jettison/jettison)
+ (The Apache Software License, Version 2.0) DataNucleus Core (org.datanucleus:datanucleus-core:4.1.17 - http://www.datanucleus.org/#/datanucleus-core)
+ (Apache Software License - Version 2.0) (Eclipse Public License - Version 1.0) Jetty Orbit :: Servlet API (org.eclipse.jetty.orbit:javax.servlet:3.0.0.v201112011016 - http://www.eclipse.org/jetty/jetty-orbit/javax.servlet)
+ (Apache Software License - Version 2.0) (Eclipse Public License - Version 1.0) Jetty Server (org.mortbay.jetty:jetty:6.1.26 - http://www.eclipse.org/jetty/jetty-parent/project/modules/jetty)
+ (Apache Software License - Version 2.0) oro (oro:oro:2.0.8 - https://mvnrepository.com/artifact/oro/oro/2.0.8)
+ (The Apache Software License, Version 2.0) StAX API (stax:stax-api:1.0.1 - http://stax.codehaus.org/)
+ (The Apache Software License, Version 2.0) jasper-compiler (tomcat:jasper-compiler:5.5.23 - http://tomcat.apache.org/jasper-compiler)
+ (The Apache Software License, Version 2.0) jasper-runtime (tomcat:jasper-runtime:5.5.23 - http://tomcat.apache.org/jasper-runtime)
+ (The Apache Software License, Version 2.0) JAX-RS provider for JSON content type (org.codehaus.jackson:jackson-jaxrs:1.9.13 - http://jackson.codehaus.org)
+ (The Apache Software License, Version 2.0) Xml Compatibility extensions for Jackson (org.codehaus.jackson:jackson-xc:1.9.13 - http://jackson.codehaus.org)
+
========================================================================
MIT License
========================================================================
@@ -980,6 +1025,13 @@ The text of each license is also included at licenses/LICENSE-[project].txt.
(The BSD License) xmlenc Library (xmlenc:xmlenc:0.52 - http://xmlenc.sourceforge.net)
(The New BSD License) Jodd Core (org.jodd:jodd-core:3.5.2 - http://jodd.org)
(The New BSD License) Py4J (net.sf.py4j:py4j:0.10.7 - http://nexus.sonatype.org/oss-repository-hosting.html/py4j)
+ (The BSD License) xmlenc Library (xmlenc:xmlenc:0.52 - http://xmlenc.sourceforge.net)
+ (The BSD License) ASM Core (asm:asm:3.1 - http://asm.objectweb.org/asm/)
+ (The BSD License) ASM Commons (asm:asm-commons:3.1 - http://asm.objectweb.org/asm-commons/)
+ (The BSD License) ASM Tree (asm:asm-tree:3.1 - http://asm.objectweb.org/asm-tree/)
+ (New BSD License) Commons Compiler (org.codehaus.janino:commons-compiler:2.7.6 - http://docs.codehaus.org/display/JANINO/Home/commons-compiler)
+ (New BSD License) Janino (org.codehaus.janino:janino:3.1.6 - http://docs.codehaus.org/display/JANINO/Home/janino)
+ (BSD licence) ANTLR ST4 4.0.4 (org.antlr:ST4:4.0.4 - http://www.stringtemplate.org)
========================================================================
@@ -1038,6 +1090,9 @@ The text of each license is also included at licenses/LICENSE-[project].txt.
(CDDL License) JavaBeans Activation Framework (com.sun.activation:javax.activation:1.2.0 - http://java.net/all/javax.activation/)
(CDDL License) JavaBeans Activation Framework API jar (javax.activation:javax.activation-api:1.2.0 - http://java.net/all/javax.activation-api/)
(CDDL License) JavaMail API (com.sun.mail:javax.mail:1.5.6 - http://javamail.java.net/javax.mail)
+ (Common Development and Distribution License (CDDL) v1.0) JavaMail API (javax.mail:mail:1.4.1 - https://glassfish.dev.java.net/javaee5/mail/)
+ (COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0) (GNU General Public Library) Streaming API for XML (javax.xml.stream:stax-api:1.0-2 - no url defined)
+
========================================================================
diff --git a/seatunnel-dist/release-docs/NOTICE b/seatunnel-dist/release-docs/NOTICE
index fe3c949f6..564463f1a 100644
--- a/seatunnel-dist/release-docs/NOTICE
+++ b/seatunnel-dist/release-docs/NOTICE
@@ -4389,4 +4389,144 @@ Copyright 1999-2020 The Apache Software Foundation
This product includes software developed at
The Apache Software Foundation (http://www.apache.org/).
+=========================================================================
+
+Open Json NOTICE
+
+=========================================================================
+Android JSON library
+Copyright (C) 2010 The Android Open Source Project
+
+This product includes software developed by
+The Android Open Source Project
+=========================================================================
+
+Metrics NOTICE
+
+=========================================================================
+
+Metrics
+Copyright 2010-2013 Coda Hale and Yammer, Inc.
+
+This product includes software developed by Coda Hale and Yammer, Inc.
+
+This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64,
+LongAdder), which was released with the following comments:
+
+ Written by Doug Lea with assistance from members of JCP JSR-166
+ Expert Group and released to the public domain, as explained at
+ http://creativecommons.org/publicdomain/zero/1.0/
+
+
+=========================================================================
+
+Joda-Time NOTICE
+
+=============================================================================
+= NOTICE file corresponding to section 4d of the Apache License Version 2.0 =
+=============================================================================
+This product includes software developed by
+Joda.org (https://www.joda.org/).
+
+
+=========================================================================
+
+eigenbase-properties NOTICE
+
+=========================================================================
+
+eigenbase-properties
+Copyright (C) 2012-2015, Julian Hyde
+
+This product includes software from the Eigenbase project, licensed from
+DynamoBI Corporation.
+
+Copyright (C) 2005 Dynamo BI Corporation
+
+===============================================================================
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this software 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 lang
+
+=========================================================================
+
+ Apache Ant NOTICE
+
+=========================================================================
+ Apache Ant
+ Copyright 1999-2013 The Apache Software Foundation
+
+ The <sync> task is based on code Copyright (c) 2002, Landmark
+ Graphics Corp that has been kindly donated to the Apache Software
+ Foundation.
+
+
+=========================================================================
+
+Apache Geronimo NOTICE
+
+=========================================================================
+Apache Geronimo
+Copyright 2003-2008 The Apache Software Foundation
+
+This product includes software developed by
+The Apache Software Foundation (http://www.apache.org/).
+
+=========================================================================
+
+Apache Ivy NOTICE
+
+=========================================================================
+Apache Ivy (TM)
+Copyright 2007-2014 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+Portions of Ivy were originally developed at
+Jayasoft SARL (http://www.jayasoft.fr/)
+and are licensed to the Apache Software Foundation under the
+"Software Grant License Agreement"
+
+SSH and SFTP support is provided by the JCraft JSch package,
+which is open source software, available under
+the terms of a BSD style license.
+The original software and related information is available
+at http://www.jcraft.com/jsch/.
+
+=========================================================================
+
+Apache Parquet Hadoop Bundle NOTICE
+
+=========================================================================
+Apache Parquet Hadoop Bundle
+Copyright 2015 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+=========================================================================
+
+Apache Velocity NOTICE
+
+=========================================================================
+
+Apache Velocity
+
+Copyright (C) 2000-2007 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+The files
+ - velocity-engine-scripting/src/main/java/org/apache/velocity/script/VelocityScriptEngine.java
+ - velocity-engine-scripting/src/main/java/org/apache/velocity/script/VelocityScriptEngineFactory.java
+are Copyright 2006 Sun Microsystems, Inc., and licenced under a BSD-like licence.
+
=========================================================================
\ No newline at end of file
diff --git a/seatunnel-dist/release-docs/licenses/LICENSE-ant.txt b/seatunnel-dist/release-docs/licenses/LICENSE-ant.txt
new file mode 100644
index 000000000..d20c10c91
--- /dev/null
+++ b/seatunnel-dist/release-docs/licenses/LICENSE-ant.txt
@@ -0,0 +1,48 @@
+
+ changes were made. (We recommend you provide URIs to the location from
+ which the code is derived.)
+
+THIS SOFTWARE AND DOCUMENTATION IS PROVIDED "AS IS," AND COPYRIGHT HOLDERS MAKE
+NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED
+TO, WARRANTIES OF MERCHANTABILITY OR FITNESS FOR ANY PARTICULAR PURPOSE OR THAT
+THE USE OF THE SOFTWARE OR DOCUMENTATION WILL NOT INFRINGE ANY THIRD PARTY
+PATENTS, COPYRIGHTS, TRADEMARKS OR OTHER RIGHTS.
+
+COPYRIGHT HOLDERS WILL NOT BE LIABLE FOR ANY DIRECT, INDIRECT, SPECIAL OR
+CONSEQUENTIAL DAMAGES ARISING OUT OF ANY USE OF THE SOFTWARE OR DOCUMENTATION.
+
+The name and trademarks of copyright holders may NOT be used in advertising or
+publicity pertaining to the software without specific, written prior permission.
+Title to copyright in this software and any associated documentation will at
+all times remain with copyright holders.
+
+____________________________________
+
+This formulation of W3C's notice and license became active on December 31 2002.
+This version removes the copyright ownership notice such that this license can
+be used with materials other than those owned by the W3C, reflects that ERCIM
+is now a host of the W3C, includes references to this specific dated version of
+the license, and removes the ambiguous grant of "use". Otherwise, this version
+is the same as the previous version and is written so as to preserve the Free
+Software Foundation's assessment of GPL compatibility and OSI's certification
+under the Open Source Definition. Please see our Copyright FAQ for common
+questions about using materials from our site, including specific terms and
+conditions for packages like libwww, Amaya, and Jigsaw. Other questions about
+this notice can be directed to site-policy@w3.org.
+
+Joseph Reagle <si...@w3.org>
+
+This license came from: http://www.megginson.com/SAX/copying.html
+ However please note future versions of SAX may be covered
+ under http://saxproject.org/?selected=pd
+
+SAX2 is Free!
+
+I hereby abandon any property rights to SAX 2.0 (the Simple API for
+XML), and release all of the SAX 2.0 source code, compiled code, and
+documentation contained in this distribution into the Public Domain.
+SAX comes with NO WARRANTY or guarantee of fitness for any
+purpose.
+
+David Megginson, david@megginson.com
+2000-05-05
\ No newline at end of file
diff --git a/seatunnel-dist/release-docs/licenses/LICENSE-ivy.txt b/seatunnel-dist/release-docs/licenses/LICENSE-ivy.txt
new file mode 100644
index 000000000..87a7f0136
--- /dev/null
+++ b/seatunnel-dist/release-docs/licenses/LICENSE-ivy.txt
@@ -0,0 +1,259 @@
+
+
+ Apache License
+ Version 2.0, January 2004
+ http://www.apache.org/licenses/
+
+ TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+ 1. Definitions.
+
+ "License" shall mean the terms and conditions for use, reproduction,
+ and distribution as defined by Sections 1 through 9 of this document.
+
+ "Licensor" shall mean the copyright owner or entity authorized by
+ the copyright owner that is granting the License.
+
+ "Legal Entity" shall mean the union of the acting entity and all
+ other entities that control, are controlled by, or are under common
+ control with that entity. For the purposes of this definition,
+ "control" means (i) the power, direct or indirect, to cause the
+ direction or management of such entity, whether by contract or
+ otherwise, or (ii) ownership of fifty percent (50%) or more of the
+ outstanding shares, or (iii) beneficial ownership of such entity.
+
+ "You" (or "Your") shall mean an individual or Legal Entity
+ exercising permissions granted by this License.
+
+ "Source" form shall mean the preferred form for making modifications,
+ including but not limited to software source code, documentation
+ source, and configuration files.
+
+ "Object" form shall mean any form resulting from mechanical
+ transformation or translation of a Source form, including but
+ not limited to compiled object code, generated documentation,
+ and conversions to other media types.
+
+ "Work" shall mean the work of authorship, whether in Source or
+ Object form, made available under the License, as indicated by a
+ copyright notice that is included in or attached to the work
+ (an example is provided in the Appendix below).
+
+ "Derivative Works" shall mean any work, whether in Source or Object
+ form, that is based on (or derived from) the Work and for which the
+ editorial revisions, annotations, elaborations, or other modifications
+ represent, as a whole, an original work of authorship. For the purposes
+ of this License, Derivative Works shall not include works that remain
+ separable from, or merely link (or bind by name) to the interfaces of,
+ the Work and Derivative Works thereof.
+
+ "Contribution" shall mean any work of authorship, including
+ the original version of the Work and any modifications or additions
+ to that Work or Derivative Works thereof, that is intentionally
+ submitted to Licensor for inclusion in the Work by the copyright owner
+ or by an individual or Legal Entity authorized to submit on behalf of
+ the copyright owner. For the purposes of this definition, "submitted"
+ means any form of electronic, verbal, or written communication sent
+ to the Licensor or its representatives, including but not limited to
+ communication on electronic mailing lists, source code control systems,
+ and issue tracking systems that are managed by, or on behalf of, the
+ Licensor for the purpose of discussing and improving the Work, but
+ excluding communication that is conspicuously marked or otherwise
+ designated in writing by the copyright owner as "Not a Contribution."
+
+ "Contributor" shall mean Licensor and any individual or Legal Entity
+ on behalf of whom a Contribution has been received by Licensor and
+ subsequently incorporated within the Work.
+
+ 2. Grant of Copyright License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ copyright license to reproduce, prepare Derivative Works of,
+ publicly display, publicly perform, sublicense, and distribute the
+ Work and such Derivative Works in Source or Object form.
+
+ 3. Grant of Patent License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ (except as stated in this section) patent license to make, have made,
+ use, offer to sell, sell, import, and otherwise transfer the Work,
+ where such license applies only to those patent claims licensable
+ by such Contributor that are necessarily infringed by their
+ Contribution(s) alone or by combination of their Contribution(s)
+ with the Work to which such Contribution(s) was submitted. If You
+ institute patent litigation against any entity (including a
+ cross-claim or counterclaim in a lawsuit) alleging that the Work
+ or a Contribution incorporated within the Work constitutes direct
+ or contributory patent infringement, then any patent licenses
+ granted to You under this License for that Work shall terminate
+ as of the date such litigation is filed.
+
+ 4. Redistribution. You may reproduce and distribute copies of the
+ Work or Derivative Works thereof in any medium, with or without
+ modifications, and in Source or Object form, provided that You
+ meet the following conditions:
+
+ (a) You must give any other recipients of the Work or
+ Derivative Works a copy of this License; and
+
+ (b) You must cause any modified files to carry prominent notices
+ stating that You changed the files; and
+
+ (c) You must retain, in the Source form of any Derivative Works
+ that You distribute, all copyright, patent, trademark, and
+ attribution notices from the Source form of the Work,
+ excluding those notices that do not pertain to any part of
+ the Derivative Works; and
+
+ (d) If the Work includes a "NOTICE" text file as part of its
+ distribution, then any Derivative Works that You distribute must
+ include a readable copy of the attribution notices contained
+ within such NOTICE file, excluding those notices that do not
+ pertain to any part of the Derivative Works, in at least one
+ of the following places: within a NOTICE text file distributed
+ as part of the Derivative Works; within the Source form or
+ documentation, if provided along with the Derivative Works; or,
+ within a display generated by the Derivative Works, if and
+ wherever such third-party notices normally appear. The contents
+ of the NOTICE file are for informational purposes only and
+ do not modify the License. You may add Your own attribution
+ notices within Derivative Works that You distribute, alongside
+ or as an addendum to the NOTICE text from the Work, provided
+ that such additional attribution notices cannot be construed
+ as modifying the License.
+
+ You may add Your own copyright statement to Your modifications and
+ may provide additional or different license terms and conditions
+ for use, reproduction, or distribution of Your modifications, or
+ for any such Derivative Works as a whole, provided Your use,
+ reproduction, and distribution of the Work otherwise complies with
+ the conditions stated in this License.
+
+ 5. Submission of Contributions. Unless You explicitly state otherwise,
+ any Contribution intentionally submitted for inclusion in the Work
+ by You to the Licensor shall be under the terms and conditions of
+ this License, without any additional terms or conditions.
+ Notwithstanding the above, nothing herein shall supersede or modify
+ the terms of any separate license agreement you may have executed
+ with Licensor regarding such Contributions.
+
+ 6. Trademarks. This License does not grant permission to use the trade
+ names, trademarks, service marks, or product names of the Licensor,
+ except as required for reasonable and customary use in describing the
+ origin of the Work and reproducing the content of the NOTICE file.
+
+ 7. Disclaimer of Warranty. Unless required by applicable law or
+ agreed to in writing, Licensor provides the Work (and each
+ Contributor provides its Contributions) on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ implied, including, without limitation, any warranties or conditions
+ of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+ PARTICULAR PURPOSE. You are solely responsible for determining the
+ appropriateness of using or redistributing the Work and assume any
+ risks associated with Your exercise of permissions under this License.
+
+ 8. Limitation of Liability. In no event and under no legal theory,
+ whether in tort (including negligence), contract, or otherwise,
+ unless required by applicable law (such as deliberate and grossly
+ negligent acts) or agreed to in writing, shall any Contributor be
+ liable to You for damages, including any direct, indirect, special,
+ incidental, or consequential damages of any character arising as a
+ result of this License or out of the use or inability to use the
+ Work (including but not limited to damages for loss of goodwill,
+ work stoppage, computer failure or malfunction, or any and all
+ other commercial damages or losses), even if such Contributor
+ has been advised of the possibility of such damages.
+
+ 9. Accepting Warranty or Additional Liability. While redistributing
+ the Work or Derivative Works thereof, You may choose to offer,
+ and charge a fee for, acceptance of support, warranty, indemnity,
+ or other liability obligations and/or rights consistent with this
+ License. However, in accepting such obligations, You may act only
+ on Your own behalf and on Your sole responsibility, not on behalf
+ of any other Contributor, and only if You agree to indemnify,
+ defend, and hold each Contributor harmless for any liability
+ incurred by, or claims asserted against, such Contributor by reason
+ of your accepting any such warranty or additional liability.
+
+ END OF TERMS AND CONDITIONS
+
+ APPENDIX: How to apply the Apache License to your work.
+
+ To apply the Apache License to your work, attach the following
+ boilerplate notice, with the fields enclosed by brackets "[]"
+ replaced with your own identifying information. (Don't include
+ the brackets!) The text should be enclosed in the appropriate
+ comment syntax for the file format. We also recommend that a
+ file or class name and description of purpose be included on the
+ same "printed page" as the copyright notice for easier
+ identification within third-party archives.
+
+ Copyright [yyyy] [name of copyright owner]
+
+ Licensed 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.
+
+
+------------------------------------------------------------------------------
+License for JCraft JSch package
+------------------------------------------------------------------------------
+Copyright (c) 2002,2003,2004,2005,2006,2007 Atsuhiko Yamanaka, JCraft,Inc.
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+ 1. Redistributions of source code must retain the above copyright notice,
+ this list of conditions and the following disclaimer.
+
+ 2. Redistributions in binary form must reproduce the above copyright
+ notice, this list of conditions and the following disclaimer in
+ the documentation and/or other materials provided with the distribution.
+
+ 3. The names of the authors may not be used to endorse or promote products
+ derived from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED WARRANTIES,
+INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND
+FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL JCRAFT,
+INC. OR ANY CONTRIBUTORS TO THIS SOFTWARE BE LIABLE FOR ANY DIRECT, INDIRECT,
+INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA,
+OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
+LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE,
+EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+------------------------------------------------------------------------------
+License for jQuery
+------------------------------------------------------------------------------
+Copyright (c) 2007 John Resig, http://jquery.com/
+
+Permission is hereby granted, free of charge, to any person obtaining
+a copy of this software and associated documentation files (the
+"Software"), to deal in the Software without restriction, including
+without limitation the rights to use, copy, modify, merge, publish,
+distribute, sublicense, and/or sell copies of the Software, and to
+permit persons to whom the Software is furnished to do so, subject to
+the following conditions:
+
+The above copyright notice and this permission notice shall be
+included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
+WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
\ No newline at end of file
diff --git a/seatunnel-dist/release-docs/licenses/LICENSE-parquet-hadoop-bundle.txt b/seatunnel-dist/release-docs/licenses/LICENSE-parquet-hadoop-bundle.txt
new file mode 100644
index 000000000..b53f78f5c
--- /dev/null
+++ b/seatunnel-dist/release-docs/licenses/LICENSE-parquet-hadoop-bundle.txt
@@ -0,0 +1,248 @@
+
+ Apache License
+ Version 2.0, January 2004
+ http://www.apache.org/licenses/
+
+ TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+ 1. Definitions.
+
+ "License" shall mean the terms and conditions for use, reproduction,
+ and distribution as defined by Sections 1 through 9 of this document.
+
+ "Licensor" shall mean the copyright owner or entity authorized by
+ the copyright owner that is granting the License.
+
+ "Legal Entity" shall mean the union of the acting entity and all
+ other entities that control, are controlled by, or are under common
+ control with that entity. For the purposes of this definition,
+ "control" means (i) the power, direct or indirect, to cause the
+ direction or management of such entity, whether by contract or
+ otherwise, or (ii) ownership of fifty percent (50%) or more of the
+ outstanding shares, or (iii) beneficial ownership of such entity.
+
+ "You" (or "Your") shall mean an individual or Legal Entity
+ exercising permissions granted by this License.
+
+ "Source" form shall mean the preferred form for making modifications,
+ including but not limited to software source code, documentation
+ source, and configuration files.
+
+ "Object" form shall mean any form resulting from mechanical
+ transformation or translation of a Source form, including but
+ not limited to compiled object code, generated documentation,
+ and conversions to other media types.
+
+ "Work" shall mean the work of authorship, whether in Source or
+ Object form, made available under the License, as indicated by a
+ copyright notice that is included in or attached to the work
+ (an example is provided in the Appendix below).
+
+ "Derivative Works" shall mean any work, whether in Source or Object
+ form, that is based on (or derived from) the Work and for which the
+ editorial revisions, annotations, elaborations, or other modifications
+ represent, as a whole, an original work of authorship. For the purposes
+ of this License, Derivative Works shall not include works that remain
+ separable from, or merely link (or bind by name) to the interfaces of,
+ the Work and Derivative Works thereof.
+
+ "Contribution" shall mean any work of authorship, including
+ the original version of the Work and any modifications or additions
+ to that Work or Derivative Works thereof, that is intentionally
+ submitted to Licensor for inclusion in the Work by the copyright owner
+ or by an individual or Legal Entity authorized to submit on behalf of
+ the copyright owner. For the purposes of this definition, "submitted"
+ means any form of electronic, verbal, or written communication sent
+ to the Licensor or its representatives, including but not limited to
+ communication on electronic mailing lists, source code control systems,
+ and issue tracking systems that are managed by, or on behalf of, the
+ Licensor for the purpose of discussing and improving the Work, but
+ excluding communication that is conspicuously marked or otherwise
+ designated in writing by the copyright owner as "Not a Contribution."
+
+ "Contributor" shall mean Licensor and any individual or Legal Entity
+ on behalf of whom a Contribution has been received by Licensor and
+ subsequently incorporated within the Work.
+
+ 2. Grant of Copyright License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ copyright license to reproduce, prepare Derivative Works of,
+ publicly display, publicly perform, sublicense, and distribute the
+ Work and such Derivative Works in Source or Object form.
+
+ 3. Grant of Patent License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ (except as stated in this section) patent license to make, have made,
+ use, offer to sell, sell, import, and otherwise transfer the Work,
+ where such license applies only to those patent claims licensable
+ by such Contributor that are necessarily infringed by their
+ Contribution(s) alone or by combination of their Contribution(s)
+ with the Work to which such Contribution(s) was submitted. If You
+ institute patent litigation against any entity (including a
+ cross-claim or counterclaim in a lawsuit) alleging that the Work
+ or a Contribution incorporated within the Work constitutes direct
+ or contributory patent infringement, then any patent licenses
+ granted to You under this License for that Work shall terminate
+ as of the date such litigation is filed.
+
+ 4. Redistribution. You may reproduce and distribute copies of the
+ Work or Derivative Works thereof in any medium, with or without
+ modifications, and in Source or Object form, provided that You
+ meet the following conditions:
+
+ (a) You must give any other recipients of the Work or
+ Derivative Works a copy of this License; and
+
+ (b) You must cause any modified files to carry prominent notices
+ stating that You changed the files; and
+
+ (c) You must retain, in the Source form of any Derivative Works
+ that You distribute, all copyright, patent, trademark, and
+ attribution notices from the Source form of the Work,
+ excluding those notices that do not pertain to any part of
+ the Derivative Works; and
+
+ (d) If the Work includes a "NOTICE" text file as part of its
+ distribution, then any Derivative Works that You distribute must
+ include a readable copy of the attribution notices contained
+ within such NOTICE file, excluding those notices that do not
+ pertain to any part of the Derivative Works, in at least one
+ of the following places: within a NOTICE text file distributed
+ as part of the Derivative Works; within the Source form or
+ documentation, if provided along with the Derivative Works; or,
+ within a display generated by the Derivative Works, if and
+ wherever such third-party notices normally appear. The contents
+ of the NOTICE file are for informational purposes only and
+ do not modify the License. You may add Your own attribution
+ notices within Derivative Works that You distribute, alongside
+ or as an addendum to the NOTICE text from the Work, provided
+ that such additional attribution notices cannot be construed
+ as modifying the License.
+
+ You may add Your own copyright statement to Your modifications and
+ may provide additional or different license terms and conditions
+ for use, reproduction, or distribution of Your modifications, or
+ for any such Derivative Works as a whole, provided Your use,
+ reproduction, and distribution of the Work otherwise complies with
+ the conditions stated in this License.
+
+ 5. Submission of Contributions. Unless You explicitly state otherwise,
+ any Contribution intentionally submitted for inclusion in the Work
+ by You to the Licensor shall be under the terms and conditions of
+ this License, without any additional terms or conditions.
+ Notwithstanding the above, nothing herein shall supersede or modify
+ the terms of any separate license agreement you may have executed
+ with Licensor regarding such Contributions.
+
+ 6. Trademarks. This License does not grant permission to use the trade
+ names, trademarks, service marks, or product names of the Licensor,
+ except as required for reasonable and customary use in describing the
+ origin of the Work and reproducing the content of the NOTICE file.
+
+ 7. Disclaimer of Warranty. Unless required by applicable law or
+ agreed to in writing, Licensor provides the Work (and each
+ Contributor provides its Contributions) on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ implied, including, without limitation, any warranties or conditions
+ of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+ PARTICULAR PURPOSE. You are solely responsible for determining the
+ appropriateness of using or redistributing the Work and assume any
+ risks associated with Your exercise of permissions under this License.
+
+ 8. Limitation of Liability. In no event and under no legal theory,
+ whether in tort (including negligence), contract, or otherwise,
+ unless required by applicable law (such as deliberate and grossly
+ negligent acts) or agreed to in writing, shall any Contributor be
+ liable to You for damages, including any direct, indirect, special,
+ incidental, or consequential damages of any character arising as a
+ result of this License or out of the use or inability to use the
+ Work (including but not limited to damages for loss of goodwill,
+ work stoppage, computer failure or malfunction, or any and all
+ other commercial damages or losses), even if such Contributor
+ has been advised of the possibility of such damages.
+
+ 9. Accepting Warranty or Additional Liability. While redistributing
+ the Work or Derivative Works thereof, You may choose to offer,
+ and charge a fee for, acceptance of support, warranty, indemnity,
+ or other liability obligations and/or rights consistent with this
+ License. However, in accepting such obligations, You may act only
+ on Your own behalf and on Your sole responsibility, not on behalf
+ of any other Contributor, and only if You agree to indemnify,
+ defend, and hold each Contributor harmless for any liability
+ incurred by, or claims asserted against, such Contributor by reason
+ of your accepting any such warranty or additional liability.
+
+ END OF TERMS AND CONDITIONS
+
+--------------------------------------------------------------------------------
+
+This product depends on Apache Thrift and includes it in this binary artifact.
+
+Copyright: 2006-2010 The Apache Software Foundation.
+Home page: https://thrift.apache.org/
+License: http://www.apache.org/licenses/LICENSE-2.0
+
+--------------------------------------------------------------------------------
+
+This product depends on SLF4J and includes SLF4J in this binary artifact. SLF4J
+is a simple logging facade for Java.
+
+Copyright: 2004-2013 QOS.ch.
+Home page: http://www.slf4j.org/
+License: http://slf4j.org/license.html (MIT license)
+
+The following is the SLF4J license (MIT):
+
+ Copyright (c) 2004-2013 QOS.ch
+ All rights reserved.
+
+ Permission is hereby granted, free of charge, to any person obtaining
+ a copy of this software and associated documentation files (the
+ "Software"), to deal in the Software without restriction, including
+ without limitation the rights to use, copy, modify, merge, publish,
+ distribute, sublicense, and/or sell copies of the Software, and to
+ permit persons to whom the Software is furnished to do so, subject to
+ the following conditions:
+
+ The above copyright notice and this permission notice shall be
+ included in all copies or substantial portions of the Software.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+ EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+ MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+ NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+ LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+ OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
+ WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+This project includes code from Daniel Lemire's JavaFastPFOR project in this
+binary artifact. The "Lemire" bit packing classes produced by parquet-generator
+are derived from the JavaFastPFOR project.
+
+Copyright: 2013 Daniel Lemire
+Home page: http://lemire.me/en/
+Project page: https://github.com/lemire/JavaFastPFOR
+License: Apache License Version 2.0 http://www.apache.org/licenses/LICENSE-2.0
+
+--------------------------------------------------------------------------------
+
+This product depends on fastutil and includes it in this binary artifact.
+Fastutil provides type-specific collection implementations.
+
+Copyright: 2002-2014 Sebastiano Vigna
+Home page: http://fasutil.di.unimi.it/
+License: http://www.apache.org/licenses/LICENSE-2.0.html
+
+--------------------------------------------------------------------------------
+
+This product depends on Jackson and includes it in this binary artifact.
+Jackson is a high-performance JSON processor.
+
+Copyright: 2007-2015 Tatu Saloranta and other contributors
+Home page: http://jackson.codehaus.org/
+License: http://www.apache.org/licenses/LICENSE-2.0.txt
+
diff --git a/seatunnel-dist/release-docs/licenses/LICENSE-st.txt b/seatunnel-dist/release-docs/licenses/LICENSE-st.txt
new file mode 100644
index 000000000..3a33e34b1
--- /dev/null
+++ b/seatunnel-dist/release-docs/licenses/LICENSE-st.txt
@@ -0,0 +1,26 @@
+[The "BSD license"]
+Copyright (c) 2011-2022 Terence Parr
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions
+are met:
+
+ 1. Redistributions of source code must retain the above copyright
+ notice, this list of conditions and the following disclaimer.
+ 2. Redistributions in binary form must reproduce the above copyright
+ notice, this list of conditions and the following disclaimer in the
+ documentation and/or other materials provided with the distribution.
+ 3. The name of the author may not be used to endorse or promote products
+ derived from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR
+IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
+OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
+IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
+INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
+NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
+THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
\ No newline at end of file
diff --git a/tools/dependencies/known-dependencies.txt b/tools/dependencies/known-dependencies.txt
index 9411a44bd..22d342d4c 100755
--- a/tools/dependencies/known-dependencies.txt
+++ b/tools/dependencies/known-dependencies.txt
@@ -2,6 +2,7 @@ HdrHistogram-2.1.9.jar
RoaringBitmap-0.5.11.jar
RoaringBitmap-0.9.0.jar
RoaringBitmap-0.9.22.jar
+ST4-4.0.4.jar
SparseBitSet-1.2.jar
accessors-smart-1.2.jar
activation-1.1.1.jar
@@ -18,12 +19,16 @@ aggs-matrix-stats-client-7.5.1.jar
aircompressor-0.10.jar
airline-0.7.jar
annotations-13.0.jar
+ant-1.6.5.jar
+ant-1.9.1.jar
+ant-launcher-1.9.1.jar
antlr-runtime-3.5.2.jar
antlr4-runtime-4.5.1.jar
antlr4-runtime-4.7.jar
aopalliance-1.0.jar
aopalliance-repackaged-2.4.0-b34.jar
aopalliance-repackaged-2.5.0-b32.jar
+apache-curator-2.7.1.pom
apacheds-i18n-2.0.0-M15.jar
apacheds-kerberos-codec-2.0.0-M15.jar
api-asn1-api-1.0.0-M20.jar
@@ -34,7 +39,9 @@ asm-5.0.4.jar
asm-7.1.jar
asm-all-5.0.2.jar
asm-analysis-7.1.jar
+asm-commons-3.1.jar
asm-commons-7.1.jar
+asm-tree-3.1.jar
asm-tree-7.1.jar
async-http-client-2.5.3.jar
async-http-client-netty-utils-2.5.3.jar
@@ -70,15 +77,14 @@ clickhouse-http-client-0.3.2-patch9-shaded.jar
clickhouse-http-client-0.3.2-patch9.jar
clickhouse-jdbc-0.2.jar
clickhouse-jdbc-0.3.2-patch9.jar
-commons-beanutils-1.7.0.jar
commons-beanutils-1.9.3.jar
-commons-beanutils-core-1.8.0.jar
commons-cli-1.2.jar
commons-cli-1.3.1.jar
commons-cli-1.4.jar
commons-codec-1.13.jar
commons-collections-3.2.2.jar
commons-collections4-4.4.jar
+commons-compiler-2.7.6.jar
commons-compiler-3.0.9.jar
commons-compiler-3.1.6.jar
commons-compress-1.18.jar
@@ -87,7 +93,7 @@ commons-compress-1.20.jar
commons-compress-1.21.jar
commons-compress-1.4.1.jar
commons-compress-1.8.1.jar
-commons-configuration-1.6.jar
+commons-compress-1.9.jar
commons-configuration-1.7.jar
commons-configuration2-2.1.1.jar
commons-crypto-1.0.0.jar
@@ -95,7 +101,7 @@ commons-csv-1.0.jar
commons-daemon-1.0.13.jar
commons-dbcp2-2.0.1.jar
commons-digester-1.8.1.jar
-commons-digester-1.8.jar
+commons-el-1.0.jar
commons-email-1.5.jar
commons-httpclient-3.1.jar
commons-io-2.11.0.jar
@@ -103,7 +109,6 @@ commons-io-2.4.jar
commons-io-2.5.jar
commons-lang-2.6.jar
commons-lang3-3.4.jar
-commons-logging-1.1.3.jar
commons-logging-1.2.jar
commons-math3-3.1.1.jar
commons-math3-3.4.1.jar
@@ -121,18 +126,12 @@ config-1.3.3.jar
config-magic-0.9.jar
converter-moshi-2.9.0.jar
cron-scheduler-0.1.jar
-curator-client-2.12.0.jar
-curator-client-2.6.0.jar
-curator-client-2.7.1.jar
curator-client-4.3.0.jar
-curator-framework-2.12.0.jar
-curator-framework-2.6.0.jar
curator-framework-4.3.0.jar
-curator-recipes-2.12.0.jar
-curator-recipes-2.6.0.jar
curator-recipes-4.3.0.jar
curator-x-discovery-4.3.0.jar
curvesapi-1.06.jar
+datanucleus-core-4.1.17.jar
datasketches-java-2.0.0.jar
datasketches-memory-1.3.0.jar
derby-10.14.2.0.jar
@@ -148,6 +147,7 @@ druid-indexing-hadoop-0.22.1.jar
druid-indexing-service-0.22.1.jar
druid-processing-0.22.1.jar
druid-server-0.22.1.jar
+eigenbase-properties-1.1.5.jar
elasticsearch-6.3.1.jar
elasticsearch-7.5.1.jar
elasticsearch-cli-6.3.1.jar
@@ -190,10 +190,14 @@ flink-shaded-hadoop-2-2.7.5-7.0.jar
flink-statebackend-rocksdb_2.11-1.13.6.jar
force-shading-1.13.6.jar
frocksdbjni-5.17.2-ververica-2.1.jar
+geronimo-annotation_1.0_spec-1.1.1.jar
+geronimo-jaspic_1.0_spec-1.0.jar
+geronimo-jta_1.1_spec-1.1.1.jar
google-api-client-1.26.0.jar
google-http-client-1.26.0.jar
google-http-client-jackson2-1.26.0.jar
google-oauth-client-1.26.0.jar
+groovy-all-2.4.4.jar
gson-2.2.4.jar
gson-2.9.0.jar
guava-19.0.jar
@@ -272,8 +276,21 @@ hbase-zookeeper-2.1.0.jar
hibernate-validator-5.2.5.Final.jar
hive-beeline-1.2.1.spark2.jar
hive-cli-1.2.1.spark2.jar
+hive-common-2.3.9.jar
+hive-exec-2.3.9.jar
hive-jdbc-1.2.1.spark2.jar
+hive-llap-client-2.3.9.jar
+hive-llap-common-2.3.9.jar
+hive-llap-tez-2.3.9.jar
+hive-serde-2.3.9.jar
+hive-service-rpc-2.3.9.jar
+hive-shims-0.23-2.3.9.jar
+hive-shims-2.3.9.jar
+hive-shims-common-2.3.9.jar
+hive-shims-scheduler-2.3.9.jar
+hive-storage-api-2.4.0.jar
hive-storage-api-2.6.0.jar
+hive-vector-code-gen-2.3.9.jar
hk2-api-2.4.0-b34.jar
hk2-api-2.5.0-b32.jar
hk2-locator-2.4.0-b34.jar
@@ -304,11 +321,11 @@ icu4j-charset-60.2.jar
icu4j-localespi-60.2.jar
influxdb-java-2.22.jar
ion-java-1.0.2.jar
+ivy-2.4.0.jar
j2objc-annotations-1.1.jar
jackson-annotations-2.12.6.jar
jackson-core-2.12.6.jar
jackson-core-asl-1.9.13.jar
-jackson-core-asl-1.9.2.jar
jackson-databind-2.12.6.jar
jackson-dataformat-cbor-2.12.3.jar
jackson-dataformat-cbor-2.8.10.jar
@@ -322,7 +339,6 @@ jackson-dataformat-yaml-2.8.11.jar
jackson-datatype-guava-2.10.5.jar
jackson-datatype-joda-2.10.5.jar
jackson-jaxrs-1.9.13.jar
-jackson-jaxrs-1.9.2.jar
jackson-jaxrs-base-2.10.5.jar
jackson-jaxrs-base-2.7.8.jar
jackson-jaxrs-json-provider-2.10.5.jar
@@ -335,15 +351,15 @@ jackson-module-guice-2.10.5.jar
jackson-module-jaxb-annotations-2.10.5.jar
jackson-module-jaxb-annotations-2.7.8.jar
jackson-xc-1.9.13.jar
-jackson-xc-1.9.2.jar
jakarta.activation-api-1.2.1.jar
jakarta.activation-api-1.2.2.jar
jakarta.ws.rs-api-2.1.6.jar
jakarta.xml.bind-api-2.3.2.jar
jakarta.xml.bind-api-2.3.3.jar
jamon-runtime-2.4.1.jar
-janino-3.0.9.jar
janino-3.1.6.jar
+jasper-compiler-5.5.23.jar
+jasper-runtime-5.5.23.jar
java-xmlbuilder-0.4.jar
javassist-3.18.1-GA.jar
javassist-3.20.0-GA.jar
@@ -357,6 +373,7 @@ javax.inject-1.jar
javax.inject-2.4.0-b34.jar
javax.inject-2.5.0-b32.jar
javax.mail-1.5.6.jar
+javax.servlet-3.0.0.v201112011016.jar
javax.servlet-api-3.1.0.jar
javax.servlet.jsp-api-2.3.1.jar
javax.ws.rs-api-2.0.1.jar
@@ -382,28 +399,22 @@ jersey-client-2.22.2.jar
jersey-common-2.22.2.jar
jersey-container-servlet-core-2.22.2.jar
jersey-container-servlet-core-2.25.1.jar
-jersey-core-1.19.3.jar
jersey-core-1.19.jar
-jersey-core-1.9.jar
jersey-guava-2.22.2.jar
jersey-guice-1.19.3.jar
jersey-guice-1.19.jar
jersey-guice-1.9.jar
jersey-json-1.19.jar
-jersey-json-1.9.jar
jersey-media-jaxb-2.22.2.jar
jersey-media-jaxb-2.25.1.jar
-jersey-server-1.19.3.jar
jersey-server-1.19.jar
-jersey-server-1.9.jar
jersey-server-2.22.2.jar
jersey-server-2.25.1.jar
jersey-servlet-1.19.3.jar
jersey-servlet-1.19.jar
-jets3t-0.9.0.jar
jettison-1.1.jar
jettison-1.3.8.jar
-jetty-6.1.26.jar
+jetty-all-7.6.0.v20120127.jar
jetty-client-9.4.40.v20210413.jar
jetty-continuation-9.4.40.v20210413.jar
jetty-http-9.3.19.v20170502.jar
@@ -419,6 +430,8 @@ jetty-servlet-9.3.19.v20170502.jar
jetty-servlet-9.4.40.v20210413.jar
jetty-servlets-9.4.40.v20210413.jar
jetty-sslengine-6.1.26.jar
+jetty-6.1.26.jar
+jets3t-0.9.0.jar
jetty-util-6.1.26.jar
jetty-util-9.3.19.v20170502.jar
jetty-util-9.4.40.v20210413.jar
@@ -435,6 +448,7 @@ jna-4.5.1.jar
joda-time-1.6.jar
joda-time-2.10.3.jar
joda-time-2.10.5.jar
+joda-time-2.8.1.jar
joda-time-2.9.9.jar
joni-2.1.11.jar
joni-2.1.2.jar
@@ -442,11 +456,14 @@ joni-2.1.27.jar
jopt-simple-5.0.2.jar
jpam-1.1.jar
jsch-0.1.54.jar
+json-1.8.jar
json-path-2.3.0.jar
json-smart-2.3.jar
+jsp-api-2.0.jar
jsp-api-2.1.jar
jsr305-1.3.9.jar
jsr305-2.0.1.jar
+jsr305-3.0.0.jar
jsr311-api-1.1.1.jar
jul-to-slf4j-1.7.25.jar
jvm-attach-api-1.5.jar
@@ -476,7 +493,9 @@ kudu-spark2_2.11-1.7.0.jar
lang-mustache-client-6.3.1.jar
lang-mustache-client-7.5.1.jar
leveldbjni-all-1.8.jar
+libfb303-0.9.3.jar
libthrift-0.9.0.jar
+libthrift-0.9.3.jar
log4j-1.2-api-2.15.0.jar
log4j-1.2.14.jar
log4j-1.2.17.jar
@@ -524,6 +543,7 @@ lz4-java-1.4.1.jar
lz4-java-1.6.0.jar
lz4-java-1.7.1.jar
lz4-java-1.8.0.jar
+mail-1.4.1.jar
mapper-extras-client-7.5.1.jar
maven-aether-provider-3.1.1.jar
maven-artifact-3.6.0.jar
@@ -533,9 +553,12 @@ maven-repository-metadata-3.1.1.jar
maven-settings-3.1.1.jar
maven-settings-builder-3.1.1.jar
memory-0.9.0.jar
+metrics-core-3.1.0.jar
metrics-core-3.1.5.jar
metrics-core-3.2.1.jar
metrics-core-4.0.0.jar
+metrics-json-3.1.0.jar
+metrics-jvm-3.1.0.jar
minlog-1.3.0.jar
mongo-java-driver-3.4.2.jar
mongo-spark-connector_2.11-2.2.0.jar
@@ -585,10 +608,12 @@ okhttp-2.4.0.jar
okhttp-4.9.1.jar
okio-1.4.0.jar
okio-2.8.0.jar
+opencsv-2.3.jar
opencsv-4.6.jar
orc-core-1.5.6.jar
orc-shims-1.5.2.jar
orc-shims-1.5.6.jar
+oro-2.0.8.jar
osgi-resource-locator-1.0.1.jar
paranamer-2.3.jar
paranamer-2.7.jar
@@ -605,6 +630,7 @@ parquet-format-2.4.0.jar
parquet-format-structures-1.11.1.jar
parquet-hadoop-1.10.0.jar
parquet-hadoop-1.11.1.jar
+parquet-hadoop-bundle-1.8.1.jar
parquet-jackson-1.10.0.jar
parquet-jackson-1.11.1.jar
percolator-client-6.3.1.jar
@@ -668,6 +694,7 @@ sqlline-1.2.0.jar
sshd-common-2.7.0.jar
sshd-core-2.7.0.jar
sshd-scp-2.7.0.jar
+stax-api-1.0.1.jar
stax-api-1.0-2.jar
stax2-api-3.1.4.jar
stream-2.7.0.jar
@@ -697,6 +724,7 @@ uzaygezen-core-0.2.jar
validation-api-1.1.0.Final.jar
vavr-0.10.2.jar
vavr-match-0.10.2.jar
+velocity-1.5.jar
wagon-provider-api-2.4.jar
woodstox-core-5.0.3.jar
xbean-asm6-shaded-4.10.jar