From 2902241b5044b3cbd7c5da6fef25ac371f896fc0 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Mon, 4 Sep 2023 22:38:00 +0800 Subject: [PATCH 01/58] :sparkles: "Added Hive support to the JDBC connector This commit includes the addition of Hive JDBC dependencies in the pom.xml file to enable the JDBC connector support Hive. The Hive dialect, type mapper and row converter were added accordingly. This update allows users to connect and interact with Hive directly using the JDBC connector." --- seatunnel-connectors-v2/connector-jdbc/pom.xml | 11 +++++++++++ .../jdbc/exception/JdbcConnectorErrorCode.java | 4 ++-- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/pom.xml b/seatunnel-connectors-v2/connector-jdbc/pom.xml index e76237e7e07..b3faf6b8200 100644 --- a/seatunnel-connectors-v2/connector-jdbc/pom.xml +++ b/seatunnel-connectors-v2/connector-jdbc/pom.xml @@ -46,6 +46,7 @@ 3.13.29 12.0.3-0 2.5.1 + 3.1.2 @@ -143,6 +144,12 @@ ${vertica.version} provided + + org.apache.hive + hive-jdbc + ${hive.jdbc.version} + provided + @@ -218,5 +225,9 @@ com.vertica.jdbc vertica-jdbc + + org.apache.hive + hive-jdbc + diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/exception/JdbcConnectorErrorCode.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/exception/JdbcConnectorErrorCode.java index 3d53b102bdc..5ebde65d19c 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/exception/JdbcConnectorErrorCode.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/exception/JdbcConnectorErrorCode.java @@ -27,8 +27,8 @@ public enum JdbcConnectorErrorCode implements SeaTunnelErrorCode { CONNECT_DATABASE_FAILED("JDBC-04", "Connector database failed"), TRANSACTION_OPERATION_FAILED( "JDBC-05", "transaction operation failed, such as (commit, rollback) etc.."), - NO_SUITABLE_DIALECT_FACTORY("JDBC-06", "No suitable dialect factory found"); - + NO_SUITABLE_DIALECT_FACTORY("JDBC-06", "No suitable dialect factory found"), + DONT_SUPPORT_SINK("JDBC-07", "The jdbc type don't support sink"); private final String code; private final String description; From 55f9a07f3237985d37aa387c3f4cbcf2fd96f96f Mon Sep 17 00:00:00 2001 From: yangpeng Date: Mon, 4 Sep 2023 22:38:55 +0800 Subject: [PATCH 02/58] :sparkles: "Added Hive support to the JDBC connector This commit includes the addition of Hive JDBC dependencies in the pom.xml file to enable the JDBC connector support Hive. The Hive dialect, type mapper and row converter were added accordingly. This update allows users to connect and interact with Hive directly using the JDBC connector." --- .../internal/dialect/hive/HiveDialect.java | 60 +++++++++ .../dialect/hive/HiveDialectFactory.java | 38 ++++++ .../dialect/hive/HiveJdbcRowConverter.java | 42 ++++++ .../internal/dialect/hive/HiveTypeMapper.java | 120 ++++++++++++++++++ 4 files changed, 260 insertions(+) create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialect.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialectFactory.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcRowConverter.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveTypeMapper.java diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialect.java new file mode 100644 index 00000000000..acb60004bcb --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialect.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.hive; + +import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.converter.JdbcRowConverter; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectTypeMapper; + +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.util.Optional; + +public class HiveDialect implements JdbcDialect { + + @Override + public String dialectName() { + return "HIVE"; + } + + @Override + public JdbcRowConverter getRowConverter() { + return new HiveJdbcRowConverter(); + } + + @Override + public JdbcDialectTypeMapper getJdbcDialectTypeMapper() { + return new HiveTypeMapper(); + } + + @Override + public Optional getUpsertStatement( + String database, String tableName, String[] fieldNames, String[] uniqueKeyFields) { + return Optional.empty(); + } + + @Override + public ResultSetMetaData getResultSetMetaData( + Connection conn, JdbcSourceConfig jdbcSourceConfig) throws SQLException { + PreparedStatement ps = conn.prepareStatement(jdbcSourceConfig.getQuery()); + return ps.executeQuery().getMetaData(); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialectFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialectFactory.java new file mode 100644 index 00000000000..56bd81b7f83 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialectFactory.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.hive; + +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectFactory; + +import com.google.auto.service.AutoService; + +/** Factory for {@link HiveDialect}. */ +@AutoService(JdbcDialectFactory.class) +public class HiveDialectFactory implements JdbcDialectFactory { + + @Override + public boolean acceptsURL(String url) { + return url.startsWith("jdbc:hive2:"); + } + + @Override + public JdbcDialect create() { + return new HiveDialect(); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcRowConverter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcRowConverter.java new file mode 100644 index 00000000000..5517caa9623 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcRowConverter.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.jdbc.internal.dialect.hive; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorErrorCode; +import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorException; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.converter.AbstractJdbcRowConverter; + +import java.sql.PreparedStatement; + +public class HiveJdbcRowConverter extends AbstractJdbcRowConverter { + + @Override + public String converterName() { + return "Hive"; + } + + @Override + public PreparedStatement toExternal( + SeaTunnelRowType rowType, SeaTunnelRow row, PreparedStatement statement) { + throw new JdbcConnectorException( + JdbcConnectorErrorCode.DONT_SUPPORT_SINK, + "The Hive jdbc connector don't support sink"); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveTypeMapper.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveTypeMapper.java new file mode 100644 index 00000000000..8415986d897 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveTypeMapper.java @@ -0,0 +1,120 @@ +/* + * 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.jdbc.internal.dialect.hive; + +import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.api.table.type.DecimalType; +import org.apache.seatunnel.api.table.type.LocalTimeType; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.common.exception.CommonErrorCode; +import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorException; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectTypeMapper; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.ResultSetMetaData; +import java.sql.SQLException; + +public class HiveTypeMapper implements JdbcDialectTypeMapper { + + private static final Logger LOG = LoggerFactory.getLogger(HiveTypeMapper.class); + + // reference https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Types + + // Numeric Types + private static final String HIVE_TINYINT = "TINYINT"; + private static final String HIVE_SMALLINT = "SMALLINT"; + private static final String HIVE_INT = "INT"; + private static final String HIVE_INTEGER = "INTEGER"; + private static final String HIVE_BIGINT = "BIGINT"; + private static final String HIVE_FLOAT = "FLOAT"; + private static final String HIVE_DOUBLE = "DOUBLE"; + private static final String HIVE_DOUBLE_PRECISION = "DOUBLE PRECISION"; + private static final String HIVE_DECIMAL = "DECIMAL"; + private static final String HIVE_NUMERIC = "NUMERIC"; + // Date/Time Types + private static final String HIVE_TIMESTAMP = "TIMESTAMP"; + private static final String HIVE_DATE = "DATE"; + private static final String HIVE_INTERVAL = "INTERVAL"; + // String Types + private static final String HIVE_STRING = "STRING"; + private static final String HIVE_VARCHAR = "VARCHAR"; + private static final String HIVE_CHAR = "CHAR"; + // Misc Types + private static final String HIVE_BOOLEAN = "BOOLEAN"; + private static final String HIVE_BINARY = "BINARY"; + // Complex Types + private static final String HIVE_ARRAY = "ARRAY"; + private static final String HIVE_MAP = "MAP"; + // float + private static final String HIVE_STRUCT = "STRUCT"; + private static final String HIVE_UNIONTYPE = "UNIONTYPE"; + + @Override + public SeaTunnelDataType mapping(ResultSetMetaData metadata, int colIndex) + throws SQLException { + String columnType = metadata.getColumnTypeName(colIndex).toUpperCase(); + int precision = metadata.getPrecision(colIndex); + switch (columnType) { + case HIVE_TINYINT: + case HIVE_SMALLINT: + return BasicType.SHORT_TYPE; + case HIVE_INT: + case HIVE_INTEGER: + return BasicType.INT_TYPE; + case HIVE_BIGINT: + return BasicType.LONG_TYPE; + case HIVE_FLOAT: + return BasicType.FLOAT_TYPE; + case HIVE_DOUBLE: + case HIVE_DOUBLE_PRECISION: + return BasicType.DOUBLE_TYPE; + case HIVE_DECIMAL: + case HIVE_NUMERIC: + if (precision > 0) { + return new DecimalType(precision, metadata.getScale(colIndex)); + } + LOG.warn("decimal did define precision,scale, will be Decimal(38,18)"); + return new DecimalType(38, 18); + case HIVE_TIMESTAMP: + return LocalTimeType.LOCAL_DATE_TIME_TYPE; + case HIVE_DATE: + return LocalTimeType.LOCAL_DATE_TYPE; + case HIVE_STRING: + case HIVE_VARCHAR: + case HIVE_CHAR: + case HIVE_BINARY: + return BasicType.STRING_TYPE; + case HIVE_BOOLEAN: + return BasicType.BOOLEAN_TYPE; + case HIVE_ARRAY: + case HIVE_INTERVAL: + case HIVE_MAP: + case HIVE_STRUCT: + case HIVE_UNIONTYPE: + default: + final String jdbcColumnName = metadata.getColumnName(colIndex); + throw new JdbcConnectorException( + CommonErrorCode.UNSUPPORTED_OPERATION, + String.format( + "Doesn't support hive type '%s' on column '%s' yet.", + columnType, jdbcColumnName)); + } + } +} From 5072173a027fd1bbf7ed03906e5c8043e1f36b81 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Mon, 4 Sep 2023 23:05:10 +0800 Subject: [PATCH 03/58] :art: Refactor HiveTypeMapper for JDBC connector --- .../seatunnel/jdbc/internal/dialect/hive/HiveTypeMapper.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveTypeMapper.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveTypeMapper.java index 8415986d897..17a6db9d35e 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveTypeMapper.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveTypeMapper.java @@ -99,10 +99,10 @@ public SeaTunnelDataType mapping(ResultSetMetaData metadata, int colIndex) case HIVE_STRING: case HIVE_VARCHAR: case HIVE_CHAR: - case HIVE_BINARY: return BasicType.STRING_TYPE; case HIVE_BOOLEAN: return BasicType.BOOLEAN_TYPE; + case HIVE_BINARY: case HIVE_ARRAY: case HIVE_INTERVAL: case HIVE_MAP: From 255cbbc1de3abadfc84f20052fde7edfd9b546a5 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Wed, 6 Sep 2023 21:17:58 +0800 Subject: [PATCH 04/58] :sparkles: Add Hive JDBC source connector documentation This commit introduces a new documentation file under `/docs/en/connector-v2/source/` explaining how to use the Hive JDBC source connector. The included information covers the supported engines, key features, support DataSource info, data type mapping, source options and database dependency, along with examples for how to use the connector with different configurations. This documentation is intended to help users to understand and effectively use the Hive JDBC source connector in their data pipeline. --- docs/en/connector-v2/source/Hive-jdbc.md | 156 +++++++++++++++++++++++ 1 file changed, 156 insertions(+) create mode 100644 docs/en/connector-v2/source/Hive-jdbc.md diff --git a/docs/en/connector-v2/source/Hive-jdbc.md b/docs/en/connector-v2/source/Hive-jdbc.md new file mode 100644 index 00000000000..0e3352014c2 --- /dev/null +++ b/docs/en/connector-v2/source/Hive-jdbc.md @@ -0,0 +1,156 @@ +# Hive + +> JDBC Hive Source Connector + +## Support Those Engines + +> Spark
+> Flink
+> SeaTunnel Zeta
+ +## Key Features + +- [x] [batch](../../concept/connector-v2-features.md) +- [ ] [stream](../../concept/connector-v2-features.md) +- [ ] [exactly-once](../../concept/connector-v2-features.md) +- [x] [column projection](../../concept/connector-v2-features.md) +- [x] [parallelism](../../concept/connector-v2-features.md) +- [x] [support user-defined split](../../concept/connector-v2-features.md) + +> supports query SQL and can achieve projection effect. + +## Description + +Read external data source data through JDBC. + +## Supported DataSource Info + +| Datasource | Supported versions | Driver | Url | Maven | +|------------|----------------------------------------------------------|---------------------------------|--------------------------------------|--------------------------------------------------------------------------| +| Hive | Different dependency version has different driver class. | org.apache.hive.jdbc.HiveDriver | jdbc:hive2://localhost:10000/default | [Download](https://mvnrepository.com/artifact/org.apache.hive/hive-jdbc) | + +## Database Dependency + +> Please download the support list corresponding to 'Maven' and copy it to the '$SEATNUNNEL_HOME/plugins/jdbc/lib/' +> working directory
+> For example Hive datasource: cp hive-jdbc-xxx.jar $SEATNUNNEL_HOME/plugins/jdbc/lib/ + +## Data Type Mapping + +| Hive Data type | SeaTunnel Data type | +|-------------------------------------------------------------------------------------------|---------------------| +| BOOLEAN | BOOLEAN | +| TINYINT
SMALLINT | SHORT | +| INT
INTEGER | INT | +| BIGINT | LONG | +| FLOAT | FLOAT | +| DOUBLE
DOUBLE PRECISION | DOUBLE | +| DECIMAL(x,y)
NUMERIC(x,y)
(Get the designated column's specified column size.<38) | DECIMAL(x,y) | +| DECIMAL(x,y)
NUMERIC(x,y)
(Get the designated column's specified column size.>38) | DECIMAL(38,18) | +| CHAR
VARCHAR
STRING | STRING | +| DATE | DATE | +| DATETIME
TIMESTAMP | TIMESTAMP | +| BINARY
ARRAY
INTERVAL
MAP
STRUCT
UNIONTYPE | Not supported yet | + +## Source Options + +| Name | Type | Required | Default | Description | +|------------------------------|------------|----------|-----------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:hive2://localhost:10000/default | +| driver | String | Yes | - | The jdbc class name used to connect to the remote data source,
if you use Hive the value is `org.apache.hive.jdbc.HiveDriver`. | +| user | String | No | - | Connection instance user name | +| password | String | No | - | Connection instance password | +| query | String | Yes | - | Query statement | +| connection_check_timeout_sec | Int | No | 30 | The time in seconds to wait for the database operation used to validate the connection to complete | +| partition_column | String | No | - | The column name for parallelism's partition, only support numeric type,Only support numeric type primary key, and only can config one column. | +| partition_lower_bound | BigDecimal | No | - | The partition_column min value for scan, if not set SeaTunnel will query database get min value. | +| partition_upper_bound | BigDecimal | No | - | The partition_column max value for scan, if not set SeaTunnel will query database get max value. | +| partition_num | Int | No | job parallelism | The number of partition count, only support positive integer. default value is job parallelism | +| fetch_size | Int | No | 0 | For queries that return a large number of objects,you can configure
the row fetch size used in the query toimprove performance by
reducing the number database hits required to satisfy the selection criteria.
Zero means use jdbc default value. | +| common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details | + +### Tips + +> If partition_column is not set, it will run in single concurrency, and if partition_column is set, it will be executed +> in parallel according to the concurrency of tasks , When your shard read field is a large number type such as bigint( +> and above and the data is not evenly distributed, it is recommended to set the parallelism level to 1 to ensure that +> the +> data skew problem is resolved + +## Task Example + +### Simple: + +> This example queries type_bin 'table' 16 data in your test "database" in single parallel and queries all of its +> fields. You can also specify which fields to query for final output to the console. + +``` +# Defining the runtime environment +env { + # You can set flink configuration here + execution.parallelism = 2 + job.mode = "BATCH" +} +source{ + Jdbc { + url = "jdbc:hive2://localhost:10000/default" + driver = "org.apache.hive.jdbc.HiveDriver" + connection_check_timeout_sec = 100 + query = "select * from type_bin limit 16" + } +} + +transform { + # If you would like to get more information about how to configure seatunnel and see full list of transform plugins, + # please go to https://seatunnel.apache.org/docs/transform-v2/sql +} + +sink { + Console {} +} +``` + +### Parallel: + +> Read your query table in parallel with the shard field you configured and the shard data You can do this if you want +> to read the whole table + +``` +source { + Jdbc { + url = "jdbc:hive2://localhost:10000/default" + driver = "org.apache.hive.jdbc.HiveDriver" + connection_check_timeout_sec = 100 + # Define query logic as required + query = "select * from type_bin" + # Parallel sharding reads fields + partition_column = "id" + # Number of fragments + partition_num = 10 + } +} +``` + +### Parallel Boundary: + +> It is more efficient to specify the data within the upper and lower bounds of the query It is more efficient to read +> your data source according to the upper and lower boundaries you configured + +``` +source { + Jdbc { + url = "jdbc:hive2://localhost:10000/default" + driver = "org.apache.hive.jdbc.HiveDriver" + connection_check_timeout_sec = 100 + # Define query logic as required + query = "select * from type_bin" + partition_column = "id" + # Read start boundary + partition_lower_bound = 1 + # Read end boundary + partition_upper_bound = 500 + partition_num = 10 + } +} +``` + From 4828e7a9904ac210b659ef2f41e0e6ddd4e09650 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Thu, 7 Sep 2023 01:55:43 +0800 Subject: [PATCH 05/58] :sparkles: Add JDBC Hive IT and source config to SeaTunnel project Introduces JDBC Hive Integration Tests and new configuration settings for the 'connector-jdbc-e2e' module in the SeaTunnel v2 project. These additions are designed to enhance testing and configuration handling capabilities for connections to Hive databases. The new JdbcHiveIT class implements Hive-specific configurations and integration test cases. The jdbc_hive_source.conf file contains a sample configuration setup for a Hive JDBC source connection. --- .../connectors/seatunnel/jdbc/JdbcHiveIT.java | 168 ++++++++++++++++++ .../src/test/resources/jdbc_hive_source.conf | 46 +++++ 2 files changed, 214 insertions(+) create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_hive_source.conf diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java new file mode 100644 index 00000000000..724b999e890 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java @@ -0,0 +1,168 @@ +/* + * 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.jdbc; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; +import org.apache.seatunnel.common.utils.ExceptionUtils; + +import org.apache.commons.lang3.tuple.Pair; + +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.utility.DockerLoggerFactory; + +import com.google.common.collect.Lists; +import lombok.extern.slf4j.Slf4j; + +import java.sql.Statement; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +@Slf4j +public class JdbcHiveIT extends AbstractJdbcIT { + + private static final String HIVE_IMAGE = "youngyangp/hive_3.1.2_arm:1.0.0"; + private static final String HIVE_CONTAINER_HOST = "localhost"; + + private static final String HIVE_DATABASE = "default"; + + private static final String HIVE_SOURCE = "e2e_table_source"; + private static final String HIVE_USERNAME = "root"; + private static final String HIVE_PASSWORD = null; + private static final int HIVE_PORT = 10000; + private static final String HIVE_URL = "jdbc:hive2://" + HIVE_CONTAINER_HOST + ":%s/%s"; + + private static final String DRIVER_CLASS = "org.apache.hive.jdbc.HiveDriver"; + + private static final List CONFIG_FILE = + Lists.newArrayList("/jdbc_HIVE_source_and_sink.conf"); + private static final String CREATE_SQL = + "CREATE TABLE hive_e2e_source_table" + + "(" + + " int_column INT," + + " integer_column INTEGER," + + " bigint_column BIGINT," + + " smallint_column SMALLINT," + + " tinyint_column TINYINT," + + " double_column DOUBLE," + + " double_PRECISION_column DOUBLE PRECISION," + + " float_column FLOAT," + + " string_column STRING," + + " char_column CHAR(10)," + + " varchar_column VARCHAR(20)," + + " boolean_column BOOLEAN," + + " date_column DATE," + + " timestamp_column TIMESTAMP," + + " decimal_column DECIMAL(10, 2)," + + " numeric_column NUMERIC(10, 2)" + + ")"; + + @Override + JdbcCase getJdbcCase() { + Map containerEnv = new HashMap<>(); + String jdbcUrl = String.format(HIVE_URL, HIVE_PORT, HIVE_DATABASE); + return JdbcCase.builder() + .dockerImage(HIVE_IMAGE) + .networkAliases(HIVE_CONTAINER_HOST) + .containerEnv(containerEnv) + .driverClass(DRIVER_CLASS) + .host(HOST) + .port(HIVE_PORT) + .localPort(HIVE_PORT) + .jdbcTemplate(HIVE_URL) + .jdbcUrl(jdbcUrl) + .userName(HIVE_USERNAME) + .password(HIVE_PASSWORD) + .database(HIVE_DATABASE) + .sourceTable(HIVE_SOURCE) + .createSql(CREATE_SQL) + .configFile(CONFIG_FILE) + .build(); + } + + protected void createNeededTables() { + try (Statement statement = connection.createStatement()) { + String createTemplate = jdbcCase.getCreateSql(); + String createSource = + String.format( + createTemplate, + buildTableInfoWithSchema( + jdbcCase.getDatabase(), jdbcCase.getSourceTable())); + statement.execute(createSource); + } catch (Exception exception) { + log.error(ExceptionUtils.getMessage(exception)); + throw new SeaTunnelRuntimeException(JdbcITErrorCode.CREATE_TABLE_FAILED, exception); + } + } + + protected void insertTestData() { + try (Statement statement = connection.createStatement()) { + for (int i = 1; i <= 3; i++) { + statement.execute( + "INSERT INTO hive_e2e_source_table " + + "VALUES (2," + + " 1," + + " 1234567890," + + " 32767," + + " 127," + + " 123.45," + + " 123.45," + + " 67.89," + + " 'Hello, Hive'," + + " 'CharCol'," + + " 'VarcharCol'," + + " TRUE," + + " '2023-09-04'," + + " '2023-09-04 10:30:00'," + + " 42.12," + + " 42.12)"); + } + } catch (Exception exception) { + log.error(ExceptionUtils.getMessage(exception)); + throw new SeaTunnelRuntimeException(JdbcITErrorCode.INSERT_DATA_FAILED, exception); + } + } + + @Override + void compareResult() {} + + @Override + String driverUrl() { + return "https://repo1.maven.org/maven2/org/apache/hive/hive-jdbc/3.1.2/hive-jdbc-3.1.2.jar"; + } + + @Override + Pair> initTestData() { + return null; + } + + @Override + GenericContainer initContainer() { + GenericContainer container = + new GenericContainer<>(HIVE_IMAGE) + .withExposedPorts(HIVE_PORT) + .withNetwork(NETWORK) + .withNetworkAliases(HIVE_CONTAINER_HOST) + .withLogConsumer( + new Slf4jLogConsumer(DockerLoggerFactory.getLogger(HIVE_IMAGE))); + container.setPortBindings(Lists.newArrayList(String.format("%s:%s", HIVE_PORT, HIVE_PORT))); + return container; + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_hive_source.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_hive_source.conf new file mode 100644 index 00000000000..abfd0a9b938 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_hive_source.conf @@ -0,0 +1,46 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + # You can set engine configuration here + execution.parallelism = 1 + job.mode = "BATCH" + checkpoint.interval = 5000 + #execution.checkpoint.data-uri = "hdfs://localhost:9000/checkpoint" +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + Jdbc { + url = "jdbc:hive2://localhost:10000/default" + user = "root" + driver = "org.apache.hive.jdbc.HiveDriver" + query = "select * from hive_e2e_source_table" + } +} + +transform { +} + +sink { + console { + source_table_name="fake" + } +} From d4ddd38671732aaec58016cfeb26ccc35b626098 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Thu, 7 Sep 2023 02:03:55 +0800 Subject: [PATCH 06/58] :fire: Refactored JDBC configuration filename in e2e-test Fixed the case sensitivity issue in jdbc config filename in e2e-test. Changed "/jdbc_HIVE_source_and_sink.conf" to "/jdbc_hive_source_and_sink.conf". Linux servers are case sensitive hence the filename should match exact case to avoid file not found exceptions while running end-to-end testing. --- .../apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java index 724b999e890..4919ba91798 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java @@ -52,7 +52,7 @@ public class JdbcHiveIT extends AbstractJdbcIT { private static final String DRIVER_CLASS = "org.apache.hive.jdbc.HiveDriver"; private static final List CONFIG_FILE = - Lists.newArrayList("/jdbc_HIVE_source_and_sink.conf"); + Lists.newArrayList("/jdbc_hive_source_and_sink.conf"); private static final String CREATE_SQL = "CREATE TABLE hive_e2e_source_table" + "(" From ab7125bd0d45446aad05c691f8ffdda149a681f3 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Fri, 8 Sep 2023 17:50:35 +0800 Subject: [PATCH 07/58] :art: Add BYTE_TYPE mapping in HiveTypeMapper This commit adds a BYTE_TYPE mapping to the HiveTypeMapper class for the HIVE_TINYINT field. This change allows mapping of HIVE_TINYINT to BasicType.BYTE_TYPE, ensuring type consistency. --- .../seatunnel/jdbc/internal/dialect/hive/HiveTypeMapper.java | 1 + 1 file changed, 1 insertion(+) diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveTypeMapper.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveTypeMapper.java index 17a6db9d35e..08be422ec24 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveTypeMapper.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveTypeMapper.java @@ -73,6 +73,7 @@ public SeaTunnelDataType mapping(ResultSetMetaData metadata, int colIndex) int precision = metadata.getPrecision(colIndex); switch (columnType) { case HIVE_TINYINT: + return BasicType.BYTE_TYPE; case HIVE_SMALLINT: return BasicType.SHORT_TYPE; case HIVE_INT: From 7bc4992be15d5749e4e61d31af94eb46cc8f36d6 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Thu, 14 Sep 2023 23:20:41 +0800 Subject: [PATCH 08/58] :art: Refactor JDBC connector tests for Hive Removed redundant file and tidied up pom.xml. Removed the file jdbc_hive_source.conf, added needed dependencies for Hive in pom.xml, and removed an unused module from connector-file pom. Improved tests by updating test configuration to better demonstrate features and modified corresponding test class accordingly. Changes aim to enhance test readability and correctness. --- .../connector-jdbc/pom.xml | 20 +++ .../connector-jdbc-e2e-part-3/pom.xml | 10 ++ .../connectors/seatunnel/jdbc/JdbcHiveIT.java | 10 +- .../src/test/resources/jdbc_hive_source.conf | 46 ------ .../jdbc_hive_source_and_assert.conf | 140 ++++++++++++++++++ 5 files changed, 177 insertions(+), 49 deletions(-) delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_hive_source.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_hive_source_and_assert.conf diff --git a/seatunnel-connectors-v2/connector-jdbc/pom.xml b/seatunnel-connectors-v2/connector-jdbc/pom.xml index 242591d6b5f..7893f152951 100644 --- a/seatunnel-connectors-v2/connector-jdbc/pom.xml +++ b/seatunnel-connectors-v2/connector-jdbc/pom.xml @@ -145,6 +145,18 @@ ${vertica.version} provided + + org.apache.hive + hive-jdbc + ${hive.jdbc.version} + provided + + + org.apache.hive + hive-service + ${hive.jdbc.version} + provided + @@ -220,5 +232,13 @@ com.vertica.jdbc vertica-jdbc + + org.apache.hive + hive-jdbc + + + org.apache.hive + hive-service + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml index 8628e2b80b6..efcfa4c949e 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml @@ -91,6 +91,16 @@ vertica-jdbc test + + org.apache.hive + hive-jdbc + test + + + org.apache.hive + hive-service + test + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java index 4919ba91798..4d902715075 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java @@ -39,7 +39,7 @@ public class JdbcHiveIT extends AbstractJdbcIT { private static final String HIVE_IMAGE = "youngyangp/hive_3.1.2_arm:1.0.0"; - private static final String HIVE_CONTAINER_HOST = "localhost"; + private static final String HIVE_CONTAINER_HOST = "e2ehivejdbc"; private static final String HIVE_DATABASE = "default"; @@ -47,12 +47,12 @@ public class JdbcHiveIT extends AbstractJdbcIT { private static final String HIVE_USERNAME = "root"; private static final String HIVE_PASSWORD = null; private static final int HIVE_PORT = 10000; - private static final String HIVE_URL = "jdbc:hive2://" + HIVE_CONTAINER_HOST + ":%s/%s"; + private static final String HIVE_URL = "jdbc:hive2://" + HOST + ":%s/%s"; private static final String DRIVER_CLASS = "org.apache.hive.jdbc.HiveDriver"; private static final List CONFIG_FILE = - Lists.newArrayList("/jdbc_hive_source_and_sink.conf"); + Lists.newArrayList("/jdbc_hive_source_and_assert.conf"); private static final String CREATE_SQL = "CREATE TABLE hive_e2e_source_table" + "(" @@ -165,4 +165,8 @@ GenericContainer initContainer() { container.setPortBindings(Lists.newArrayList(String.format("%s:%s", HIVE_PORT, HIVE_PORT))); return container; } + + public void clearTable(String schema, String table) { + // do nothing. + } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_hive_source.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_hive_source.conf deleted file mode 100644 index abfd0a9b938..00000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_hive_source.conf +++ /dev/null @@ -1,46 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# -###### -###### This config file is a demonstration of streaming processing in seatunnel config -###### - -env { - # You can set engine configuration here - execution.parallelism = 1 - job.mode = "BATCH" - checkpoint.interval = 5000 - #execution.checkpoint.data-uri = "hdfs://localhost:9000/checkpoint" -} - -source { - # This is a example source plugin **only for test and demonstrate the feature source plugin** - Jdbc { - url = "jdbc:hive2://localhost:10000/default" - user = "root" - driver = "org.apache.hive.jdbc.HiveDriver" - query = "select * from hive_e2e_source_table" - } -} - -transform { -} - -sink { - console { - source_table_name="fake" - } -} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_hive_source_and_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_hive_source_and_assert.conf new file mode 100644 index 00000000000..a6b1a062a38 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_hive_source_and_assert.conf @@ -0,0 +1,140 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + # You can set engine configuration here + execution.parallelism = 1 + job.mode = "BATCH" + checkpoint.interval = 5000 + #execution.checkpoint.data-uri = "hdfs://localhost:9000/checkpoint" +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + Jdbc { + url = "jdbc:hive2://e2ehivejdbc:10000/default" + user = "root" + driver = "org.apache.hive.jdbc.HiveDriver" + query = "select * from hive_e2e_source_table" + } +} + +transform { +} + +sink{ + assert { + rules = + { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 3 + }, + { + rule_type = MIN_ROW + rule_value = 3 + } + ], + field_rules = [ + { + field_name = hive_e2e_source_table.int_column + field_type = int + field_value = [{equals_to = 2}] + }, + { + field_name = hive_e2e_source_table.integer_column + field_type = int + field_value = [{equals_to = 1}] + }, + { + field_name = hive_e2e_source_table.bigint_column + field_type = long + field_value = [{equals_to = 1234567890}] + }, + { + field_name = hive_e2e_source_table.smallint_column + field_type = short + field_value = [{equals_to = 32767}] + }, + { + field_name = hive_e2e_source_table.tinyint_column + field_type = byte + field_value = [{equals_to = 127}] + }, + { + field_name = hive_e2e_source_table.double_column + field_type = double + field_value = [{equals_to = 123.45}] + }, + { + field_name = hive_e2e_source_table.double_precision_column + field_type = double + field_value = [{equals_to = 123.45}] + }, + { + field_name = hive_e2e_source_table.float_column + field_type = float + field_value = [{equals_to = 67.89}] + }, + { + field_name = hive_e2e_source_table.string_column + field_type = string + field_value = [{equals_to = "Hello, Hive"}] + }, + { + field_name = hive_e2e_source_table.char_column + field_type = string + field_value = [{equals_to = "CharCol "}] + }, + { + field_name = hive_e2e_source_table.varchar_column + field_type = string + field_value = [{equals_to = "VarcharCol"}] + }, + { + field_name = hive_e2e_source_table.boolean_column + field_type = boolean + field_value = [{equals_to = "TRUE"}] + }, + { + field_name = hive_e2e_source_table.date_column + field_type = date + field_value = [{equals_to = "2023-09-04"}] + }, + { + field_name = hive_e2e_source_table.timestamp_column + field_type = timestamp + field_value = [{equals_to = "2023-09-04T10:30:00"}] + }, + { + field_name = hive_e2e_source_table.decimal_column + field_type = decimal + field_value = [{equals_to = 42.12}] + }, + { + field_name = hive_e2e_source_table.numeric_column + field_type = decimal + field_value = [{equals_to = 42.12}] + }, + ] + } + } +} From 62d4ecfa94bd13bfd14b76f602d8f58e0ffc0003 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Fri, 15 Sep 2023 15:26:11 +0800 Subject: [PATCH 09/58] :art: Update Hive Image and Add Environment Variable Updated the HIVE_IMAGE from "youngyangp/hive_3.1.2_arm:1.0.0" to "apache/hive:3.1.3" in the JdbcHiveIT test. Also, introduced "SERVICE_NAME" as an environment variable to the Hive server container. The HIVE_IMAGE was updated for using an official Apache hive image. The environment variable is crucial for service internal recognition. --- .../apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java index 4d902715075..5a4163e2a23 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java @@ -38,7 +38,7 @@ @Slf4j public class JdbcHiveIT extends AbstractJdbcIT { - private static final String HIVE_IMAGE = "youngyangp/hive_3.1.2_arm:1.0.0"; + private static final String HIVE_IMAGE = "apache/hive:3.1.3"; private static final String HIVE_CONTAINER_HOST = "e2ehivejdbc"; private static final String HIVE_DATABASE = "default"; @@ -160,6 +160,7 @@ GenericContainer initContainer() { .withExposedPorts(HIVE_PORT) .withNetwork(NETWORK) .withNetworkAliases(HIVE_CONTAINER_HOST) + .withEnv("SERVICE_NAME","hiveserver2") .withLogConsumer( new Slf4jLogConsumer(DockerLoggerFactory.getLogger(HIVE_IMAGE))); container.setPortBindings(Lists.newArrayList(String.format("%s:%s", HIVE_PORT, HIVE_PORT))); From 4d31e346de5d030fba40ebc41cd48d30c3d6040d Mon Sep 17 00:00:00 2001 From: yangpeng Date: Fri, 15 Sep 2023 15:28:45 +0800 Subject: [PATCH 10/58] :art: Refactor styling in JdbcHiveIT file Uniformly formatted the environment variables for better code readability. The change is on line 160 from '.withEnv("SERVICE_NAME","hiveserver2")' to '.withEnv("SERVICE_NAME", "hiveserver2")'. The whitespace after file. --- .../apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java index 5a4163e2a23..d1118e3f438 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java @@ -160,7 +160,7 @@ GenericContainer initContainer() { .withExposedPorts(HIVE_PORT) .withNetwork(NETWORK) .withNetworkAliases(HIVE_CONTAINER_HOST) - .withEnv("SERVICE_NAME","hiveserver2") + .withEnv("SERVICE_NAME", "hiveserver2") .withLogConsumer( new Slf4jLogConsumer(DockerLoggerFactory.getLogger(HIVE_IMAGE))); container.setPortBindings(Lists.newArrayList(String.format("%s:%s", HIVE_PORT, HIVE_PORT))); From 069dee26432d2ad68ba697e7f0cb1333ca017d34 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Fri, 15 Sep 2023 16:07:26 +0800 Subject: [PATCH 11/58] Added auto_commit to jdbc_hive config file Added a new parameter, 'auto_commit', to the jdbc_hive configuration file to improve database transaction management. This is pivotal for applications that require a high level of data integrity and consistency, or for those that need to manage complex, multi-step transactions. --- .../src/test/resources/jdbc_hive_source_and_assert.conf | 1 + 1 file changed, 1 insertion(+) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_hive_source_and_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_hive_source_and_assert.conf index a6b1a062a38..04b0240a3bf 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_hive_source_and_assert.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_hive_source_and_assert.conf @@ -33,6 +33,7 @@ source { user = "root" driver = "org.apache.hive.jdbc.HiveDriver" query = "select * from hive_e2e_source_table" + auto_commit= false } } From 008eb267c05b47f7135489baf217fcec98f39ea7 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Fri, 15 Sep 2023 17:50:22 +0800 Subject: [PATCH 12/58] Update hive.jdbc version and remove 'provided' scope Updated hive.jdbc.version to 3.1.3 in connector-jdbc/pom.xml to have access to the latest features, bug fixes and improvements. Also, removed the 'provided' scope from the same file as it is not necessary and its presence could lead to potential issues with the dependency resolution during the build process. --- seatunnel-connectors-v2/connector-jdbc/pom.xml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/pom.xml b/seatunnel-connectors-v2/connector-jdbc/pom.xml index 7893f152951..00a8e715635 100644 --- a/seatunnel-connectors-v2/connector-jdbc/pom.xml +++ b/seatunnel-connectors-v2/connector-jdbc/pom.xml @@ -47,7 +47,7 @@ 12.0.3-0 2.5.1 8.6.0 - 3.1.2 + 3.1.3 @@ -155,7 +155,6 @@ org.apache.hive hive-service ${hive.jdbc.version} - provided From 27bd0018e725738d74f26b7568ef570f1991b03b Mon Sep 17 00:00:00 2001 From: yangpeng Date: Tue, 26 Sep 2023 13:53:28 +0800 Subject: [PATCH 13/58] :sparkles: "Added kerberos support in JDBC connections This commit enables kerberos authentication while establishing JDBC connections. The JdbcOptions class has been updated to include new options such as 'use_kerberos', 'kerberos_principal', etc to configure Kerberos settings. A new connection provider 'HiveJdbcConnectionProvider' has been added which takes care of connection creation when using Kerberos. This change also makes the connection provider more flexible as it can now be selected based on the JDBC dialect." --- .../jdbc/config/JdbcConnectionConfig.java | 43 ++++++++++++++++++- .../seatunnel/jdbc/config/JdbcOptions.java | 27 ++++++++++++ .../exception/JdbcConnectorErrorCode.java | 3 +- .../SimpleJdbcConnectionProvider.java | 4 +- .../jdbc/internal/dialect/JdbcDialect.java | 8 ++++ .../internal/dialect/hive/HiveDialect.java | 8 ++++ .../seatunnel/jdbc/sink/JdbcSinkWriter.java | 3 +- .../seatunnel/jdbc/source/JdbcSource.java | 5 +-- .../jdbc/source/JdbcSourceFactory.java | 5 +-- 9 files changed, 95 insertions(+), 11 deletions(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcConnectionConfig.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcConnectionConfig.java index 555963af2cf..b38206a30b1 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcConnectionConfig.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcConnectionConfig.java @@ -45,6 +45,14 @@ public class JdbcConnectionConfig implements Serializable { public int transactionTimeoutSec = JdbcOptions.TRANSACTION_TIMEOUT_SEC.defaultValue(); + public boolean useKerberos = JdbcOptions.USE_KERBEROS.defaultValue(); + + public String kerberosPrincipal; + + public String kerberosKeytabPath; + + public String krb5Path = JdbcOptions.KRB5_PATH.defaultValue(); + public static JdbcConnectionConfig of(ReadonlyConfig config) { JdbcConnectionConfig.Builder builder = JdbcConnectionConfig.builder(); builder.url(config.get(JdbcOptions.URL)); @@ -60,7 +68,12 @@ public static JdbcConnectionConfig of(ReadonlyConfig config) { builder.transactionTimeoutSec(config.get(JdbcOptions.TRANSACTION_TIMEOUT_SEC)); builder.maxRetries(0); } - + if (config.get(JdbcOptions.USE_KERBEROS)) { + builder.useKerberos(config.get(JdbcOptions.USE_KERBEROS)); + builder.kerberosPrincipal(config.get(JdbcOptions.KERBEROS_PRINCIPAL)); + builder.kerberosKeytabPath(config.get(JdbcOptions.KERBEROS_KEYTAB_PATH)); + builder.kerberosKeytabPath(config.get(JdbcOptions.KRB5_PATH)); + } config.getOptional(JdbcOptions.USER).ifPresent(builder::username); config.getOptional(JdbcOptions.PASSWORD).ifPresent(builder::password); return builder.build(); @@ -133,6 +146,10 @@ public static final class Builder { private String xaDataSourceClassName; private int maxCommitAttempts = JdbcOptions.MAX_COMMIT_ATTEMPTS.defaultValue(); private int transactionTimeoutSec = JdbcOptions.TRANSACTION_TIMEOUT_SEC.defaultValue(); + public boolean useKerberos = JdbcOptions.USE_KERBEROS.defaultValue(); + public String kerberosPrincipal; + public String kerberosKeytabPath; + public String krb5Path = JdbcOptions.KRB5_PATH.defaultValue(); private Builder() {} @@ -201,6 +218,26 @@ public Builder transactionTimeoutSec(int transactionTimeoutSec) { return this; } + public Builder useKerberos(boolean useKerberos) { + this.useKerberos = useKerberos; + return this; + } + + public Builder kerberosPrincipal(String kerberosPrincipal) { + this.kerberosPrincipal = kerberosPrincipal; + return this; + } + + public Builder kerberosKeytabPath(String kerberosKeytabPath) { + this.kerberosKeytabPath = kerberosKeytabPath; + return this; + } + + public Builder krb5Path(String krb5Path) { + this.krb5Path = krb5Path; + return this; + } + public JdbcConnectionConfig build() { JdbcConnectionConfig jdbcConnectionConfig = new JdbcConnectionConfig(); jdbcConnectionConfig.batchSize = this.batchSize; @@ -215,6 +252,10 @@ public JdbcConnectionConfig build() { jdbcConnectionConfig.transactionTimeoutSec = this.transactionTimeoutSec; jdbcConnectionConfig.maxCommitAttempts = this.maxCommitAttempts; jdbcConnectionConfig.xaDataSourceClassName = this.xaDataSourceClassName; + jdbcConnectionConfig.useKerberos = this.useKerberos; + jdbcConnectionConfig.kerberosPrincipal = this.kerberosPrincipal; + jdbcConnectionConfig.kerberosKeytabPath = this.kerberosKeytabPath; + jdbcConnectionConfig.krb5Path = this.krb5Path; return jdbcConnectionConfig; } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java index b01fc872f31..aa74cc35011 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java @@ -161,4 +161,31 @@ public interface JdbcOptions { .enumType(FieldIdeEnum.class) .noDefaultValue() .withDescription("Whether case conversion is required"); + + Option USE_KERBEROS = + Options.key("use_kerberos") + .booleanType() + .defaultValue(false) + .withDescription("Whether to enable Kerberos, default is false."); + + Option KERBEROS_PRINCIPAL = + Options.key("kerberos_principal") + .stringType() + .noDefaultValue() + .withDescription( + "When use kerberos, we should set kerberos principal such as 'test_user@xxx'. "); + + Option KERBEROS_KEYTAB_PATH = + Options.key("kerberos_keytab_path") + .stringType() + .noDefaultValue() + .withDescription( + "When use kerberos, we should set kerberos principal file path such as '/home/test/test_user.keytab'. "); + + Option KRB5_PATH = + Options.key("krb5_path") + .stringType() + .defaultValue("/etc/krb5.conf") + .withDescription( + "When use kerberos, we should set krb5 path file path such as '/etc/krb5.conf'. "); } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/exception/JdbcConnectorErrorCode.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/exception/JdbcConnectorErrorCode.java index 5ebde65d19c..22438de84c3 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/exception/JdbcConnectorErrorCode.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/exception/JdbcConnectorErrorCode.java @@ -28,7 +28,8 @@ public enum JdbcConnectorErrorCode implements SeaTunnelErrorCode { TRANSACTION_OPERATION_FAILED( "JDBC-05", "transaction operation failed, such as (commit, rollback) etc.."), NO_SUITABLE_DIALECT_FACTORY("JDBC-06", "No suitable dialect factory found"), - DONT_SUPPORT_SINK("JDBC-07", "The jdbc type don't support sink"); + DONT_SUPPORT_SINK("JDBC-07", "The jdbc type don't support sink"), + KERBEROS_AUTHENTICATION_FAILED("JDBC-08", "Kerberos authentication failed"); private final String code; private final String description; diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/connection/SimpleJdbcConnectionProvider.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/connection/SimpleJdbcConnectionProvider.java index 815d51a3f08..499b33ff47d 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/connection/SimpleJdbcConnectionProvider.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/connection/SimpleJdbcConnectionProvider.java @@ -24,6 +24,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import lombok.Data; import lombok.NonNull; import java.io.Serializable; @@ -37,6 +38,7 @@ import static com.google.common.base.Preconditions.checkNotNull; /** Simple JDBC connection provider. */ +@Data public class SimpleJdbcConnectionProvider implements JdbcConnectionProvider, Serializable { private static final Logger LOG = LoggerFactory.getLogger(SimpleJdbcConnectionProvider.class); @@ -88,7 +90,7 @@ private static Driver loadDriver(String driverName) throws ClassNotFoundExceptio } } - private Driver getLoadedDriver() throws SQLException, ClassNotFoundException { + protected Driver getLoadedDriver() throws SQLException, ClassNotFoundException { if (loadedDriver == null) { loadedDriver = loadDriver(jdbcConfig.getDriverName()); } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java index e0cf5252a60..d6e983af8de 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java @@ -18,7 +18,10 @@ package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect; import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcConnectionConfig; import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.JdbcConnectionProvider; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.SimpleJdbcConnectionProvider; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.converter.JdbcRowConverter; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.dialectenum.FieldIdeEnum; @@ -240,4 +243,9 @@ default String getFieldIde(String identifier, String fieldIde) { return identifier; } } + + default JdbcConnectionProvider getJdbcConnectionProvider( + JdbcConnectionConfig jdbcConnectionConfig) { + return new SimpleJdbcConnectionProvider(jdbcConnectionConfig); + } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialect.java index acb60004bcb..dd40443c7eb 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialect.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialect.java @@ -17,7 +17,9 @@ package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.hive; +import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcConnectionConfig; import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.JdbcConnectionProvider; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.converter.JdbcRowConverter; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectTypeMapper; @@ -57,4 +59,10 @@ public ResultSetMetaData getResultSetMetaData( PreparedStatement ps = conn.prepareStatement(jdbcSourceConfig.getQuery()); return ps.executeQuery().getMetaData(); } + + @Override + public JdbcConnectionProvider getJdbcConnectionProvider( + JdbcConnectionConfig jdbcConnectionConfig) { + return new HiveJdbcConnectionProvider(jdbcConnectionConfig); + } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkWriter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkWriter.java index db543e1705f..d2bd1c4adbe 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkWriter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkWriter.java @@ -27,7 +27,6 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.JdbcOutputFormat; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.JdbcOutputFormatBuilder; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.JdbcConnectionProvider; -import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.SimpleJdbcConnectionProvider; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.executor.JdbcBatchStatementExecutor; import org.apache.seatunnel.connectors.seatunnel.jdbc.state.JdbcSinkState; @@ -56,7 +55,7 @@ public JdbcSinkWriter( SeaTunnelRowType rowType) { this.context = context; this.connectionProvider = - new SimpleJdbcConnectionProvider(jdbcSinkConfig.getJdbcConnectionConfig()); + dialect.getJdbcConnectionProvider(jdbcSinkConfig.getJdbcConnectionConfig()); this.outputFormat = new JdbcOutputFormatBuilder(dialect, connectionProvider, jdbcSinkConfig, rowType) .build(); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSource.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSource.java index 1bf1b332fa9..a8b16322d1c 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSource.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSource.java @@ -36,7 +36,6 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceConfig; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.JdbcInputFormat; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.JdbcConnectionProvider; -import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.SimpleJdbcConnectionProvider; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectLoader; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectTypeMapper; @@ -97,13 +96,13 @@ public void prepare(Config pluginConfig) throws PrepareFailException { ReadonlyConfig config = ReadonlyConfig.fromConfig(pluginConfig); ConfigValidator.of(config).validate(new JdbcSourceFactory().optionRule()); this.jdbcSourceConfig = JdbcSourceConfig.of(config); - this.jdbcConnectionProvider = - new SimpleJdbcConnectionProvider(jdbcSourceConfig.getJdbcConnectionConfig()); this.query = jdbcSourceConfig.getQuery(); this.jdbcDialect = JdbcDialectLoader.load( jdbcSourceConfig.getJdbcConnectionConfig().getUrl(), jdbcSourceConfig.getJdbcConnectionConfig().getCompatibleMode()); + this.jdbcConnectionProvider = + jdbcDialect.getJdbcConnectionProvider(jdbcSourceConfig.getJdbcConnectionConfig()); try (Connection connection = jdbcConnectionProvider.getOrEstablishConnection()) { this.typeInfo = initTableField(connection); this.partitionParameter = diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceFactory.java index 8c21a842339..f7ae59791e1 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceFactory.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceFactory.java @@ -38,7 +38,6 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorException; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.JdbcInputFormat; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.JdbcConnectionProvider; -import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.SimpleJdbcConnectionProvider; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectLoader; @@ -81,13 +80,13 @@ public String factoryIdentifier() { TableSource createSource(TableFactoryContext context) { CatalogTable catalogTable = context.getCatalogTable(); JdbcSourceConfig config = JdbcSourceConfig.of(context.getOptions()); - JdbcConnectionProvider connectionProvider = - new SimpleJdbcConnectionProvider(config.getJdbcConnectionConfig()); final String querySql = config.getQuery(); JdbcDialect dialect = JdbcDialectLoader.load( config.getJdbcConnectionConfig().getUrl(), config.getJdbcConnectionConfig().getCompatibleMode()); + JdbcConnectionProvider connectionProvider = + dialect.getJdbcConnectionProvider(config.getJdbcConnectionConfig()); TableSchema tableSchema = catalogTable.getTableSchema(); SeaTunnelRowType rowType = tableSchema.toPhysicalRowDataType(); Optional partitionParameter = From b1460259f8b01a386ef9569248969c473771c949 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Tue, 26 Sep 2023 13:56:00 +0800 Subject: [PATCH 14/58] :sparkles: "Added kerberos support in JDBC connections This commit enables kerberos authentication while establishing JDBC connections. The JdbcOptions class has been updated to include new options such as 'use_kerberos', 'kerberos_principal', etc to configure Kerberos settings. A new connection provider 'HiveJdbcConnectionProvider' has been added which takes care of connection creation when using Kerberos. This change also makes the connection provider more flexible as it can now be selected based on the JDBC dialect." --- .../hive/HiveJdbcConnectionProvider.java | 63 +++++++++++++++++++ 1 file changed, 63 insertions(+) create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java new file mode 100644 index 00000000000..ae8a5f90166 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java @@ -0,0 +1,63 @@ +package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.hive; + +import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcConnectionConfig; +import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorErrorCode; +import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorException; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.SimpleJdbcConnectionProvider; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; + +import lombok.NonNull; + +import java.io.IOException; +import java.sql.Connection; +import java.sql.Driver; +import java.sql.SQLException; +import java.util.Properties; + +import static org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorErrorCode.KERBEROS_AUTHENTICATION_FAILED; + +public class HiveJdbcConnectionProvider extends SimpleJdbcConnectionProvider { + + public HiveJdbcConnectionProvider(@NonNull JdbcConnectionConfig jdbcConfig) { + super(jdbcConfig); + } + + @Override + public Connection getOrEstablishConnection() throws SQLException, ClassNotFoundException { + if (isConnectionValid()) { + return super.getConnection(); + } + JdbcConnectionConfig jdbcConfig = super.getJdbcConfig(); + if (jdbcConfig.useKerberos) { + System.setProperty("java.security.krb5.conf", jdbcConfig.krb5Path); + Configuration configuration = new Configuration(); + configuration.set("hadoop.security.authentication", "kerberos"); + UserGroupInformation.setConfiguration(configuration); + try { + UserGroupInformation.loginUserFromKeytab( + jdbcConfig.kerberosPrincipal, jdbcConfig.kerberosKeytabPath); + } catch (IOException e) { + throw new JdbcConnectorException(KERBEROS_AUTHENTICATION_FAILED, e); + } + } + Driver driver = getLoadedDriver(); + Properties info = new Properties(); + if (super.getJdbcConfig().getUsername().isPresent()) { + info.setProperty("user", super.getJdbcConfig().getUsername().get()); + } + if (super.getJdbcConfig().getPassword().isPresent()) { + info.setProperty("password", super.getJdbcConfig().getPassword().get()); + } + super.setConnection(driver.connect(super.getJdbcConfig().getUrl(), info)); + if (super.getConnection() == null) { + // Throw same exception as DriverManager.getConnection when no driver found to match + // caller expectation. + throw new JdbcConnectorException( + JdbcConnectorErrorCode.NO_SUITABLE_DRIVER, + "No suitable driver found for " + super.getJdbcConfig().getUrl()); + } + return super.getConnection(); + } +} From 1797d83fcfdd451301a05dd814d9c06b1990a701 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Tue, 26 Sep 2023 14:02:52 +0800 Subject: [PATCH 15/58] Add ASF license header to HiveJdbcConnectionProvider file This commit adds the Apache License Version 2.0 header to the HiveJdbcConnectionProvider file. Rights, terms, and conditions associated with the usage of the file in compliance with ASF policies are now clearly outlined within the header of the file. --- .../hive/HiveJdbcConnectionProvider.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java index ae8a5f90166..56841937d20 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java @@ -1,3 +1,20 @@ +/* + * 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.jdbc.internal.dialect.hive; import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcConnectionConfig; From 147bb802d43c9cdf632e3c318b3fac4b1b3b2657 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Tue, 26 Sep 2023 14:16:37 +0800 Subject: [PATCH 16/58] revert: "Add Kingbase8 dependency to JDBC connector The Kingbase8 database connector was added to the JDBC connector's pom.xml. This change allows our application to interact with Kingbase8 databases, expanding possible integration scenarios for applications using our connector." --- seatunnel-connectors-v2/connector-jdbc/pom.xml | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/seatunnel-connectors-v2/connector-jdbc/pom.xml b/seatunnel-connectors-v2/connector-jdbc/pom.xml index 00a8e715635..baeac2bda12 100644 --- a/seatunnel-connectors-v2/connector-jdbc/pom.xml +++ b/seatunnel-connectors-v2/connector-jdbc/pom.xml @@ -145,6 +145,12 @@ ${vertica.version} provided + + cn.com.kingbase + kingbase8 + ${kingbase8.version} + provided + org.apache.hive hive-jdbc @@ -231,6 +237,10 @@ com.vertica.jdbc vertica-jdbc + + cn.com.kingbase + kingbase8 + org.apache.hive hive-jdbc From 0c236f2ec653c1567d569683fc166e52f01fd540 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Tue, 26 Sep 2023 14:37:04 +0800 Subject: [PATCH 17/58] :art: Refactor HiveJdbcConnectionProvider and SimpleJdbcConnectionProvider classes Changed the method of accessing JdbcConfig object and the Connection object in both classes from getter methods to direct access. This was done to create a more cohesive codebase and promote a uniform way of accessing these objects. This change also aids in improving the code readability and reducing ambiguities when dealing with these objects across different packages. --- .../connection/SimpleJdbcConnectionProvider.java | 5 ++--- .../dialect/hive/HiveJdbcConnectionProvider.java | 14 +++++++------- 2 files changed, 9 insertions(+), 10 deletions(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/connection/SimpleJdbcConnectionProvider.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/connection/SimpleJdbcConnectionProvider.java index 499b33ff47d..da66691e400 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/connection/SimpleJdbcConnectionProvider.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/connection/SimpleJdbcConnectionProvider.java @@ -38,17 +38,16 @@ import static com.google.common.base.Preconditions.checkNotNull; /** Simple JDBC connection provider. */ -@Data public class SimpleJdbcConnectionProvider implements JdbcConnectionProvider, Serializable { private static final Logger LOG = LoggerFactory.getLogger(SimpleJdbcConnectionProvider.class); private static final long serialVersionUID = 1L; - private final JdbcConnectionConfig jdbcConfig; + protected final JdbcConnectionConfig jdbcConfig; private transient Driver loadedDriver; - private transient Connection connection; + protected transient Connection connection; public SimpleJdbcConnectionProvider(@NonNull JdbcConnectionConfig jdbcConfig) { this.jdbcConfig = jdbcConfig; diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java index 56841937d20..cbd013574bd 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java @@ -46,7 +46,7 @@ public Connection getOrEstablishConnection() throws SQLException, ClassNotFoundE if (isConnectionValid()) { return super.getConnection(); } - JdbcConnectionConfig jdbcConfig = super.getJdbcConfig(); + JdbcConnectionConfig jdbcConfig = super.jdbcConfig; if (jdbcConfig.useKerberos) { System.setProperty("java.security.krb5.conf", jdbcConfig.krb5Path); Configuration configuration = new Configuration(); @@ -61,19 +61,19 @@ public Connection getOrEstablishConnection() throws SQLException, ClassNotFoundE } Driver driver = getLoadedDriver(); Properties info = new Properties(); - if (super.getJdbcConfig().getUsername().isPresent()) { - info.setProperty("user", super.getJdbcConfig().getUsername().get()); + if (super.jdbcConfig.getUsername().isPresent()) { + info.setProperty("user", super.jdbcConfig.getUsername().get()); } - if (super.getJdbcConfig().getPassword().isPresent()) { - info.setProperty("password", super.getJdbcConfig().getPassword().get()); + if (super.jdbcConfig.getPassword().isPresent()) { + info.setProperty("password", super.jdbcConfig.getPassword().get()); } - super.setConnection(driver.connect(super.getJdbcConfig().getUrl(), info)); + super.connection=driver.connect(super.jdbcConfig.getUrl(), info); if (super.getConnection() == null) { // Throw same exception as DriverManager.getConnection when no driver found to match // caller expectation. throw new JdbcConnectorException( JdbcConnectorErrorCode.NO_SUITABLE_DRIVER, - "No suitable driver found for " + super.getJdbcConfig().getUrl()); + "No suitable driver found for " + super.jdbcConfig.getUrl()); } return super.getConnection(); } From 01e7aad36343a90be271e7618e31f076f7405a9e Mon Sep 17 00:00:00 2001 From: yangpeng Date: Tue, 26 Sep 2023 14:38:43 +0800 Subject: [PATCH 18/58] :art: Refactor HiveJdbcConnectionProvider and SimpleJdbcConnectionProvider classes Changed the method of accessing JdbcConfig object and the Connection object in both classes from getter methods to direct access. This was done to create a more cohesive codebase and promote a uniform way of accessing these objects. This change also aids in improving the code readability and reducing ambiguities when dealing with these objects across different packages. --- .../jdbc/internal/connection/SimpleJdbcConnectionProvider.java | 1 - .../jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/connection/SimpleJdbcConnectionProvider.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/connection/SimpleJdbcConnectionProvider.java index da66691e400..40642b9c555 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/connection/SimpleJdbcConnectionProvider.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/connection/SimpleJdbcConnectionProvider.java @@ -24,7 +24,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import lombok.Data; import lombok.NonNull; import java.io.Serializable; diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java index cbd013574bd..88a41ccebbf 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java @@ -67,7 +67,7 @@ public Connection getOrEstablishConnection() throws SQLException, ClassNotFoundE if (super.jdbcConfig.getPassword().isPresent()) { info.setProperty("password", super.jdbcConfig.getPassword().get()); } - super.connection=driver.connect(super.jdbcConfig.getUrl(), info); + super.connection = driver.connect(super.jdbcConfig.getUrl(), info); if (super.getConnection() == null) { // Throw same exception as DriverManager.getConnection when no driver found to match // caller expectation. From 17a8fb63b2d013373a26e853a8ff81091d7b149a Mon Sep 17 00:00:00 2001 From: yangpeng Date: Sun, 8 Oct 2023 18:31:19 +0800 Subject: [PATCH 19/58] =?UTF-8?q?=E8=A7=A3=E5=86=B3=E5=86=B2=E7=AA=81?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../jdbc/source/JdbcSourceFactory.java | 67 +++++++++++++++---- 1 file changed, 54 insertions(+), 13 deletions(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceFactory.java index f7ae59791e1..da192971529 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceFactory.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceFactory.java @@ -22,12 +22,13 @@ import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.source.SourceSplit; import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.catalog.PrimaryKey; import org.apache.seatunnel.api.table.catalog.TableSchema; import org.apache.seatunnel.api.table.connector.TableSource; import org.apache.seatunnel.api.table.factory.Factory; -import org.apache.seatunnel.api.table.factory.TableFactoryContext; import org.apache.seatunnel.api.table.factory.TableSourceFactory; +import org.apache.seatunnel.api.table.factory.TableSourceFactoryContext; import org.apache.seatunnel.api.table.type.BasicType; import org.apache.seatunnel.api.table.type.DecimalType; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; @@ -40,6 +41,7 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.JdbcConnectionProvider; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectLoader; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectTypeMapper; import com.google.auto.service.AutoService; import lombok.extern.slf4j.Slf4j; @@ -48,7 +50,9 @@ import java.math.BigDecimal; import java.sql.Connection; import java.sql.ResultSet; +import java.sql.ResultSetMetaData; import java.sql.SQLException; +import java.util.ArrayList; import java.util.HashMap; import java.util.Map; import java.util.Optional; @@ -77,20 +81,34 @@ public String factoryIdentifier() { @Override @SuppressWarnings("unchecked") public - TableSource createSource(TableFactoryContext context) { - CatalogTable catalogTable = context.getCatalogTable(); + TableSource createSource(TableSourceFactoryContext context) { JdbcSourceConfig config = JdbcSourceConfig.of(context.getOptions()); - final String querySql = config.getQuery(); JdbcDialect dialect = JdbcDialectLoader.load( config.getJdbcConnectionConfig().getUrl(), config.getJdbcConnectionConfig().getCompatibleMode()); JdbcConnectionProvider connectionProvider = - dialect.getJdbcConnectionProvider(config.getJdbcConnectionConfig()); - TableSchema tableSchema = catalogTable.getTableSchema(); - SeaTunnelRowType rowType = tableSchema.toPhysicalRowDataType(); - Optional partitionParameter = - createPartitionParameter(config, tableSchema, connectionProvider); + dialect.getJdbcConnectionProvider((config.getJdbcConnectionConfig())); + SeaTunnelRowType rowType; + Optional partitionParameter = Optional.empty(); + try { + CatalogTable catalogTable = + CatalogTableUtil.getCatalogTablesFromConfig( + dialect.dialectName(), + context.getOptions(), + context.getClassLoader()) + .get(0); + TableSchema tableSchema = catalogTable.getTableSchema(); + rowType = tableSchema.toPhysicalRowDataType(); + partitionParameter = createPartitionParameter(config, tableSchema, connectionProvider); + } catch (Exception e) { + try (Connection connection = connectionProvider.getOrEstablishConnection()) { + rowType = initTableField(connection, config, dialect); + } catch (Exception k) { + throw new PrepareFailException("jdbc", PluginType.SOURCE, k.toString()); + } + } + final String querySql = config.getQuery(); JdbcInputFormat inputFormat = new JdbcInputFormat( connectionProvider, @@ -99,18 +117,20 @@ TableSource createSource(TableFactoryContext context) { querySql, config.getFetchSize(), config.getJdbcConnectionConfig().isAutoCommit()); + Optional finalPartitionParameter = partitionParameter; + SeaTunnelRowType finalRowType = rowType; return () -> (SeaTunnelSource) new JdbcSource( config, - rowType, + finalRowType, dialect, inputFormat, - partitionParameter.orElse(null), + finalPartitionParameter.orElse(null), connectionProvider, - partitionParameter.isPresent() + finalPartitionParameter.isPresent() ? obtainPartitionSql( - dialect, partitionParameter.get(), querySql) + dialect, finalPartitionParameter.get(), querySql) : querySql); } @@ -131,6 +151,27 @@ static String obtainPartitionSql( partitionParameter.getPartitionColumnName()); } + private SeaTunnelRowType initTableField( + Connection conn, JdbcSourceConfig jdbcSourceConfig, JdbcDialect jdbcDialect) { + JdbcDialectTypeMapper jdbcDialectTypeMapper = jdbcDialect.getJdbcDialectTypeMapper(); + ArrayList> seaTunnelDataTypes = new ArrayList<>(); + ArrayList fieldNames = new ArrayList<>(); + try { + ResultSetMetaData resultSetMetaData = + jdbcDialect.getResultSetMetaData(conn, jdbcSourceConfig); + for (int i = 1; i <= resultSetMetaData.getColumnCount(); i++) { + // Support AS syntax + fieldNames.add(resultSetMetaData.getColumnLabel(i)); + seaTunnelDataTypes.add(jdbcDialectTypeMapper.mapping(resultSetMetaData, i)); + } + } catch (Exception e) { + log.warn("get row type info exception", e); + } + return new SeaTunnelRowType( + fieldNames.toArray(new String[0]), + seaTunnelDataTypes.toArray(new SeaTunnelDataType[0])); + } + public static Optional createPartitionParameter( JdbcSourceConfig config, TableSchema tableSchema, From a1d1effb80153ca0c7deb223cda516a30ccfbc27 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Sun, 8 Oct 2023 18:32:54 +0800 Subject: [PATCH 20/58] =?UTF-8?q?=E8=A7=A3=E5=86=B3=E5=86=B2=E7=AA=81?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../apache/seatunnel/example/engine/SeaTunnelEngineExample.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java b/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java index 2a7c25e0830..40320aec12c 100644 --- a/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java +++ b/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java @@ -31,7 +31,7 @@ public class SeaTunnelEngineExample { public static void main(String[] args) throws FileNotFoundException, URISyntaxException, CommandException { - String configurePath = args.length > 0 ? args[0] : "/examples/fake_to_console.conf"; + String configurePath = args.length > 0 ? args[0] : "/examples/fake_to_console1.conf"; String configFile = getTestConfigFile(configurePath); ClientCommandArgs clientCommandArgs = new ClientCommandArgs(); clientCommandArgs.setConfigFile(configFile); From 52244c56172ce5695a5356b15ff4ce5ee8716f1b Mon Sep 17 00:00:00 2001 From: yangpeng Date: Sun, 8 Oct 2023 18:34:12 +0800 Subject: [PATCH 21/58] =?UTF-8?q?=E8=A7=A3=E5=86=B3=E5=86=B2=E7=AA=81?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../connectors/seatunnel/jdbc/source/JdbcSourceFactory.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceFactory.java index da192971529..264df5eafa0 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceFactory.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceFactory.java @@ -39,6 +39,7 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorException; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.JdbcInputFormat; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.JdbcConnectionProvider; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.SimpleJdbcConnectionProvider; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectLoader; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectTypeMapper; @@ -88,7 +89,8 @@ TableSource createSource(TableSourceFactoryContext context) { config.getJdbcConnectionConfig().getUrl(), config.getJdbcConnectionConfig().getCompatibleMode()); JdbcConnectionProvider connectionProvider = - dialect.getJdbcConnectionProvider((config.getJdbcConnectionConfig())); + new SimpleJdbcConnectionProvider(config.getJdbcConnectionConfig()); + SeaTunnelRowType rowType; Optional partitionParameter = Optional.empty(); try { From dcf61488181af925561a1021489546fa6e3b7109 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Sun, 8 Oct 2023 18:35:31 +0800 Subject: [PATCH 22/58] revert --- .../apache/seatunnel/example/engine/SeaTunnelEngineExample.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java b/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java index 40320aec12c..2a7c25e0830 100644 --- a/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java +++ b/seatunnel-examples/seatunnel-engine-examples/src/main/java/org/apache/seatunnel/example/engine/SeaTunnelEngineExample.java @@ -31,7 +31,7 @@ public class SeaTunnelEngineExample { public static void main(String[] args) throws FileNotFoundException, URISyntaxException, CommandException { - String configurePath = args.length > 0 ? args[0] : "/examples/fake_to_console1.conf"; + String configurePath = args.length > 0 ? args[0] : "/examples/fake_to_console.conf"; String configFile = getTestConfigFile(configurePath); ClientCommandArgs clientCommandArgs = new ClientCommandArgs(); clientCommandArgs.setConfigFile(configFile); From d553e2412523ecb51976d80171efff2b2f43871c Mon Sep 17 00:00:00 2001 From: yangpeng Date: Sun, 8 Oct 2023 18:41:24 +0800 Subject: [PATCH 23/58] revert --- .../connectors/seatunnel/jdbc/source/JdbcSourceFactory.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceFactory.java index 264df5eafa0..d062d81dc47 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceFactory.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceFactory.java @@ -39,7 +39,6 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorException; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.JdbcInputFormat; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.JdbcConnectionProvider; -import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.SimpleJdbcConnectionProvider; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectLoader; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectTypeMapper; @@ -89,8 +88,7 @@ TableSource createSource(TableSourceFactoryContext context) { config.getJdbcConnectionConfig().getUrl(), config.getJdbcConnectionConfig().getCompatibleMode()); JdbcConnectionProvider connectionProvider = - new SimpleJdbcConnectionProvider(config.getJdbcConnectionConfig()); - + dialect.getJdbcConnectionProvider(config.getJdbcConnectionConfig()); SeaTunnelRowType rowType; Optional partitionParameter = Optional.empty(); try { From 44507bfac12dfdbf47bcc875ece9b62de0b5f38e Mon Sep 17 00:00:00 2001 From: yangpeng Date: Thu, 12 Oct 2023 10:30:35 +0800 Subject: [PATCH 24/58] :art: --- .../seatunnel/jdbc/config/JdbcConnectionConfig.java | 6 +++--- .../connectors/seatunnel/jdbc/config/JdbcOptions.java | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcConnectionConfig.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcConnectionConfig.java index b38206a30b1..77da9d3023b 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcConnectionConfig.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcConnectionConfig.java @@ -51,7 +51,7 @@ public class JdbcConnectionConfig implements Serializable { public String kerberosKeytabPath; - public String krb5Path = JdbcOptions.KRB5_PATH.defaultValue(); + public String krb5Path = JdbcOptions.KERBEROS_KRB5_CONF_PATH.defaultValue(); public static JdbcConnectionConfig of(ReadonlyConfig config) { JdbcConnectionConfig.Builder builder = JdbcConnectionConfig.builder(); @@ -72,7 +72,7 @@ public static JdbcConnectionConfig of(ReadonlyConfig config) { builder.useKerberos(config.get(JdbcOptions.USE_KERBEROS)); builder.kerberosPrincipal(config.get(JdbcOptions.KERBEROS_PRINCIPAL)); builder.kerberosKeytabPath(config.get(JdbcOptions.KERBEROS_KEYTAB_PATH)); - builder.kerberosKeytabPath(config.get(JdbcOptions.KRB5_PATH)); + builder.kerberosKeytabPath(config.get(JdbcOptions.KERBEROS_KRB5_CONF_PATH)); } config.getOptional(JdbcOptions.USER).ifPresent(builder::username); config.getOptional(JdbcOptions.PASSWORD).ifPresent(builder::password); @@ -149,7 +149,7 @@ public static final class Builder { public boolean useKerberos = JdbcOptions.USE_KERBEROS.defaultValue(); public String kerberosPrincipal; public String kerberosKeytabPath; - public String krb5Path = JdbcOptions.KRB5_PATH.defaultValue(); + public String krb5Path = JdbcOptions.KERBEROS_KRB5_CONF_PATH.defaultValue(); private Builder() {} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java index aa74cc35011..8be906001fa 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java @@ -182,8 +182,8 @@ public interface JdbcOptions { .withDescription( "When use kerberos, we should set kerberos principal file path such as '/home/test/test_user.keytab'. "); - Option KRB5_PATH = - Options.key("krb5_path") + Option KERBEROS_KRB5_CONF_PATH = + Options.key("kerberos_krb5_conf_path") .stringType() .defaultValue("/etc/krb5.conf") .withDescription( From 8a9fe618341ca5068cf0d845e75e18480d3e1c1d Mon Sep 17 00:00:00 2001 From: yangpeng Date: Thu, 12 Oct 2023 17:37:50 +0800 Subject: [PATCH 25/58] :art: Refactor kerberos authentication for HiveJdbcConnection --- .../hive/HiveJdbcConnectionProvider.java | 18 +---- .../seatunnel/jdbc/utils/HiveJdbcUtils.java | 68 +++++++++++++++++++ 2 files changed, 70 insertions(+), 16 deletions(-) create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/utils/HiveJdbcUtils.java diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java index 88a41ccebbf..36da36485e1 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java @@ -21,20 +21,15 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorErrorCode; import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorException; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.SimpleJdbcConnectionProvider; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.security.UserGroupInformation; +import org.apache.seatunnel.connectors.seatunnel.jdbc.utils.HiveJdbcUtils; import lombok.NonNull; -import java.io.IOException; import java.sql.Connection; import java.sql.Driver; import java.sql.SQLException; import java.util.Properties; -import static org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorErrorCode.KERBEROS_AUTHENTICATION_FAILED; - public class HiveJdbcConnectionProvider extends SimpleJdbcConnectionProvider { public HiveJdbcConnectionProvider(@NonNull JdbcConnectionConfig jdbcConfig) { @@ -48,16 +43,7 @@ public Connection getOrEstablishConnection() throws SQLException, ClassNotFoundE } JdbcConnectionConfig jdbcConfig = super.jdbcConfig; if (jdbcConfig.useKerberos) { - System.setProperty("java.security.krb5.conf", jdbcConfig.krb5Path); - Configuration configuration = new Configuration(); - configuration.set("hadoop.security.authentication", "kerberos"); - UserGroupInformation.setConfiguration(configuration); - try { - UserGroupInformation.loginUserFromKeytab( - jdbcConfig.kerberosPrincipal, jdbcConfig.kerberosKeytabPath); - } catch (IOException e) { - throw new JdbcConnectorException(KERBEROS_AUTHENTICATION_FAILED, e); - } + HiveJdbcUtils.doKerberosAuthentication(jdbcConfig); } Driver driver = getLoadedDriver(); Properties info = new Properties(); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/utils/HiveJdbcUtils.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/utils/HiveJdbcUtils.java new file mode 100644 index 00000000000..0b503b30848 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/utils/HiveJdbcUtils.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.utils; + +import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcConnectionConfig; +import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorException; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; + +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; + +import static org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorErrorCode.KERBEROS_AUTHENTICATION_FAILED; + +@Slf4j +public class HiveJdbcUtils { + + public static synchronized void doKerberosAuthentication(JdbcConnectionConfig jdbcConfig) { + String principal = jdbcConfig.kerberosPrincipal; + String keytabPath = jdbcConfig.kerberosKeytabPath; + String krb5Path = jdbcConfig.krb5Path; + System.setProperty("java.security.krb5.conf", krb5Path); + Configuration configuration = new Configuration(); + + if (StringUtils.isBlank(principal) || StringUtils.isBlank(keytabPath)) { + log.warn( + "Principal [{}] or keytabPath [{}] is empty, it will skip kerberos authentication", + principal, + keytabPath); + } else { + configuration.set("hadoop.security.authentication", "kerberos"); + UserGroupInformation.setConfiguration(configuration); + try { + log.info( + "Start Kerberos authentication using principal {} and keytab {}", + principal, + keytabPath); + UserGroupInformation.loginUserFromKeytab(principal, keytabPath); + log.info("Kerberos authentication successful"); + } catch (IOException e) { + String errorMsg = + String.format( + "Kerberos authentication failed using this " + + "principal [%s] and keytab path [%s]", + principal, keytabPath); + throw new JdbcConnectorException(KERBEROS_AUTHENTICATION_FAILED, errorMsg, e); + } + } + } +} From 2e89d3d53f00eb43d18bd139ad64ed13c24495a3 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Thu, 19 Oct 2023 18:15:43 +0800 Subject: [PATCH 26/58] Added "provided" scope to hive-service dependency in pom.xml --- seatunnel-connectors-v2/connector-jdbc/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/seatunnel-connectors-v2/connector-jdbc/pom.xml b/seatunnel-connectors-v2/connector-jdbc/pom.xml index baeac2bda12..9c479a5bb81 100644 --- a/seatunnel-connectors-v2/connector-jdbc/pom.xml +++ b/seatunnel-connectors-v2/connector-jdbc/pom.xml @@ -161,6 +161,7 @@ org.apache.hive hive-service ${hive.jdbc.version} + provided From e49e08d8e3a9f577276dd2af7b9b836e3d9d40be Mon Sep 17 00:00:00 2001 From: yangpeng Date: Thu, 19 Oct 2023 21:25:18 +0800 Subject: [PATCH 27/58] :sparkles: Add hive-service JAR to JDBC test setup --- .../seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java index 2738f5ccc36..2b69fd64097 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java @@ -76,7 +76,7 @@ public abstract class AbstractJdbcIT extends TestSuiteBase implements TestResour "bash", "-c", "mkdir -p /tmp/seatunnel/plugins/Jdbc/lib && cd /tmp/seatunnel/plugins/Jdbc/lib && wget " - + driverUrl()); + + driverUrl()+ "&& wget https://repo1.maven.org/maven2/org/apache/hive/hive-service/3.1.3/hive-service-3.1.3.jar"); Assertions.assertEquals(0, extraCommands.getExitCode(), extraCommands.getStderr()); }; From 28bcebd6fe7ec253c043132bf83856ddf5d9af2e Mon Sep 17 00:00:00 2001 From: yangpeng Date: Thu, 19 Oct 2023 21:27:21 +0800 Subject: [PATCH 28/58] :art: format --- .../seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java index 2b69fd64097..704e38509e4 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java @@ -76,7 +76,8 @@ public abstract class AbstractJdbcIT extends TestSuiteBase implements TestResour "bash", "-c", "mkdir -p /tmp/seatunnel/plugins/Jdbc/lib && cd /tmp/seatunnel/plugins/Jdbc/lib && wget " - + driverUrl()+ "&& wget https://repo1.maven.org/maven2/org/apache/hive/hive-service/3.1.3/hive-service-3.1.3.jar"); + + driverUrl() + + "&& wget https://repo1.maven.org/maven2/org/apache/hive/hive-service/3.1.3/hive-service-3.1.3.jar"); Assertions.assertEquals(0, extraCommands.getExitCode(), extraCommands.getStderr()); }; From d268f8b2e9a37fec13ef4b711c13112b0f4083aa Mon Sep 17 00:00:00 2001 From: yangpeng Date: Thu, 19 Oct 2023 22:27:06 +0800 Subject: [PATCH 29/58] :art: format --- .../seatunnel/jdbc/AbstractJdbcIT.java | 3 +-- .../connectors/seatunnel/jdbc/JdbcHiveIT.java | 18 +++++++++++++++++- 2 files changed, 18 insertions(+), 3 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java index 704e38509e4..2738f5ccc36 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-common/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/AbstractJdbcIT.java @@ -76,8 +76,7 @@ public abstract class AbstractJdbcIT extends TestSuiteBase implements TestResour "bash", "-c", "mkdir -p /tmp/seatunnel/plugins/Jdbc/lib && cd /tmp/seatunnel/plugins/Jdbc/lib && wget " - + driverUrl() - + "&& wget https://repo1.maven.org/maven2/org/apache/hive/hive-service/3.1.3/hive-service-3.1.3.jar"); + + driverUrl()); Assertions.assertEquals(0, extraCommands.getExitCode(), extraCommands.getStderr()); }; diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java index d1118e3f438..250d4819056 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java @@ -23,6 +23,10 @@ import org.apache.commons.lang3.tuple.Pair; +import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; +import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; +import org.junit.jupiter.api.Assertions; +import org.testcontainers.containers.Container; import org.testcontainers.containers.GenericContainer; import org.testcontainers.containers.output.Slf4jLogConsumer; import org.testcontainers.utility.DockerLoggerFactory; @@ -74,6 +78,18 @@ public class JdbcHiveIT extends AbstractJdbcIT { + " numeric_column NUMERIC(10, 2)" + ")"; + @TestContainerExtension + protected final ContainerExtendedFactory extendedFactory = + container -> { + Container.ExecResult extraCommands = + container.execInContainer( + "bash", + "-c", + "mkdir -p /tmp/seatunnel/plugins/Jdbc/lib && cd /tmp/seatunnel/plugins/Jdbc/lib && wget " + + driverUrl() + + "&& wget https://repo1.maven.org/maven2/org/apache/hive/hive-service/3.1.3/hive-service-3.1.3.jar"); + Assertions.assertEquals(0, extraCommands.getExitCode(), extraCommands.getStderr()); + }; @Override JdbcCase getJdbcCase() { Map containerEnv = new HashMap<>(); @@ -145,7 +161,7 @@ void compareResult() {} @Override String driverUrl() { - return "https://repo1.maven.org/maven2/org/apache/hive/hive-jdbc/3.1.2/hive-jdbc-3.1.2.jar"; + return "https://repo1.maven.org/maven2/org/apache/hive/hive-jdbc/3.1.2/hive-jdbc-3.1.3.jar"; } @Override From 14ead5e93f864a7af59fb5298609bd0b001ab7cf Mon Sep 17 00:00:00 2001 From: yangpeng Date: Thu, 19 Oct 2023 23:07:09 +0800 Subject: [PATCH 30/58] :art: format --- .../seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java index 250d4819056..45fcd05f77b 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java @@ -20,11 +20,11 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; import org.apache.seatunnel.common.utils.ExceptionUtils; +import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; +import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; import org.apache.commons.lang3.tuple.Pair; -import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; -import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; import org.junit.jupiter.api.Assertions; import org.testcontainers.containers.Container; import org.testcontainers.containers.GenericContainer; @@ -90,6 +90,7 @@ public class JdbcHiveIT extends AbstractJdbcIT { + "&& wget https://repo1.maven.org/maven2/org/apache/hive/hive-service/3.1.3/hive-service-3.1.3.jar"); Assertions.assertEquals(0, extraCommands.getExitCode(), extraCommands.getStderr()); }; + @Override JdbcCase getJdbcCase() { Map containerEnv = new HashMap<>(); From 4dd0dc3105195bf5e5beff6dfb41827ea2dad798 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Fri, 20 Oct 2023 12:49:17 +0800 Subject: [PATCH 31/58] Add Hive dependencies to pom.xml --- seatunnel-dist/pom.xml | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/seatunnel-dist/pom.xml b/seatunnel-dist/pom.xml index acc6a4fc321..1c5fe4a2af8 100644 --- a/seatunnel-dist/pom.xml +++ b/seatunnel-dist/pom.xml @@ -105,6 +105,7 @@ 3.1.4 1.11.271 4.1.89.Final + 3.1.3 @@ -634,6 +635,20 @@ optional provided + + + org.apache.hive + hive-jdbc + ${hive.jdbc.version} + provided + + + + org.apache.hive + hive-service + ${hive.jdbc.version} + provided + From 0079e49aeba95aa393571cf3cae84d644b43b3f4 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Fri, 20 Oct 2023 13:01:38 +0800 Subject: [PATCH 32/58] Added Apache Hive JDBC and Hive Service dependencies to the assembly-bin-ci.xml file, as they were missing. This change ensures that Hive-related tasks can be correctly executed without runtime errors due to missing libraries. --- seatunnel-dist/src/main/assembly/assembly-bin-ci.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml b/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml index 5c1171a82da..75f8f1ae173 100644 --- a/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml +++ b/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml @@ -203,6 +203,8 @@ com.teradata.jdbc:terajdbc4:jar com.amazon.redshift:redshift-jdbc42:jar net.snowflake.snowflake-jdbc:jar + org.apache.hive:hive-jdbc:jar + org.apache.hive:hive-service:jar ${artifact.file.name} /lib From be99e25a152c4239b3e1f369f5fb7365ef5f9542 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Fri, 20 Oct 2023 13:37:39 +0800 Subject: [PATCH 33/58] Update Hive JDBC driver URL in JdbcHiveIT test --- .../apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java index 45fcd05f77b..062ced27583 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java @@ -162,7 +162,7 @@ void compareResult() {} @Override String driverUrl() { - return "https://repo1.maven.org/maven2/org/apache/hive/hive-jdbc/3.1.2/hive-jdbc-3.1.3.jar"; + return "https://repo1.maven.org/maven2/org/apache/hive/hive-jdbc/3.1.3/hive-jdbc-3.1.3.jar"; } @Override From 077065f4975a980fb6b74571e5431cba80e3fbfc Mon Sep 17 00:00:00 2001 From: yangpeng Date: Fri, 20 Oct 2023 15:11:15 +0800 Subject: [PATCH 34/58] Update Hive JDBC driver URL in JdbcHiveIT test --- seatunnel-connectors-v2/connector-jdbc/pom.xml | 2 +- seatunnel-dist/pom.xml | 2 +- seatunnel-dist/src/main/assembly/assembly-bin-ci.xml | 2 +- .../connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml | 2 +- .../apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/pom.xml b/seatunnel-connectors-v2/connector-jdbc/pom.xml index 9c479a5bb81..0627be41f4f 100644 --- a/seatunnel-connectors-v2/connector-jdbc/pom.xml +++ b/seatunnel-connectors-v2/connector-jdbc/pom.xml @@ -248,7 +248,7 @@ org.apache.hive - hive-service + hive-exec diff --git a/seatunnel-dist/pom.xml b/seatunnel-dist/pom.xml index 1c5fe4a2af8..2b159735fe9 100644 --- a/seatunnel-dist/pom.xml +++ b/seatunnel-dist/pom.xml @@ -645,7 +645,7 @@ org.apache.hive - hive-service + hive-exec ${hive.jdbc.version} provided diff --git a/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml b/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml index 75f8f1ae173..a1374b832e1 100644 --- a/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml +++ b/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml @@ -204,7 +204,7 @@ com.amazon.redshift:redshift-jdbc42:jar net.snowflake.snowflake-jdbc:jar org.apache.hive:hive-jdbc:jar - org.apache.hive:hive-service:jar + org.apache.hive:hive-exec:jar ${artifact.file.name} /lib diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml index efcfa4c949e..81c6d85c51d 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml @@ -98,7 +98,7 @@ org.apache.hive - hive-service + hive-exec test diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java index 062ced27583..7e732b7e4d8 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java @@ -87,7 +87,7 @@ public class JdbcHiveIT extends AbstractJdbcIT { "-c", "mkdir -p /tmp/seatunnel/plugins/Jdbc/lib && cd /tmp/seatunnel/plugins/Jdbc/lib && wget " + driverUrl() - + "&& wget https://repo1.maven.org/maven2/org/apache/hive/hive-service/3.1.3/hive-service-3.1.3.jar"); + + "&& wget https://repo1.maven.org/maven2/org/apache/hive/hive-exec/3.1.3/hive-exec-3.1.3.jar"); Assertions.assertEquals(0, extraCommands.getExitCode(), extraCommands.getStderr()); }; From 355d63fa619c5d45589504331450c0a261029650 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Fri, 20 Oct 2023 15:18:29 +0800 Subject: [PATCH 35/58] Update Hive JDBC driver URL in JdbcHiveIT test --- seatunnel-connectors-v2/connector-jdbc/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/pom.xml b/seatunnel-connectors-v2/connector-jdbc/pom.xml index 0627be41f4f..684a256b3ca 100644 --- a/seatunnel-connectors-v2/connector-jdbc/pom.xml +++ b/seatunnel-connectors-v2/connector-jdbc/pom.xml @@ -159,7 +159,7 @@ org.apache.hive - hive-service + hive-exec ${hive.jdbc.version} provided From 24ef7ebbd16c409d9d6c87cdbd2c906032103ae7 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Fri, 20 Oct 2023 15:53:43 +0800 Subject: [PATCH 36/58] Update Hive JDBC driver URL in JdbcHiveIT test --- .../connector-jdbc/pom.xml | 10 ++++++ seatunnel-dist/pom.xml | 34 ++++++++++++------- .../src/main/assembly/assembly-bin-ci.xml | 1 + .../connector-jdbc-e2e-part-3/pom.xml | 5 +++ .../connectors/seatunnel/jdbc/JdbcHiveIT.java | 3 +- 5 files changed, 39 insertions(+), 14 deletions(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/pom.xml b/seatunnel-connectors-v2/connector-jdbc/pom.xml index 684a256b3ca..06d1e102bc6 100644 --- a/seatunnel-connectors-v2/connector-jdbc/pom.xml +++ b/seatunnel-connectors-v2/connector-jdbc/pom.xml @@ -163,6 +163,12 @@ ${hive.jdbc.version} provided + + org.apache.hive + hive-service + ${hive.jdbc.version} + provided + @@ -250,5 +256,9 @@ org.apache.hive hive-exec + + org.apache.hive + hive-service + diff --git a/seatunnel-dist/pom.xml b/seatunnel-dist/pom.xml index 2b159735fe9..c6dc518e07e 100644 --- a/seatunnel-dist/pom.xml +++ b/seatunnel-dist/pom.xml @@ -604,6 +604,27 @@ ${snowflake.version} provided + + + org.apache.hive + hive-jdbc + ${hive.jdbc.version} + provided + + + + org.apache.hive + hive-exec + ${hive.jdbc.version} + provided + + + + org.apache.hive + hive-service + ${hive.jdbc.version} + provided + @@ -636,19 +657,6 @@ provided - - org.apache.hive - hive-jdbc - ${hive.jdbc.version} - provided - - - - org.apache.hive - hive-exec - ${hive.jdbc.version} - provided - diff --git a/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml b/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml index a1374b832e1..e03f0442bee 100644 --- a/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml +++ b/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml @@ -205,6 +205,7 @@ net.snowflake.snowflake-jdbc:jar org.apache.hive:hive-jdbc:jar org.apache.hive:hive-exec:jar + org.apache.hive:hive-service:jar ${artifact.file.name} /lib diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml index 81c6d85c51d..9df8f151015 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml @@ -101,6 +101,11 @@ hive-exec test + + org.apache.hive + hive-service + test + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java index 7e732b7e4d8..23c04555152 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java @@ -87,7 +87,8 @@ public class JdbcHiveIT extends AbstractJdbcIT { "-c", "mkdir -p /tmp/seatunnel/plugins/Jdbc/lib && cd /tmp/seatunnel/plugins/Jdbc/lib && wget " + driverUrl() - + "&& wget https://repo1.maven.org/maven2/org/apache/hive/hive-exec/3.1.3/hive-exec-3.1.3.jar"); + + "&& wget https://repo1.maven.org/maven2/org/apache/hive/hive-exec/3.1.3/hive-exec-3.1.3.jar" + + "&& wget https://repo1.maven.org/maven2/org/apache/hive/hive-service/3.1.3/hive-service-3.1.3.jar"); Assertions.assertEquals(0, extraCommands.getExitCode(), extraCommands.getStderr()); }; From 9b04b1f13ebbb652dc13f4175a494317477522b8 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Fri, 20 Oct 2023 16:37:28 +0800 Subject: [PATCH 37/58] Update Hive JDBC driver URL in JdbcHiveIT test --- seatunnel-connectors-v2/connector-jdbc/pom.xml | 10 ++++++++++ seatunnel-dist/pom.xml | 7 +++++++ seatunnel-dist/src/main/assembly/assembly-bin-ci.xml | 1 + .../connector-jdbc-e2e-part-3/pom.xml | 7 +++++++ .../connectors/seatunnel/jdbc/JdbcHiveIT.java | 3 ++- 5 files changed, 27 insertions(+), 1 deletion(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/pom.xml b/seatunnel-connectors-v2/connector-jdbc/pom.xml index 06d1e102bc6..c9ba9622678 100644 --- a/seatunnel-connectors-v2/connector-jdbc/pom.xml +++ b/seatunnel-connectors-v2/connector-jdbc/pom.xml @@ -169,6 +169,12 @@ ${hive.jdbc.version} provided + + org.apache.httpcomponents + httpclient + 4.5.14 + provided + @@ -260,5 +266,9 @@ org.apache.hive hive-service + + org.apache.httpcomponents + httpclient + diff --git a/seatunnel-dist/pom.xml b/seatunnel-dist/pom.xml index c6dc518e07e..6c8c40ba790 100644 --- a/seatunnel-dist/pom.xml +++ b/seatunnel-dist/pom.xml @@ -625,6 +625,13 @@ ${hive.jdbc.version} provided + + + org.apache.httpcomponents + httpclient + 4.5.14 + provided + diff --git a/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml b/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml index e03f0442bee..1fe31bb30d8 100644 --- a/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml +++ b/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml @@ -206,6 +206,7 @@ org.apache.hive:hive-jdbc:jar org.apache.hive:hive-exec:jar org.apache.hive:hive-service:jar + org.apache.httpcomponents:httpclient:jar ${artifact.file.name} /lib diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml index 9df8f151015..91a5da4e3c4 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml @@ -106,6 +106,13 @@ hive-service test + + + org.apache.httpcomponents + httpclient + 4.5.14 + test + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java index 23c04555152..63dbb4694bf 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java @@ -88,7 +88,8 @@ public class JdbcHiveIT extends AbstractJdbcIT { "mkdir -p /tmp/seatunnel/plugins/Jdbc/lib && cd /tmp/seatunnel/plugins/Jdbc/lib && wget " + driverUrl() + "&& wget https://repo1.maven.org/maven2/org/apache/hive/hive-exec/3.1.3/hive-exec-3.1.3.jar" - + "&& wget https://repo1.maven.org/maven2/org/apache/hive/hive-service/3.1.3/hive-service-3.1.3.jar"); + + "&& wget https://repo1.maven.org/maven2/org/apache/hive/hive-service/3.1.3/hive-service-3.1.3.jar" + + "&& wget https://repo1.maven.org/maven2/org/apache/httpcomponents/httpclient/4.5.14/httpclient-4.5.14.jar"); Assertions.assertEquals(0, extraCommands.getExitCode(), extraCommands.getStderr()); }; From 05331c87e6b0fce4338da1f0d9125402aa54a6fb Mon Sep 17 00:00:00 2001 From: yangpeng Date: Fri, 20 Oct 2023 17:20:28 +0800 Subject: [PATCH 38/58] Update Hive JDBC driver URL in JdbcHiveIT test --- .../seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java index 63dbb4694bf..26d615fc083 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java @@ -86,10 +86,7 @@ public class JdbcHiveIT extends AbstractJdbcIT { "bash", "-c", "mkdir -p /tmp/seatunnel/plugins/Jdbc/lib && cd /tmp/seatunnel/plugins/Jdbc/lib && wget " - + driverUrl() - + "&& wget https://repo1.maven.org/maven2/org/apache/hive/hive-exec/3.1.3/hive-exec-3.1.3.jar" - + "&& wget https://repo1.maven.org/maven2/org/apache/hive/hive-service/3.1.3/hive-service-3.1.3.jar" - + "&& wget https://repo1.maven.org/maven2/org/apache/httpcomponents/httpclient/4.5.14/httpclient-4.5.14.jar"); + + driverUrl()); Assertions.assertEquals(0, extraCommands.getExitCode(), extraCommands.getStderr()); }; @@ -164,7 +161,7 @@ void compareResult() {} @Override String driverUrl() { - return "https://repo1.maven.org/maven2/org/apache/hive/hive-jdbc/3.1.3/hive-jdbc-3.1.3.jar"; + return "https://repo1.maven.org/maven2/org/apache/hive/hive-jdbc/3.1.3/hive-jdbc-3.1.3-standalone.jar"; } @Override From 3c5b455283af1e82947aa8fc9671800ac5234638 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Fri, 20 Oct 2023 17:27:38 +0800 Subject: [PATCH 39/58] Update Hive JDBC driver URL in JdbcHiveIT test --- .../apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java index 26d615fc083..f03989abf4d 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java @@ -86,7 +86,7 @@ public class JdbcHiveIT extends AbstractJdbcIT { "bash", "-c", "mkdir -p /tmp/seatunnel/plugins/Jdbc/lib && cd /tmp/seatunnel/plugins/Jdbc/lib && wget " - + driverUrl()); + + driverUrl() + "&& mkdir -p /lib && cd /lib && wget "+ driverUrl()); Assertions.assertEquals(0, extraCommands.getExitCode(), extraCommands.getStderr()); }; From be7b208debbdf6ecb5bd1a0fd22902be085c5197 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Fri, 20 Oct 2023 17:28:09 +0800 Subject: [PATCH 40/58] Update Hive JDBC driver URL in JdbcHiveIT test --- .../seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java index f03989abf4d..e981ce12777 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java @@ -86,7 +86,9 @@ public class JdbcHiveIT extends AbstractJdbcIT { "bash", "-c", "mkdir -p /tmp/seatunnel/plugins/Jdbc/lib && cd /tmp/seatunnel/plugins/Jdbc/lib && wget " - + driverUrl() + "&& mkdir -p /lib && cd /lib && wget "+ driverUrl()); + + driverUrl() + + "&& mkdir -p /lib && cd /lib && wget " + + driverUrl()); Assertions.assertEquals(0, extraCommands.getExitCode(), extraCommands.getStderr()); }; From 347b5d98edca0bda3443ef41f86522b44d6e1b04 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Fri, 20 Oct 2023 18:14:45 +0800 Subject: [PATCH 41/58] revert --- seatunnel-dist/pom.xml | 30 ------------------- .../src/main/assembly/assembly-bin-ci.xml | 4 --- .../connectors/seatunnel/jdbc/JdbcHiveIT.java | 18 ----------- 3 files changed, 52 deletions(-) diff --git a/seatunnel-dist/pom.xml b/seatunnel-dist/pom.xml index 6c8c40ba790..acc6a4fc321 100644 --- a/seatunnel-dist/pom.xml +++ b/seatunnel-dist/pom.xml @@ -105,7 +105,6 @@ 3.1.4 1.11.271 4.1.89.Final - 3.1.3 @@ -604,34 +603,6 @@ ${snowflake.version} provided - - - org.apache.hive - hive-jdbc - ${hive.jdbc.version} - provided - - - - org.apache.hive - hive-exec - ${hive.jdbc.version} - provided - - - - org.apache.hive - hive-service - ${hive.jdbc.version} - provided - - - - org.apache.httpcomponents - httpclient - 4.5.14 - provided - @@ -663,7 +634,6 @@ optional provided - diff --git a/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml b/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml index 1fe31bb30d8..5c1171a82da 100644 --- a/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml +++ b/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml @@ -203,10 +203,6 @@ com.teradata.jdbc:terajdbc4:jar com.amazon.redshift:redshift-jdbc42:jar net.snowflake.snowflake-jdbc:jar - org.apache.hive:hive-jdbc:jar - org.apache.hive:hive-exec:jar - org.apache.hive:hive-service:jar - org.apache.httpcomponents:httpclient:jar ${artifact.file.name} /lib diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java index e981ce12777..69542598d4d 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java @@ -20,13 +20,9 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; import org.apache.seatunnel.common.utils.ExceptionUtils; -import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; -import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; import org.apache.commons.lang3.tuple.Pair; -import org.junit.jupiter.api.Assertions; -import org.testcontainers.containers.Container; import org.testcontainers.containers.GenericContainer; import org.testcontainers.containers.output.Slf4jLogConsumer; import org.testcontainers.utility.DockerLoggerFactory; @@ -78,20 +74,6 @@ public class JdbcHiveIT extends AbstractJdbcIT { + " numeric_column NUMERIC(10, 2)" + ")"; - @TestContainerExtension - protected final ContainerExtendedFactory extendedFactory = - container -> { - Container.ExecResult extraCommands = - container.execInContainer( - "bash", - "-c", - "mkdir -p /tmp/seatunnel/plugins/Jdbc/lib && cd /tmp/seatunnel/plugins/Jdbc/lib && wget " - + driverUrl() - + "&& mkdir -p /lib && cd /lib && wget " - + driverUrl()); - Assertions.assertEquals(0, extraCommands.getExitCode(), extraCommands.getStderr()); - }; - @Override JdbcCase getJdbcCase() { Map containerEnv = new HashMap<>(); From 53d774c91cfd91c957515ce197db16f9323fa03a Mon Sep 17 00:00:00 2001 From: yangpeng Date: Fri, 20 Oct 2023 18:21:19 +0800 Subject: [PATCH 42/58] revert --- seatunnel-connectors-v2/connector-jdbc/pom.xml | 18 ------------------ .../connector-jdbc-e2e-part-3/pom.xml | 17 ----------------- 2 files changed, 35 deletions(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/pom.xml b/seatunnel-connectors-v2/connector-jdbc/pom.xml index c9ba9622678..16b472f11cc 100644 --- a/seatunnel-connectors-v2/connector-jdbc/pom.xml +++ b/seatunnel-connectors-v2/connector-jdbc/pom.xml @@ -157,24 +157,6 @@ ${hive.jdbc.version} provided - - org.apache.hive - hive-exec - ${hive.jdbc.version} - provided - - - org.apache.hive - hive-service - ${hive.jdbc.version} - provided - - - org.apache.httpcomponents - httpclient - 4.5.14 - provided - diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml index 91a5da4e3c4..1ec13aad68e 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml @@ -96,23 +96,6 @@ hive-jdbc test - - org.apache.hive - hive-exec - test - - - org.apache.hive - hive-service - test - - - - org.apache.httpcomponents - httpclient - 4.5.14 - test - From 7f39cf7d338065c56dd5158a546d0ed3993e942f Mon Sep 17 00:00:00 2001 From: yangpeng Date: Fri, 20 Oct 2023 18:26:17 +0800 Subject: [PATCH 43/58] revert --- seatunnel-connectors-v2/connector-jdbc/pom.xml | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/pom.xml b/seatunnel-connectors-v2/connector-jdbc/pom.xml index 16b472f11cc..3f309a5490f 100644 --- a/seatunnel-connectors-v2/connector-jdbc/pom.xml +++ b/seatunnel-connectors-v2/connector-jdbc/pom.xml @@ -240,17 +240,5 @@ org.apache.hive hive-jdbc - - org.apache.hive - hive-exec - - - org.apache.hive - hive-service - - - org.apache.httpcomponents - httpclient - From d92733b02e49ff151b1aaf07b24190352fe1f03a Mon Sep 17 00:00:00 2001 From: yangpeng Date: Fri, 20 Oct 2023 18:39:37 +0800 Subject: [PATCH 44/58] "Remove unnecessary comment in HiveTypeMapper" --- .../seatunnel/jdbc/internal/dialect/hive/HiveTypeMapper.java | 1 - 1 file changed, 1 deletion(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveTypeMapper.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveTypeMapper.java index 08be422ec24..0940b5a2c6f 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveTypeMapper.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveTypeMapper.java @@ -62,7 +62,6 @@ public class HiveTypeMapper implements JdbcDialectTypeMapper { // Complex Types private static final String HIVE_ARRAY = "ARRAY"; private static final String HIVE_MAP = "MAP"; - // float private static final String HIVE_STRUCT = "STRUCT"; private static final String HIVE_UNIONTYPE = "UNIONTYPE"; From f9850aaa5fe06ae7915cbbb8661aae1445300752 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Mon, 23 Oct 2023 13:48:20 +0800 Subject: [PATCH 45/58] :art: format --- .../seatunnel/jdbc/config/JdbcConnectionConfig.java | 8 +++----- .../connectors/seatunnel/jdbc/config/JdbcOptions.java | 2 -- .../seatunnel/jdbc/internal/dialect/JdbcDialect.java | 3 +-- .../connectors/seatunnel/jdbc/sink/JdbcSinkWriter.java | 3 --- 4 files changed, 4 insertions(+), 12 deletions(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcConnectionConfig.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcConnectionConfig.java index afbba2f7a47..a5a3acaeab6 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcConnectionConfig.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcConnectionConfig.java @@ -47,7 +47,6 @@ public class JdbcConnectionConfig implements Serializable { public int transactionTimeoutSec = JdbcOptions.TRANSACTION_TIMEOUT_SEC.defaultValue(); - public boolean useKerberos = JdbcOptions.USE_KERBEROS.defaultValue(); public String kerberosPrincipal; @@ -58,7 +57,6 @@ public class JdbcConnectionConfig implements Serializable { private Map properties; - public static JdbcConnectionConfig of(ReadonlyConfig config) { JdbcConnectionConfig.Builder builder = JdbcConnectionConfig.builder(); builder.url(config.get(JdbcOptions.URL)); @@ -78,7 +76,7 @@ public static JdbcConnectionConfig of(ReadonlyConfig config) { builder.useKerberos(config.get(JdbcOptions.USE_KERBEROS)); builder.kerberosPrincipal(config.get(JdbcOptions.KERBEROS_PRINCIPAL)); builder.kerberosKeytabPath(config.get(JdbcOptions.KERBEROS_KEYTAB_PATH)); - builder.kerberosKeytabPath(config.get(JdbcOptions.KERBEROS_KRB5_CONF_PATH)); + builder.krb5Path(config.get(JdbcOptions.KERBEROS_KRB5_CONF_PATH)); } config.getOptional(JdbcOptions.USER).ifPresent(builder::username); config.getOptional(JdbcOptions.PASSWORD).ifPresent(builder::password); @@ -165,7 +163,6 @@ public static final class Builder { private Map properties; - private Builder() {} public Builder url(String url) { @@ -233,7 +230,6 @@ public Builder transactionTimeoutSec(int transactionTimeoutSec) { return this; } - public Builder useKerberos(boolean useKerberos) { this.useKerberos = useKerberos; return this; @@ -251,6 +247,8 @@ public Builder kerberosKeytabPath(String kerberosKeytabPath) { public Builder krb5Path(String krb5Path) { this.krb5Path = krb5Path; + return this; + } public Builder properties(Map properties) { this.properties = properties; diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java index d65cefa342b..c1e946d0e4f 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java @@ -163,7 +163,6 @@ public interface JdbcOptions { .noDefaultValue() .withDescription("Whether case conversion is required"); - Option USE_KERBEROS = Options.key("use_kerberos") .booleanType() @@ -196,5 +195,4 @@ public interface JdbcOptions { .mapType() .noDefaultValue() .withDescription("additional connection configuration parameters"); - } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java index 11c759ac5d0..11739a0736b 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java @@ -246,10 +246,10 @@ default String getFieldIde(String identifier, String fieldIde) { } } - default JdbcConnectionProvider getJdbcConnectionProvider( JdbcConnectionConfig jdbcConnectionConfig) { return new SimpleJdbcConnectionProvider(jdbcConnectionConfig); + } default Map defaultParameter() { return new HashMap<>(); @@ -263,6 +263,5 @@ default void connectionUrlParse( info.put(key, value); } }); - } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkWriter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkWriter.java index d4fbbf461f1..5b6682cf8e1 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkWriter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkWriter.java @@ -29,10 +29,7 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.JdbcOutputFormat; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.JdbcOutputFormatBuilder; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.JdbcConnectionProvider; - import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.SimpleJdbcConnectionPoolProviderProxy; -import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.SimpleJdbcConnectionProvider; - import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.executor.JdbcBatchStatementExecutor; import org.apache.seatunnel.connectors.seatunnel.jdbc.state.JdbcSinkState; From 87258426fe13f554928905fa25dd81bbb462a995 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Thu, 26 Oct 2023 17:15:32 +0800 Subject: [PATCH 46/58] resolve conflict --- docs/en/connector-v2/source/Hive-jdbc.md | 156 ++++++++++++++++ .../connector-jdbc/pom.xml | 22 ++- .../jdbc/config/JdbcConnectionConfig.java | 43 ++++- .../seatunnel/jdbc/config/JdbcOptions.java | 27 +++ .../exception/JdbcConnectorErrorCode.java | 5 +- .../SimpleJdbcConnectionProvider.java | 4 +- .../jdbc/internal/dialect/JdbcDialect.java | 8 + .../internal/dialect/hive/HiveDialect.java | 66 +++++++ .../dialect/hive/HiveDialectFactory.java | 38 ++++ .../hive/HiveJdbcConnectionProvider.java | 63 +++++++ .../dialect/hive/HiveJdbcRowConverter.java | 43 +++++ .../internal/dialect/hive/HiveTypeMapper.java | 120 ++++++++++++ .../seatunnel/jdbc/sink/JdbcSinkWriter.java | 3 +- .../seatunnel/jdbc/source/ChunkSplitter.java | 5 +- .../jdbc/utils/JdbcCatalogUtils.java | 8 +- .../connector-jdbc-e2e-part-3/pom.xml | 10 + .../connectors/seatunnel/jdbc/JdbcHiveIT.java | 173 ++++++++++++++++++ .../jdbc_hive_source_and_assert.conf | 141 ++++++++++++++ 18 files changed, 915 insertions(+), 20 deletions(-) create mode 100644 docs/en/connector-v2/source/Hive-jdbc.md create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialect.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialectFactory.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcRowConverter.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveTypeMapper.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_hive_source_and_assert.conf diff --git a/docs/en/connector-v2/source/Hive-jdbc.md b/docs/en/connector-v2/source/Hive-jdbc.md new file mode 100644 index 00000000000..0e3352014c2 --- /dev/null +++ b/docs/en/connector-v2/source/Hive-jdbc.md @@ -0,0 +1,156 @@ +# Hive + +> JDBC Hive Source Connector + +## Support Those Engines + +> Spark
+> Flink
+> SeaTunnel Zeta
+ +## Key Features + +- [x] [batch](../../concept/connector-v2-features.md) +- [ ] [stream](../../concept/connector-v2-features.md) +- [ ] [exactly-once](../../concept/connector-v2-features.md) +- [x] [column projection](../../concept/connector-v2-features.md) +- [x] [parallelism](../../concept/connector-v2-features.md) +- [x] [support user-defined split](../../concept/connector-v2-features.md) + +> supports query SQL and can achieve projection effect. + +## Description + +Read external data source data through JDBC. + +## Supported DataSource Info + +| Datasource | Supported versions | Driver | Url | Maven | +|------------|----------------------------------------------------------|---------------------------------|--------------------------------------|--------------------------------------------------------------------------| +| Hive | Different dependency version has different driver class. | org.apache.hive.jdbc.HiveDriver | jdbc:hive2://localhost:10000/default | [Download](https://mvnrepository.com/artifact/org.apache.hive/hive-jdbc) | + +## Database Dependency + +> Please download the support list corresponding to 'Maven' and copy it to the '$SEATNUNNEL_HOME/plugins/jdbc/lib/' +> working directory
+> For example Hive datasource: cp hive-jdbc-xxx.jar $SEATNUNNEL_HOME/plugins/jdbc/lib/ + +## Data Type Mapping + +| Hive Data type | SeaTunnel Data type | +|-------------------------------------------------------------------------------------------|---------------------| +| BOOLEAN | BOOLEAN | +| TINYINT
SMALLINT | SHORT | +| INT
INTEGER | INT | +| BIGINT | LONG | +| FLOAT | FLOAT | +| DOUBLE
DOUBLE PRECISION | DOUBLE | +| DECIMAL(x,y)
NUMERIC(x,y)
(Get the designated column's specified column size.<38) | DECIMAL(x,y) | +| DECIMAL(x,y)
NUMERIC(x,y)
(Get the designated column's specified column size.>38) | DECIMAL(38,18) | +| CHAR
VARCHAR
STRING | STRING | +| DATE | DATE | +| DATETIME
TIMESTAMP | TIMESTAMP | +| BINARY
ARRAY
INTERVAL
MAP
STRUCT
UNIONTYPE | Not supported yet | + +## Source Options + +| Name | Type | Required | Default | Description | +|------------------------------|------------|----------|-----------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| url | String | Yes | - | The URL of the JDBC connection. Refer to a case: jdbc:hive2://localhost:10000/default | +| driver | String | Yes | - | The jdbc class name used to connect to the remote data source,
if you use Hive the value is `org.apache.hive.jdbc.HiveDriver`. | +| user | String | No | - | Connection instance user name | +| password | String | No | - | Connection instance password | +| query | String | Yes | - | Query statement | +| connection_check_timeout_sec | Int | No | 30 | The time in seconds to wait for the database operation used to validate the connection to complete | +| partition_column | String | No | - | The column name for parallelism's partition, only support numeric type,Only support numeric type primary key, and only can config one column. | +| partition_lower_bound | BigDecimal | No | - | The partition_column min value for scan, if not set SeaTunnel will query database get min value. | +| partition_upper_bound | BigDecimal | No | - | The partition_column max value for scan, if not set SeaTunnel will query database get max value. | +| partition_num | Int | No | job parallelism | The number of partition count, only support positive integer. default value is job parallelism | +| fetch_size | Int | No | 0 | For queries that return a large number of objects,you can configure
the row fetch size used in the query toimprove performance by
reducing the number database hits required to satisfy the selection criteria.
Zero means use jdbc default value. | +| common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details | + +### Tips + +> If partition_column is not set, it will run in single concurrency, and if partition_column is set, it will be executed +> in parallel according to the concurrency of tasks , When your shard read field is a large number type such as bigint( +> and above and the data is not evenly distributed, it is recommended to set the parallelism level to 1 to ensure that +> the +> data skew problem is resolved + +## Task Example + +### Simple: + +> This example queries type_bin 'table' 16 data in your test "database" in single parallel and queries all of its +> fields. You can also specify which fields to query for final output to the console. + +``` +# Defining the runtime environment +env { + # You can set flink configuration here + execution.parallelism = 2 + job.mode = "BATCH" +} +source{ + Jdbc { + url = "jdbc:hive2://localhost:10000/default" + driver = "org.apache.hive.jdbc.HiveDriver" + connection_check_timeout_sec = 100 + query = "select * from type_bin limit 16" + } +} + +transform { + # If you would like to get more information about how to configure seatunnel and see full list of transform plugins, + # please go to https://seatunnel.apache.org/docs/transform-v2/sql +} + +sink { + Console {} +} +``` + +### Parallel: + +> Read your query table in parallel with the shard field you configured and the shard data You can do this if you want +> to read the whole table + +``` +source { + Jdbc { + url = "jdbc:hive2://localhost:10000/default" + driver = "org.apache.hive.jdbc.HiveDriver" + connection_check_timeout_sec = 100 + # Define query logic as required + query = "select * from type_bin" + # Parallel sharding reads fields + partition_column = "id" + # Number of fragments + partition_num = 10 + } +} +``` + +### Parallel Boundary: + +> It is more efficient to specify the data within the upper and lower bounds of the query It is more efficient to read +> your data source according to the upper and lower boundaries you configured + +``` +source { + Jdbc { + url = "jdbc:hive2://localhost:10000/default" + driver = "org.apache.hive.jdbc.HiveDriver" + connection_check_timeout_sec = 100 + # Define query logic as required + query = "select * from type_bin" + partition_column = "id" + # Read start boundary + partition_lower_bound = 1 + # Read end boundary + partition_upper_bound = 500 + partition_num = 10 + } +} +``` + diff --git a/seatunnel-connectors-v2/connector-jdbc/pom.xml b/seatunnel-connectors-v2/connector-jdbc/pom.xml index 7f210b338ea..f48277cbb35 100644 --- a/seatunnel-connectors-v2/connector-jdbc/pom.xml +++ b/seatunnel-connectors-v2/connector-jdbc/pom.xml @@ -48,6 +48,7 @@ 4.0.3 2.5.1 8.6.0 + 3.1.3 @@ -151,11 +152,16 @@ provided - cn.com.kingbase - kingbase8 - ${kingbase8.version} + org.apache.hive + hive-jdbc + ${hive.jdbc.version} provided + + org.apache.hive + hive-service + ${hive.jdbc.version} + @@ -237,11 +243,13 @@ com.vertica.jdbc vertica-jdbc - - cn.com.kingbase - kingbase8 + org.apache.hive + hive-jdbc + + + org.apache.hive + hive-service - diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcConnectionConfig.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcConnectionConfig.java index eeeff227b2f..2362f827b6b 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcConnectionConfig.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcConnectionConfig.java @@ -47,6 +47,14 @@ public class JdbcConnectionConfig implements Serializable { public int transactionTimeoutSec = JdbcOptions.TRANSACTION_TIMEOUT_SEC.defaultValue(); + public boolean useKerberos = JdbcOptions.USE_KERBEROS.defaultValue(); + + public String kerberosPrincipal; + + public String kerberosKeytabPath; + + public String krb5Path = JdbcOptions.KRB5_PATH.defaultValue(); + private Map properties; public static JdbcConnectionConfig of(ReadonlyConfig config) { @@ -64,7 +72,12 @@ public static JdbcConnectionConfig of(ReadonlyConfig config) { builder.transactionTimeoutSec(config.get(JdbcOptions.TRANSACTION_TIMEOUT_SEC)); builder.maxRetries(0); } - + if (config.get(JdbcOptions.USE_KERBEROS)) { + builder.useKerberos(config.get(JdbcOptions.USE_KERBEROS)); + builder.kerberosPrincipal(config.get(JdbcOptions.KERBEROS_PRINCIPAL)); + builder.kerberosKeytabPath(config.get(JdbcOptions.KERBEROS_KEYTAB_PATH)); + builder.kerberosKeytabPath(config.get(JdbcOptions.KRB5_PATH)); + } config.getOptional(JdbcOptions.USER).ifPresent(builder::username); config.getOptional(JdbcOptions.PASSWORD).ifPresent(builder::password); config.getOptional(JdbcOptions.PROPERTIES).ifPresent(builder::properties); @@ -143,6 +156,10 @@ public static final class Builder { private int maxCommitAttempts = JdbcOptions.MAX_COMMIT_ATTEMPTS.defaultValue(); private int transactionTimeoutSec = JdbcOptions.TRANSACTION_TIMEOUT_SEC.defaultValue(); private Map properties; + public boolean useKerberos = JdbcOptions.USE_KERBEROS.defaultValue(); + public String kerberosPrincipal; + public String kerberosKeytabPath; + public String krb5Path = JdbcOptions.KRB5_PATH.defaultValue(); private Builder() {} @@ -211,6 +228,26 @@ public Builder transactionTimeoutSec(int transactionTimeoutSec) { return this; } + public Builder useKerberos(boolean useKerberos) { + this.useKerberos = useKerberos; + return this; + } + + public Builder kerberosPrincipal(String kerberosPrincipal) { + this.kerberosPrincipal = kerberosPrincipal; + return this; + } + + public Builder kerberosKeytabPath(String kerberosKeytabPath) { + this.kerberosKeytabPath = kerberosKeytabPath; + return this; + } + + public Builder krb5Path(String krb5Path) { + this.krb5Path = krb5Path; + return this; + } + public Builder properties(Map properties) { this.properties = properties; return this; @@ -230,6 +267,10 @@ public JdbcConnectionConfig build() { jdbcConnectionConfig.transactionTimeoutSec = this.transactionTimeoutSec; jdbcConnectionConfig.maxCommitAttempts = this.maxCommitAttempts; jdbcConnectionConfig.xaDataSourceClassName = this.xaDataSourceClassName; + jdbcConnectionConfig.useKerberos = this.useKerberos; + jdbcConnectionConfig.kerberosPrincipal = this.kerberosPrincipal; + jdbcConnectionConfig.kerberosKeytabPath = this.kerberosKeytabPath; + jdbcConnectionConfig.krb5Path = this.krb5Path; jdbcConnectionConfig.properties = this.properties == null ? new HashMap<>() : this.properties; return jdbcConnectionConfig; diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java index 91ba5e2030f..77b72bd9424 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java @@ -163,6 +163,33 @@ public interface JdbcOptions { .noDefaultValue() .withDescription("Whether case conversion is required"); + Option USE_KERBEROS = + Options.key("use_kerberos") + .booleanType() + .defaultValue(false) + .withDescription("Whether to enable Kerberos, default is false."); + + Option KERBEROS_PRINCIPAL = + Options.key("kerberos_principal") + .stringType() + .noDefaultValue() + .withDescription( + "When use kerberos, we should set kerberos principal such as 'test_user@xxx'. "); + + Option KERBEROS_KEYTAB_PATH = + Options.key("kerberos_keytab_path") + .stringType() + .noDefaultValue() + .withDescription( + "When use kerberos, we should set kerberos principal file path such as '/home/test/test_user.keytab'. "); + + Option KRB5_PATH = + Options.key("krb5_path") + .stringType() + .defaultValue("/etc/krb5.conf") + .withDescription( + "When use kerberos, we should set krb5 path file path such as '/etc/krb5.conf'. "); + Option> PROPERTIES = Options.key("properties") .mapType() diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/exception/JdbcConnectorErrorCode.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/exception/JdbcConnectorErrorCode.java index 3d53b102bdc..22438de84c3 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/exception/JdbcConnectorErrorCode.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/exception/JdbcConnectorErrorCode.java @@ -27,8 +27,9 @@ public enum JdbcConnectorErrorCode implements SeaTunnelErrorCode { CONNECT_DATABASE_FAILED("JDBC-04", "Connector database failed"), TRANSACTION_OPERATION_FAILED( "JDBC-05", "transaction operation failed, such as (commit, rollback) etc.."), - NO_SUITABLE_DIALECT_FACTORY("JDBC-06", "No suitable dialect factory found"); - + NO_SUITABLE_DIALECT_FACTORY("JDBC-06", "No suitable dialect factory found"), + DONT_SUPPORT_SINK("JDBC-07", "The jdbc type don't support sink"), + KERBEROS_AUTHENTICATION_FAILED("JDBC-08", "Kerberos authentication failed"); private final String code; private final String description; diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/connection/SimpleJdbcConnectionProvider.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/connection/SimpleJdbcConnectionProvider.java index 40b75ced6af..50441d3f8a4 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/connection/SimpleJdbcConnectionProvider.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/connection/SimpleJdbcConnectionProvider.java @@ -24,6 +24,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import lombok.Data; import lombok.NonNull; import java.io.Serializable; @@ -37,6 +38,7 @@ import static com.google.common.base.Preconditions.checkNotNull; /** Simple JDBC connection provider. */ +@Data public class SimpleJdbcConnectionProvider implements JdbcConnectionProvider, Serializable { private static final Logger LOG = LoggerFactory.getLogger(SimpleJdbcConnectionProvider.class); @@ -88,7 +90,7 @@ private static Driver loadDriver(String driverName) throws ClassNotFoundExceptio } } - private Driver getLoadedDriver() throws SQLException, ClassNotFoundException { + protected Driver getLoadedDriver() throws SQLException, ClassNotFoundException { if (loadedDriver == null) { loadedDriver = loadDriver(jdbcConfig.getDriverName()); } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java index 0f554df3e88..aa7c03ccdb9 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java @@ -18,6 +18,9 @@ package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect; import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcConnectionConfig; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.JdbcConnectionProvider; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.SimpleJdbcConnectionProvider; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.converter.JdbcRowConverter; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.dialectenum.FieldIdeEnum; import org.apache.seatunnel.connectors.seatunnel.jdbc.source.JdbcSourceTable; @@ -391,4 +394,9 @@ default Object queryNextChunkMax( } } } + + default JdbcConnectionProvider getJdbcConnectionProvider( + JdbcConnectionConfig jdbcConnectionConfig) { + return new SimpleJdbcConnectionProvider(jdbcConnectionConfig); + } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialect.java new file mode 100644 index 00000000000..08e68632f7e --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialect.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.hive; + +import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcConnectionConfig; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.JdbcConnectionProvider; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.converter.JdbcRowConverter; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.DatabaseIdentifier; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectTypeMapper; + +import java.sql.Connection; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.util.Optional; + +public class HiveDialect implements JdbcDialect { + + @Override + public String dialectName() { + return DatabaseIdentifier.HIVE; + } + + @Override + public JdbcRowConverter getRowConverter() { + return new HiveJdbcRowConverter(); + } + + @Override + public JdbcDialectTypeMapper getJdbcDialectTypeMapper() { + return new HiveTypeMapper(); + } + + @Override + public Optional getUpsertStatement( + String database, String tableName, String[] fieldNames, String[] uniqueKeyFields) { + return Optional.empty(); + } + + @Override + public ResultSetMetaData getResultSetMetaData(Connection conn, String query) + throws SQLException { + return conn.prepareStatement(query).executeQuery().getMetaData(); + } + + @Override + public JdbcConnectionProvider getJdbcConnectionProvider( + JdbcConnectionConfig jdbcConnectionConfig) { + return new HiveJdbcConnectionProvider(jdbcConnectionConfig); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialectFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialectFactory.java new file mode 100644 index 00000000000..56bd81b7f83 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveDialectFactory.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.hive; + +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectFactory; + +import com.google.auto.service.AutoService; + +/** Factory for {@link HiveDialect}. */ +@AutoService(JdbcDialectFactory.class) +public class HiveDialectFactory implements JdbcDialectFactory { + + @Override + public boolean acceptsURL(String url) { + return url.startsWith("jdbc:hive2:"); + } + + @Override + public JdbcDialect create() { + return new HiveDialect(); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java new file mode 100644 index 00000000000..ae8a5f90166 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java @@ -0,0 +1,63 @@ +package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.hive; + +import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcConnectionConfig; +import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorErrorCode; +import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorException; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.SimpleJdbcConnectionProvider; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; + +import lombok.NonNull; + +import java.io.IOException; +import java.sql.Connection; +import java.sql.Driver; +import java.sql.SQLException; +import java.util.Properties; + +import static org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorErrorCode.KERBEROS_AUTHENTICATION_FAILED; + +public class HiveJdbcConnectionProvider extends SimpleJdbcConnectionProvider { + + public HiveJdbcConnectionProvider(@NonNull JdbcConnectionConfig jdbcConfig) { + super(jdbcConfig); + } + + @Override + public Connection getOrEstablishConnection() throws SQLException, ClassNotFoundException { + if (isConnectionValid()) { + return super.getConnection(); + } + JdbcConnectionConfig jdbcConfig = super.getJdbcConfig(); + if (jdbcConfig.useKerberos) { + System.setProperty("java.security.krb5.conf", jdbcConfig.krb5Path); + Configuration configuration = new Configuration(); + configuration.set("hadoop.security.authentication", "kerberos"); + UserGroupInformation.setConfiguration(configuration); + try { + UserGroupInformation.loginUserFromKeytab( + jdbcConfig.kerberosPrincipal, jdbcConfig.kerberosKeytabPath); + } catch (IOException e) { + throw new JdbcConnectorException(KERBEROS_AUTHENTICATION_FAILED, e); + } + } + Driver driver = getLoadedDriver(); + Properties info = new Properties(); + if (super.getJdbcConfig().getUsername().isPresent()) { + info.setProperty("user", super.getJdbcConfig().getUsername().get()); + } + if (super.getJdbcConfig().getPassword().isPresent()) { + info.setProperty("password", super.getJdbcConfig().getPassword().get()); + } + super.setConnection(driver.connect(super.getJdbcConfig().getUrl(), info)); + if (super.getConnection() == null) { + // Throw same exception as DriverManager.getConnection when no driver found to match + // caller expectation. + throw new JdbcConnectorException( + JdbcConnectorErrorCode.NO_SUITABLE_DRIVER, + "No suitable driver found for " + super.getJdbcConfig().getUrl()); + } + return super.getConnection(); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcRowConverter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcRowConverter.java new file mode 100644 index 00000000000..91ed90105c5 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcRowConverter.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.hive; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorErrorCode; +import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorException; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.converter.AbstractJdbcRowConverter; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.DatabaseIdentifier; + +import java.sql.PreparedStatement; + +public class HiveJdbcRowConverter extends AbstractJdbcRowConverter { + + @Override + public String converterName() { + return DatabaseIdentifier.HIVE; + } + + @Override + public PreparedStatement toExternal( + SeaTunnelRowType rowType, SeaTunnelRow row, PreparedStatement statement) { + throw new JdbcConnectorException( + JdbcConnectorErrorCode.DONT_SUPPORT_SINK, + "The Hive jdbc connector don't support sink"); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveTypeMapper.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveTypeMapper.java new file mode 100644 index 00000000000..0940b5a2c6f --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveTypeMapper.java @@ -0,0 +1,120 @@ +/* + * 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.jdbc.internal.dialect.hive; + +import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.api.table.type.DecimalType; +import org.apache.seatunnel.api.table.type.LocalTimeType; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.common.exception.CommonErrorCode; +import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorException; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectTypeMapper; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.ResultSetMetaData; +import java.sql.SQLException; + +public class HiveTypeMapper implements JdbcDialectTypeMapper { + + private static final Logger LOG = LoggerFactory.getLogger(HiveTypeMapper.class); + + // reference https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Types + + // Numeric Types + private static final String HIVE_TINYINT = "TINYINT"; + private static final String HIVE_SMALLINT = "SMALLINT"; + private static final String HIVE_INT = "INT"; + private static final String HIVE_INTEGER = "INTEGER"; + private static final String HIVE_BIGINT = "BIGINT"; + private static final String HIVE_FLOAT = "FLOAT"; + private static final String HIVE_DOUBLE = "DOUBLE"; + private static final String HIVE_DOUBLE_PRECISION = "DOUBLE PRECISION"; + private static final String HIVE_DECIMAL = "DECIMAL"; + private static final String HIVE_NUMERIC = "NUMERIC"; + // Date/Time Types + private static final String HIVE_TIMESTAMP = "TIMESTAMP"; + private static final String HIVE_DATE = "DATE"; + private static final String HIVE_INTERVAL = "INTERVAL"; + // String Types + private static final String HIVE_STRING = "STRING"; + private static final String HIVE_VARCHAR = "VARCHAR"; + private static final String HIVE_CHAR = "CHAR"; + // Misc Types + private static final String HIVE_BOOLEAN = "BOOLEAN"; + private static final String HIVE_BINARY = "BINARY"; + // Complex Types + private static final String HIVE_ARRAY = "ARRAY"; + private static final String HIVE_MAP = "MAP"; + private static final String HIVE_STRUCT = "STRUCT"; + private static final String HIVE_UNIONTYPE = "UNIONTYPE"; + + @Override + public SeaTunnelDataType mapping(ResultSetMetaData metadata, int colIndex) + throws SQLException { + String columnType = metadata.getColumnTypeName(colIndex).toUpperCase(); + int precision = metadata.getPrecision(colIndex); + switch (columnType) { + case HIVE_TINYINT: + return BasicType.BYTE_TYPE; + case HIVE_SMALLINT: + return BasicType.SHORT_TYPE; + case HIVE_INT: + case HIVE_INTEGER: + return BasicType.INT_TYPE; + case HIVE_BIGINT: + return BasicType.LONG_TYPE; + case HIVE_FLOAT: + return BasicType.FLOAT_TYPE; + case HIVE_DOUBLE: + case HIVE_DOUBLE_PRECISION: + return BasicType.DOUBLE_TYPE; + case HIVE_DECIMAL: + case HIVE_NUMERIC: + if (precision > 0) { + return new DecimalType(precision, metadata.getScale(colIndex)); + } + LOG.warn("decimal did define precision,scale, will be Decimal(38,18)"); + return new DecimalType(38, 18); + case HIVE_TIMESTAMP: + return LocalTimeType.LOCAL_DATE_TIME_TYPE; + case HIVE_DATE: + return LocalTimeType.LOCAL_DATE_TYPE; + case HIVE_STRING: + case HIVE_VARCHAR: + case HIVE_CHAR: + return BasicType.STRING_TYPE; + case HIVE_BOOLEAN: + return BasicType.BOOLEAN_TYPE; + case HIVE_BINARY: + case HIVE_ARRAY: + case HIVE_INTERVAL: + case HIVE_MAP: + case HIVE_STRUCT: + case HIVE_UNIONTYPE: + default: + final String jdbcColumnName = metadata.getColumnName(colIndex); + throw new JdbcConnectorException( + CommonErrorCode.UNSUPPORTED_OPERATION, + String.format( + "Doesn't support hive type '%s' on column '%s' yet.", + columnType, jdbcColumnName)); + } + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkWriter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkWriter.java index 376515d91e1..5b6682cf8e1 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkWriter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkWriter.java @@ -30,7 +30,6 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.JdbcOutputFormatBuilder; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.JdbcConnectionProvider; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.SimpleJdbcConnectionPoolProviderProxy; -import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.SimpleJdbcConnectionProvider; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.executor.JdbcBatchStatementExecutor; import org.apache.seatunnel.connectors.seatunnel.jdbc.state.JdbcSinkState; @@ -68,7 +67,7 @@ public JdbcSinkWriter( this.rowType = rowType; this.primaryKeyIndex = primaryKeyIndex; this.connectionProvider = - new SimpleJdbcConnectionProvider(jdbcSinkConfig.getJdbcConnectionConfig()); + dialect.getJdbcConnectionProvider(jdbcSinkConfig.getJdbcConnectionConfig()); this.outputFormat = new JdbcOutputFormatBuilder(dialect, connectionProvider, jdbcSinkConfig, rowType) .build(); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/ChunkSplitter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/ChunkSplitter.java index 720bbfdbeb3..355e95cc81a 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/ChunkSplitter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/ChunkSplitter.java @@ -28,7 +28,6 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceConfig; import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorException; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.JdbcConnectionProvider; -import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.SimpleJdbcConnectionProvider; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectLoader; @@ -64,11 +63,11 @@ public ChunkSplitter(JdbcSourceConfig config) { this.config = config; this.autoCommit = config.getJdbcConnectionConfig().isAutoCommit(); this.fetchSize = config.getFetchSize(); - this.connectionProvider = - new SimpleJdbcConnectionProvider(config.getJdbcConnectionConfig()); this.jdbcDialect = JdbcDialectLoader.load( config.getJdbcConnectionConfig().getUrl(), config.getCompatibleMode()); + this.connectionProvider = + jdbcDialect.getJdbcConnectionProvider(config.getJdbcConnectionConfig()); } public static ChunkSplitter create(JdbcSourceConfig config) { diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/utils/JdbcCatalogUtils.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/utils/JdbcCatalogUtils.java index a0863267646..306a0552cf8 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/utils/JdbcCatalogUtils.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/utils/JdbcCatalogUtils.java @@ -32,7 +32,7 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.utils.CatalogUtils; import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcConnectionConfig; import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceTableConfig; -import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.SimpleJdbcConnectionProvider; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.connection.JdbcConnectionProvider; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectLoader; import org.apache.seatunnel.connectors.seatunnel.jdbc.source.JdbcSourceTable; @@ -101,7 +101,7 @@ public static Map getTables( log.warn( "Catalog not found, loading tables from jdbc directly. url : {}", jdbcConnectionConfig.getUrl()); - try (Connection connection = getConnection(jdbcConnectionConfig)) { + try (Connection connection = getConnection(jdbcConnectionConfig, jdbcDialect)) { log.info("Loading catalog tables for jdbc : {}", jdbcConnectionConfig.getUrl()); for (JdbcSourceTableConfig tableConfig : tablesConfig) { CatalogTable catalogTable = getCatalogTable(tableConfig, connection, jdbcDialect); @@ -316,9 +316,9 @@ private static CatalogTable getCatalogTable( resultSetMetaData, jdbcDialect.getJdbcDialectTypeMapper()); } - private static Connection getConnection(JdbcConnectionConfig config) + private static Connection getConnection(JdbcConnectionConfig config, JdbcDialect jdbcDialect) throws SQLException, ClassNotFoundException { - SimpleJdbcConnectionProvider connectionProvider = new SimpleJdbcConnectionProvider(config); + JdbcConnectionProvider connectionProvider = jdbcDialect.getJdbcConnectionProvider(config); return connectionProvider.getOrEstablishConnection(); } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml index 8628e2b80b6..efcfa4c949e 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml @@ -91,6 +91,16 @@ vertica-jdbc test + + org.apache.hive + hive-jdbc + test + + + org.apache.hive + hive-service + test + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java new file mode 100644 index 00000000000..d1118e3f438 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java @@ -0,0 +1,173 @@ +/* + * 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.jdbc; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; +import org.apache.seatunnel.common.utils.ExceptionUtils; + +import org.apache.commons.lang3.tuple.Pair; + +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.utility.DockerLoggerFactory; + +import com.google.common.collect.Lists; +import lombok.extern.slf4j.Slf4j; + +import java.sql.Statement; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +@Slf4j +public class JdbcHiveIT extends AbstractJdbcIT { + + private static final String HIVE_IMAGE = "apache/hive:3.1.3"; + private static final String HIVE_CONTAINER_HOST = "e2ehivejdbc"; + + private static final String HIVE_DATABASE = "default"; + + private static final String HIVE_SOURCE = "e2e_table_source"; + private static final String HIVE_USERNAME = "root"; + private static final String HIVE_PASSWORD = null; + private static final int HIVE_PORT = 10000; + private static final String HIVE_URL = "jdbc:hive2://" + HOST + ":%s/%s"; + + private static final String DRIVER_CLASS = "org.apache.hive.jdbc.HiveDriver"; + + private static final List CONFIG_FILE = + Lists.newArrayList("/jdbc_hive_source_and_assert.conf"); + private static final String CREATE_SQL = + "CREATE TABLE hive_e2e_source_table" + + "(" + + " int_column INT," + + " integer_column INTEGER," + + " bigint_column BIGINT," + + " smallint_column SMALLINT," + + " tinyint_column TINYINT," + + " double_column DOUBLE," + + " double_PRECISION_column DOUBLE PRECISION," + + " float_column FLOAT," + + " string_column STRING," + + " char_column CHAR(10)," + + " varchar_column VARCHAR(20)," + + " boolean_column BOOLEAN," + + " date_column DATE," + + " timestamp_column TIMESTAMP," + + " decimal_column DECIMAL(10, 2)," + + " numeric_column NUMERIC(10, 2)" + + ")"; + + @Override + JdbcCase getJdbcCase() { + Map containerEnv = new HashMap<>(); + String jdbcUrl = String.format(HIVE_URL, HIVE_PORT, HIVE_DATABASE); + return JdbcCase.builder() + .dockerImage(HIVE_IMAGE) + .networkAliases(HIVE_CONTAINER_HOST) + .containerEnv(containerEnv) + .driverClass(DRIVER_CLASS) + .host(HOST) + .port(HIVE_PORT) + .localPort(HIVE_PORT) + .jdbcTemplate(HIVE_URL) + .jdbcUrl(jdbcUrl) + .userName(HIVE_USERNAME) + .password(HIVE_PASSWORD) + .database(HIVE_DATABASE) + .sourceTable(HIVE_SOURCE) + .createSql(CREATE_SQL) + .configFile(CONFIG_FILE) + .build(); + } + + protected void createNeededTables() { + try (Statement statement = connection.createStatement()) { + String createTemplate = jdbcCase.getCreateSql(); + String createSource = + String.format( + createTemplate, + buildTableInfoWithSchema( + jdbcCase.getDatabase(), jdbcCase.getSourceTable())); + statement.execute(createSource); + } catch (Exception exception) { + log.error(ExceptionUtils.getMessage(exception)); + throw new SeaTunnelRuntimeException(JdbcITErrorCode.CREATE_TABLE_FAILED, exception); + } + } + + protected void insertTestData() { + try (Statement statement = connection.createStatement()) { + for (int i = 1; i <= 3; i++) { + statement.execute( + "INSERT INTO hive_e2e_source_table " + + "VALUES (2," + + " 1," + + " 1234567890," + + " 32767," + + " 127," + + " 123.45," + + " 123.45," + + " 67.89," + + " 'Hello, Hive'," + + " 'CharCol'," + + " 'VarcharCol'," + + " TRUE," + + " '2023-09-04'," + + " '2023-09-04 10:30:00'," + + " 42.12," + + " 42.12)"); + } + } catch (Exception exception) { + log.error(ExceptionUtils.getMessage(exception)); + throw new SeaTunnelRuntimeException(JdbcITErrorCode.INSERT_DATA_FAILED, exception); + } + } + + @Override + void compareResult() {} + + @Override + String driverUrl() { + return "https://repo1.maven.org/maven2/org/apache/hive/hive-jdbc/3.1.2/hive-jdbc-3.1.2.jar"; + } + + @Override + Pair> initTestData() { + return null; + } + + @Override + GenericContainer initContainer() { + GenericContainer container = + new GenericContainer<>(HIVE_IMAGE) + .withExposedPorts(HIVE_PORT) + .withNetwork(NETWORK) + .withNetworkAliases(HIVE_CONTAINER_HOST) + .withEnv("SERVICE_NAME", "hiveserver2") + .withLogConsumer( + new Slf4jLogConsumer(DockerLoggerFactory.getLogger(HIVE_IMAGE))); + container.setPortBindings(Lists.newArrayList(String.format("%s:%s", HIVE_PORT, HIVE_PORT))); + return container; + } + + public void clearTable(String schema, String table) { + // do nothing. + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_hive_source_and_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_hive_source_and_assert.conf new file mode 100644 index 00000000000..04b0240a3bf --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_hive_source_and_assert.conf @@ -0,0 +1,141 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + # You can set engine configuration here + execution.parallelism = 1 + job.mode = "BATCH" + checkpoint.interval = 5000 + #execution.checkpoint.data-uri = "hdfs://localhost:9000/checkpoint" +} + +source { + # This is a example source plugin **only for test and demonstrate the feature source plugin** + Jdbc { + url = "jdbc:hive2://e2ehivejdbc:10000/default" + user = "root" + driver = "org.apache.hive.jdbc.HiveDriver" + query = "select * from hive_e2e_source_table" + auto_commit= false + } +} + +transform { +} + +sink{ + assert { + rules = + { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 3 + }, + { + rule_type = MIN_ROW + rule_value = 3 + } + ], + field_rules = [ + { + field_name = hive_e2e_source_table.int_column + field_type = int + field_value = [{equals_to = 2}] + }, + { + field_name = hive_e2e_source_table.integer_column + field_type = int + field_value = [{equals_to = 1}] + }, + { + field_name = hive_e2e_source_table.bigint_column + field_type = long + field_value = [{equals_to = 1234567890}] + }, + { + field_name = hive_e2e_source_table.smallint_column + field_type = short + field_value = [{equals_to = 32767}] + }, + { + field_name = hive_e2e_source_table.tinyint_column + field_type = byte + field_value = [{equals_to = 127}] + }, + { + field_name = hive_e2e_source_table.double_column + field_type = double + field_value = [{equals_to = 123.45}] + }, + { + field_name = hive_e2e_source_table.double_precision_column + field_type = double + field_value = [{equals_to = 123.45}] + }, + { + field_name = hive_e2e_source_table.float_column + field_type = float + field_value = [{equals_to = 67.89}] + }, + { + field_name = hive_e2e_source_table.string_column + field_type = string + field_value = [{equals_to = "Hello, Hive"}] + }, + { + field_name = hive_e2e_source_table.char_column + field_type = string + field_value = [{equals_to = "CharCol "}] + }, + { + field_name = hive_e2e_source_table.varchar_column + field_type = string + field_value = [{equals_to = "VarcharCol"}] + }, + { + field_name = hive_e2e_source_table.boolean_column + field_type = boolean + field_value = [{equals_to = "TRUE"}] + }, + { + field_name = hive_e2e_source_table.date_column + field_type = date + field_value = [{equals_to = "2023-09-04"}] + }, + { + field_name = hive_e2e_source_table.timestamp_column + field_type = timestamp + field_value = [{equals_to = "2023-09-04T10:30:00"}] + }, + { + field_name = hive_e2e_source_table.decimal_column + field_type = decimal + field_value = [{equals_to = 42.12}] + }, + { + field_name = hive_e2e_source_table.numeric_column + field_type = decimal + field_value = [{equals_to = 42.12}] + }, + ] + } + } +} From bbd2b3a4a0cfe9d16a6d673b75d4e9a21d798bd2 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Thu, 26 Oct 2023 18:01:55 +0800 Subject: [PATCH 47/58] :art: resolve conflict --- .../connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java | 1 - .../connectors/seatunnel/jdbc/source/JdbcSourceFactory.java | 2 -- 2 files changed, 3 deletions(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java index e1736550911..6e4e272328e 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java @@ -18,7 +18,6 @@ package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect; import org.apache.seatunnel.api.table.catalog.TablePath; - import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.converter.JdbcRowConverter; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.dialectenum.FieldIdeEnum; import org.apache.seatunnel.connectors.seatunnel.jdbc.source.JdbcSourceTable; diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceFactory.java index ac206650daf..54e8d5173b3 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceFactory.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSourceFactory.java @@ -25,7 +25,6 @@ import org.apache.seatunnel.api.table.factory.TableSourceFactory; import org.apache.seatunnel.api.table.factory.TableSourceFactoryContext; import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceConfig; - import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectLoader; @@ -75,7 +74,6 @@ TableSource createSource(TableSourceFactoryContext context) { jdbcDialect.connectionUrlParse( config.getJdbcConnectionConfig().getUrl(), config.getJdbcConnectionConfig().getProperties(), - jdbcDialect.defaultParameter()); return () -> (SeaTunnelSource) new JdbcSource(config); } From e300ea0366ff19bb55f8c49fa6a7bb11d0403747 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Thu, 26 Oct 2023 18:21:50 +0800 Subject: [PATCH 48/58] add license header --- .../dialect/hive/HiveJdbcConnectionProvider.java | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java index ae8a5f90166..1a45a8600a9 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/hive/HiveJdbcConnectionProvider.java @@ -1,3 +1,19 @@ +/* + * 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.jdbc.internal.dialect.hive; import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcConnectionConfig; From daec9a5fa6b4c99eee3b23566bc72715e3dc41f5 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Thu, 26 Oct 2023 18:30:25 +0800 Subject: [PATCH 49/58] :fire: --- seatunnel-connectors-v2/connector-jdbc/pom.xml | 4 ---- 1 file changed, 4 deletions(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/pom.xml b/seatunnel-connectors-v2/connector-jdbc/pom.xml index 0aa63e113ae..810948f2a0a 100644 --- a/seatunnel-connectors-v2/connector-jdbc/pom.xml +++ b/seatunnel-connectors-v2/connector-jdbc/pom.xml @@ -249,9 +249,5 @@ cn.com.kingbase kingbase8 - - org.apache.hive - hive-jdbc - From ecc0ae83dafb29a6d018bbe5c162693bab03139b Mon Sep 17 00:00:00 2001 From: yangpeng Date: Thu, 26 Oct 2023 18:36:09 +0800 Subject: [PATCH 50/58] :fire: --- .../apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java index d1118e3f438..69542598d4d 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcHiveIT.java @@ -145,7 +145,7 @@ void compareResult() {} @Override String driverUrl() { - return "https://repo1.maven.org/maven2/org/apache/hive/hive-jdbc/3.1.2/hive-jdbc-3.1.2.jar"; + return "https://repo1.maven.org/maven2/org/apache/hive/hive-jdbc/3.1.3/hive-jdbc-3.1.3-standalone.jar"; } @Override From cd396e48e5a93167c9ed142d033bca80024428d8 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Thu, 26 Oct 2023 19:47:05 +0800 Subject: [PATCH 51/58] format --- .../seatunnel/jdbc/internal/dialect/JdbcDialect.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java index c77ddb03412..aa7c03ccdb9 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java @@ -249,11 +249,6 @@ default String getFieldIde(String identifier, String fieldIde) { } } - default JdbcConnectionProvider getJdbcConnectionProvider( - JdbcConnectionConfig jdbcConnectionConfig) { - return new SimpleJdbcConnectionProvider(jdbcConnectionConfig); - } - default Map defaultParameter() { return new HashMap<>(); } From 5f0ca3a466c3fa96da0d2d510e5fc7d247343179 Mon Sep 17 00:00:00 2001 From: yangpeng Date: Thu, 26 Oct 2023 20:06:42 +0800 Subject: [PATCH 52/58] Add Hive JDBC dependency and update SimpleJdbcConnectionProvider This commit introduces the Apache Hive JDBC connector as a dependency in the POM.xml to support connectivity with Hive. It also removes @Data annotation from SimpleJdbcConnectionProvider and manually implements getters and setters. This was done to improve code clarity and ensure explicit control over the modifications to Connection object. --- seatunnel-connectors-v2/connector-jdbc/pom.xml | 4 ++++ .../connection/SimpleJdbcConnectionProvider.java | 10 ++++++++-- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/pom.xml b/seatunnel-connectors-v2/connector-jdbc/pom.xml index 810948f2a0a..0aa63e113ae 100644 --- a/seatunnel-connectors-v2/connector-jdbc/pom.xml +++ b/seatunnel-connectors-v2/connector-jdbc/pom.xml @@ -249,5 +249,9 @@ cn.com.kingbase kingbase8 + + org.apache.hive + hive-jdbc + diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/connection/SimpleJdbcConnectionProvider.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/connection/SimpleJdbcConnectionProvider.java index 75d3c2678fe..af329a99d4e 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/connection/SimpleJdbcConnectionProvider.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/connection/SimpleJdbcConnectionProvider.java @@ -24,7 +24,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import lombok.Data; import lombok.NonNull; import java.io.Serializable; @@ -38,7 +37,6 @@ import static com.google.common.base.Preconditions.checkNotNull; /** Simple JDBC connection provider. */ -@Data public class SimpleJdbcConnectionProvider implements JdbcConnectionProvider, Serializable { private static final Logger LOG = LoggerFactory.getLogger(SimpleJdbcConnectionProvider.class); @@ -143,4 +141,12 @@ public Connection reestablishConnection() throws SQLException, ClassNotFoundExce closeConnection(); return getOrEstablishConnection(); } + + public JdbcConnectionConfig getJdbcConfig() { + return jdbcConfig; + } + + public void setConnection(Connection connection) { + this.connection = connection; + } } From b78c44a1e48e4ec1aa359b4eb2339b962c74f05b Mon Sep 17 00:00:00 2001 From: yangpeng Date: Thu, 26 Oct 2023 20:30:02 +0800 Subject: [PATCH 53/58] Add Hive service as dependency in pom.xml This commit includes Hive service and Hive JDBC in the dependencies of the pom.xml file under seatunnel-connectors-v2/connector-jdbc. The Hive service has been added to expand the functionality of the connector-jdbc to support more databases, and enhance its capabilities of handling various types of databases. --- seatunnel-connectors-v2/connector-jdbc/pom.xml | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/seatunnel-connectors-v2/connector-jdbc/pom.xml b/seatunnel-connectors-v2/connector-jdbc/pom.xml index 0aa63e113ae..10c97022e34 100644 --- a/seatunnel-connectors-v2/connector-jdbc/pom.xml +++ b/seatunnel-connectors-v2/connector-jdbc/pom.xml @@ -163,6 +163,13 @@ ${hive.jdbc.version} provided + + + org.apache.hive + hive-service + ${hive.jdbc.version} + provided + @@ -253,5 +260,9 @@ org.apache.hive hive-jdbc + + org.apache.hive + hive-service + From 092c0b5a977f10db2cf3a768b21fc85bd5dbd31c Mon Sep 17 00:00:00 2001 From: yangpeng Date: Thu, 26 Oct 2023 21:02:11 +0800 Subject: [PATCH 54/58] Remove hive-service dependency from pom.xml This commit removes hive-service from the dependencies of the pom.xml file under seatunnel-connectors-v2/connector-jdbc. It also includes exclusions in the hive-jdbc dependency to prevent unnecessary inclusion of jdk.tools. These changes are done to avoid certain version conflict issues and to optimize the project's dependencies. --- seatunnel-connectors-v2/connector-jdbc/pom.xml | 17 ++++++----------- 1 file changed, 6 insertions(+), 11 deletions(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/pom.xml b/seatunnel-connectors-v2/connector-jdbc/pom.xml index 10c97022e34..c8d5e23d430 100644 --- a/seatunnel-connectors-v2/connector-jdbc/pom.xml +++ b/seatunnel-connectors-v2/connector-jdbc/pom.xml @@ -162,13 +162,12 @@ hive-jdbc ${hive.jdbc.version} provided - - - - org.apache.hive - hive-service - ${hive.jdbc.version} - provided + + + jdk.tools + jdk.tools + + @@ -260,9 +259,5 @@ org.apache.hive hive-jdbc - - org.apache.hive - hive-service - From c601e846ba5b264aad4d87eace48b080a6c26e2e Mon Sep 17 00:00:00 2001 From: Nick Young <72905543+NickYoungPeng@users.noreply.github.com> Date: Fri, 27 Oct 2023 23:35:28 +0800 Subject: [PATCH 55/58] Update dependency in connector-jdbc pom.xml Updated the dependency to the oceanbase-client from the com.oceanbase group in the connector-jdbc pom.xml file. This change is necessary to maintain API consistency and efficiency. --- seatunnel-connectors-v2/connector-jdbc/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/pom.xml b/seatunnel-connectors-v2/connector-jdbc/pom.xml index 6f695791e1c..5643152a8eb 100644 --- a/seatunnel-connectors-v2/connector-jdbc/pom.xml +++ b/seatunnel-connectors-v2/connector-jdbc/pom.xml @@ -169,8 +169,8 @@ jdk.tools - - + + com.oceanbase oceanbase-client ${oceanbase.jdbc.version} From 1db6ce0d756a78df61e600952d51ba42d03ef334 Mon Sep 17 00:00:00 2001 From: Nick Young <72905543+NickYoungPeng@users.noreply.github.com> Date: Mon, 30 Oct 2023 14:10:35 +0800 Subject: [PATCH 56/58] Add new error codes to Error Quick Reference Manual Introduced new error codes JDBC-07 and JDBC-08 in connection with the JDBC in the Error Quick Reference Manual. These additions are important to help users understand and diagnose the new issues related to unsupported jdbc type and failed Kerberos authentication respectively. --- docs/en/connector-v2/Error-Quick-Reference-Manual.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/connector-v2/Error-Quick-Reference-Manual.md b/docs/en/connector-v2/Error-Quick-Reference-Manual.md index 03c9412424c..b7e59b74cf5 100644 --- a/docs/en/connector-v2/Error-Quick-Reference-Manual.md +++ b/docs/en/connector-v2/Error-Quick-Reference-Manual.md @@ -194,6 +194,8 @@ problems encountered by users. | JDBC-04 | Connector database failed | When users encounter this error code, it means that database connection failure, check whether the url is correct or whether the corresponding service is normal | | JDBC-05 | transaction operation failed, such as (commit, rollback) etc.. | When users encounter this error code, it means that if a sql transaction fails, check the transaction execution of the corresponding database to determine the cause of the transaction failure | | JDBC-06 | No suitable dialect factory found | When users encounter this error code, it means that may be an unsupported dialect type | +| JDBC-07 | The jdbc type don't support sink | When users encounter this error code, it means that jdbc type don't support sink | +| JDBC-08 | Kerberos authentication failed | When users encounter this error code, it means that database connection Kerberos authentication failed | ## Pulsar Connector Error Codes From 4e00b74daea88ad97a28ad3e532e0010d98ad907 Mon Sep 17 00:00:00 2001 From: Nick Young <72905543+NickYoungPeng@users.noreply.github.com> Date: Mon, 30 Oct 2023 14:24:51 +0800 Subject: [PATCH 57/58] Update default krb5 path in jdbc configs Updated the default krb5 file path in jdbc configuration from '/etc/krb5.conf' to '/seatunnel/krb5.conf' or use the default path '/etc/krb5.conf'. This change was made to provide flexibility if the krb5.conf file is not found at the default location. The documentation has been updated to reflect this change --- docs/en/connector-v2/source/Hive-jdbc.md | 4 ++++ .../connectors/seatunnel/jdbc/config/JdbcOptions.java | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/en/connector-v2/source/Hive-jdbc.md b/docs/en/connector-v2/source/Hive-jdbc.md index 0e3352014c2..86f5b9e6e4c 100644 --- a/docs/en/connector-v2/source/Hive-jdbc.md +++ b/docs/en/connector-v2/source/Hive-jdbc.md @@ -68,6 +68,10 @@ Read external data source data through JDBC. | partition_num | Int | No | job parallelism | The number of partition count, only support positive integer. default value is job parallelism | | fetch_size | Int | No | 0 | For queries that return a large number of objects,you can configure
the row fetch size used in the query toimprove performance by
reducing the number database hits required to satisfy the selection criteria.
Zero means use jdbc default value. | | common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details | +| useKerberos | Boolean | No | no | Whether to enable Kerberos, default is false | +| kerberos_principal | String | No | - | When use kerberos, we should set kerberos principal such as 'test_user@xxx'. | +| kerberos_keytab_path | String | No | - | When use kerberos, we should set kerberos principal file path such as '/home/test/test_user.keytab' . | +| krb5_path | String | No | /etc/krb5.conf | When use kerberos, we should set krb5 path file path such as '/seatunnel/krb5.conf' or use the default path '/etc/krb5.conf '. | ### Tips diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java index 77b72bd9424..100a8493caf 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcOptions.java @@ -188,7 +188,7 @@ public interface JdbcOptions { .stringType() .defaultValue("/etc/krb5.conf") .withDescription( - "When use kerberos, we should set krb5 path file path such as '/etc/krb5.conf'. "); + "When use kerberos, we should set krb5 path file path such as '/seatunnel/krb5.conf' or use the default path '/etc/krb5.conf"); Option> PROPERTIES = Options.key("properties") From 6724258f71429e02d8cf18cb6b59fda6750b85f1 Mon Sep 17 00:00:00 2001 From: Nick Young <72905543+NickYoungPeng@users.noreply.github.com> Date: Tue, 31 Oct 2023 09:55:08 +0800 Subject: [PATCH 58/58] Add Hive version support information to docs Included definite support information for Hive version 3.1.3 and 3.1.2 in JDBC Hive Source Connector doc, while stating that other versions need testing. This provides clarity on version compatibility, helping users determine if their Hive version is definitely compatible or requires testing. --- docs/en/connector-v2/source/Hive-jdbc.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/connector-v2/source/Hive-jdbc.md b/docs/en/connector-v2/source/Hive-jdbc.md index 86f5b9e6e4c..9b4c4e11029 100644 --- a/docs/en/connector-v2/source/Hive-jdbc.md +++ b/docs/en/connector-v2/source/Hive-jdbc.md @@ -2,6 +2,10 @@ > JDBC Hive Source Connector +## Support Hive Version + +- Definitely supports 3.1.3 and 3.1.2, other versions need to be tested. + ## Support Those Engines > Spark