diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 044b24f919b1..0d7b56355e3f 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -517,6 +517,9 @@ jobs: - { modules: plugin/trino-snowflake } - { modules: plugin/trino-snowflake, profile: cloud-tests } - { modules: plugin/trino-sqlserver } + - { modules: plugin/trino-teradata } + - { modules: plugin/trino-teradata, profile: clearscape-tests } + - { modules: plugin/trino-teradata, profile: run-only-long-tests } - { modules: plugin/trino-vertica } - { modules: testing/trino-faulttolerant-tests, profile: default } - { modules: testing/trino-faulttolerant-tests, profile: test-fault-tolerant-delta } @@ -567,6 +570,8 @@ jobs: && ! (contains(matrix.modules, 'trino-filesystem-gcs') && contains(matrix.profile, 'cloud-tests')) && ! (contains(matrix.modules, 'trino-filesystem-s3') && contains(matrix.profile, 'cloud-tests')) && ! (contains(matrix.modules, 'trino-hdfs') && contains(matrix.profile, 'cloud-tests')) + && ! (contains(matrix.modules, 'trino-teradata') && contains(matrix.profile, 'clearscape-tests')) + && ! (contains(matrix.modules, 'trino-teradata') && contains(matrix.profile, 'run-only-long-tests')) run: $MAVEN test ${MAVEN_TEST} -pl ${{ matrix.modules }} ${{ matrix.profile != '' && format('-P {0}', matrix.profile) || '' }} # Additional tests for selected modules - name: HDFS file system cache isolated JVM tests @@ -792,6 +797,25 @@ jobs: # Cancelled workflows may have left the ephemeral cluster running if: always() run: .github/bin/redshift/delete-aws-redshift.sh + - name: Teradata Tests + id: tests-teradata + env: + CLEARSCAPE_TOKEN: ${{ secrets.CLEARSCAPE_TOKEN }} + CLEARSCAPE_PASSWORD: ${{ secrets.CLEARSCAPE_PASSWORD }} + CLEARSCAPE_REGION: ${{ vars.CLEARSCAPE_REGION }} + if: matrix.modules == 'plugin/trino-teradata' && contains(matrix.profile, 'clearscape-tests') && env.CLEARSCAPE_TOKEN != '' && env.CLEARSCAPE_PASSWORD != '' && env.CLEARSCAPE_REGION != '' + run: | + $MAVEN test ${MAVEN_TEST} -pl :trino-teradata -Pclearscape-tests + - name: Teradata Long running Tests + id: tests-long-run-teradata + env: + CLEARSCAPE_TOKEN: ${{ secrets.CLEARSCAPE_TOKEN }} + CLEARSCAPE_PASSWORD: ${{ secrets.CLEARSCAPE_PASSWORD }} + CLEARSCAPE_REGION: ${{ vars.CLEARSCAPE_REGION }} + TERADATA_LONG_TESTS: ${{ vars.TERADATA_LONG_TESTS }} + if: matrix.modules == 'plugin/trino-teradata' && contains(matrix.profile, 'run-only-long-tests') && env.CLEARSCAPE_TOKEN != '' && env.CLEARSCAPE_PASSWORD != '' && env.CLEARSCAPE_REGION != '' && env.TERADATA_LONG_TESTS == 'true' + run: | + $MAVEN test ${MAVEN_TEST} -pl :trino-teradata -Prun-only-long-tests - name: Sanitize artifact name if: always() run: | @@ -819,6 +843,8 @@ jobs: || steps.tests-bq-smoke.outcome == 'failure' || steps.tests-iceberg.outcome == 'failure' || steps.tests-redshift.outcome == 'failure' + || steps.tests-teradata.outcome == 'failure' + || steps.tests-long-run-teradata.outcome == 'failure' || steps.tests-snowflake.outcome == 'failure' }} upload-heap-dump: ${{ env.SECRETS_PRESENT == '' && github.event_name == 'pull_request' && github.event.pull_request.head.repo.full_name != github.repository }} diff --git a/core/trino-main/src/main/java/io/trino/sql/rewrite/ShowQueriesRewrite.java b/core/trino-main/src/main/java/io/trino/sql/rewrite/ShowQueriesRewrite.java index fba03ce4215c..7e644863b974 100644 --- a/core/trino-main/src/main/java/io/trino/sql/rewrite/ShowQueriesRewrite.java +++ b/core/trino-main/src/main/java/io/trino/sql/rewrite/ShowQueriesRewrite.java @@ -99,6 +99,7 @@ import io.trino.sql.tree.StringLiteral; import io.trino.sql.tree.TableElement; import io.trino.sql.tree.Values; +import io.trino.util.DateTimeUtils; import java.util.ArrayList; import java.util.Collection; @@ -561,7 +562,8 @@ private Query showCreateMaterializedView(ShowCreate node) query, false, false, - Optional.empty(), // TODO support GRACE PERIOD + viewDefinition.get().getGracePeriod() + .map(DateTimeUtils::formatDayTimeInterval), Optional.empty(), // TODO support WHEN STALE propertyNodes, viewDefinition.get().getComment())).trim(); diff --git a/core/trino-main/src/main/java/io/trino/util/DateTimeUtils.java b/core/trino-main/src/main/java/io/trino/util/DateTimeUtils.java index c5aabbf40656..affafb444b75 100644 --- a/core/trino-main/src/main/java/io/trino/util/DateTimeUtils.java +++ b/core/trino-main/src/main/java/io/trino/util/DateTimeUtils.java @@ -18,10 +18,12 @@ import io.trino.client.IntervalYearMonth; import io.trino.spi.TrinoException; import io.trino.spi.type.TimeZoneKey; +import io.trino.sql.tree.IntervalLiteral; import org.joda.time.DateTime; import org.joda.time.DurationFieldType; import org.joda.time.MutablePeriod; import org.joda.time.Period; +import org.joda.time.PeriodType; import org.joda.time.ReadWritablePeriod; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; @@ -34,6 +36,7 @@ import org.joda.time.format.PeriodParser; import java.time.DateTimeException; +import java.time.Duration; import java.time.LocalDate; import java.util.ArrayList; import java.util.List; @@ -46,6 +49,10 @@ import static com.google.common.base.Preconditions.checkArgument; import static io.trino.spi.StandardErrorCode.INVALID_LITERAL; import static io.trino.sql.tree.IntervalLiteral.IntervalField; +import static io.trino.sql.tree.IntervalLiteral.IntervalField.DAY; +import static io.trino.sql.tree.IntervalLiteral.IntervalField.SECOND; +import static io.trino.sql.tree.IntervalLiteral.Sign.NEGATIVE; +import static io.trino.sql.tree.IntervalLiteral.Sign.POSITIVE; import static io.trino.util.DateTimeZoneIndex.getChronology; import static io.trino.util.DateTimeZoneIndex.packDateTimeWithZone; import static java.lang.Math.toIntExact; @@ -254,6 +261,19 @@ public static long parseDayTimeInterval(String value, IntervalField startField, throw invalidQualifier(startField, endField.orElse(startField)); } + public static IntervalLiteral formatDayTimeInterval(Duration duration) + { + long millis = duration.toMillis(); + IntervalLiteral.Sign sign = millis < 0 ? NEGATIVE : POSITIVE; + Period period = new Period(Math.abs(millis)).normalizedStandard(PeriodType.dayTime()); + // Always use INTERVAL DAY TO SECOND. The output is more verbose + // (e.g., "1 0:00:00" instead of "1"), but this avoids the need to + // determine the minimal field range and choose a specialized formatter. + String value = INTERVAL_DAY_SECOND_FORMATTER.print(period); + + return new IntervalLiteral(value, sign, DAY, Optional.of(SECOND)); + } + private static long parsePeriodMillis(PeriodFormatter periodFormatter, String value) { Period period = parsePeriod(periodFormatter, value); @@ -337,7 +357,12 @@ private static PeriodFormatter createPeriodFormatter(IntervalField startField, I List parsers = new ArrayList<>(); - PeriodFormatterBuilder builder = new PeriodFormatterBuilder(); + PeriodFormatterBuilder builder = new PeriodFormatterBuilder() + // Ensures zero-valued fields are printed instead of omitted. This affects printing only, not parsing. + // Example for INTERVAL HOUR TO SECOND: + // With printZeroIfSupported(): "2:00:45" + // Without printZeroIfSupported(): "2::45" + .printZeroIfSupported(); switch (startField) { case YEAR: builder.appendYears(); @@ -372,6 +397,12 @@ private static PeriodFormatter createPeriodFormatter(IntervalField startField, I break; } builder.appendLiteral(":"); + // Ensures fixed-width, zero-padded minutes. This affects printing only, not parsing. + // Applies to the next appended field (minutes). + // Example for INTERVAL HOUR TO MINUTE: + // With minimumPrintedDigits(2): "2:05" + // Without minimumPrintedDigits(2): "2:5" + builder.minimumPrintedDigits(2); // fall through case MINUTE: @@ -381,6 +412,12 @@ private static PeriodFormatter createPeriodFormatter(IntervalField startField, I break; } builder.appendLiteral(":"); + // Ensures fixed-width, zero-padded seconds. This affects printing only, not parsing. + // Applies to the next appended field (seconds). + // Example for INTERVAL HOUR TO SECOND: + // With minimumPrintedDigits(2): "2:05:07" + // Without minimumPrintedDigits(2): "2:05:7" + builder.minimumPrintedDigits(2); // fall through case SECOND: diff --git a/core/trino-main/src/test/java/io/trino/type/TestIntervalDayTime.java b/core/trino-main/src/test/java/io/trino/type/TestIntervalDayTime.java index 815cc6190a25..921bfd5c8be8 100644 --- a/core/trino-main/src/test/java/io/trino/type/TestIntervalDayTime.java +++ b/core/trino-main/src/test/java/io/trino/type/TestIntervalDayTime.java @@ -13,13 +13,19 @@ */ package io.trino.type; +import io.trino.sql.ExpressionFormatter; import io.trino.sql.query.QueryAssertions; +import io.trino.sql.query.QueryAssertions.ExpressionAssertProvider.Result; +import io.trino.sql.tree.IntervalLiteral; +import org.intellij.lang.annotations.Language; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.parallel.Execution; +import java.time.Duration; + import static io.trino.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT; import static io.trino.spi.function.OperatorType.ADD; import static io.trino.spi.function.OperatorType.DIVIDE; @@ -32,6 +38,7 @@ import static io.trino.spi.function.OperatorType.SUBTRACT; import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.testing.assertions.TrinoExceptionAssert.assertTrinoExceptionThrownBy; +import static io.trino.util.DateTimeUtils.formatDayTimeInterval; import static java.util.concurrent.TimeUnit.DAYS; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; @@ -523,4 +530,74 @@ public void testIndeterminate() assertThat(assertions.operator(INDETERMINATE, "INTERVAL '45' MINUTE TO SECOND")) .isEqualTo(false); } + + @Test + void testIntervalDayTimeRoundTrip() + { + testIntervalDayTimeRoundTrip("INTERVAL '0' SECOND", "INTERVAL '0 0:00:00' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL -'0' SECOND", "INTERVAL '0 0:00:00' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '0.000' SECOND", "INTERVAL '0 0:00:00' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '0.4' SECOND", "INTERVAL '0 0:00:00.400' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '0.04' SECOND", "INTERVAL '0 0:00:00.040' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '0.040' SECOND", "INTERVAL '0 0:00:00.040' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '45' SECOND", "INTERVAL '0 0:00:45' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL -'45' SECOND", "INTERVAL -'0 0:00:45' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '0.555' SECOND", "INTERVAL '0 0:00:00.555' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '59.999' SECOND", "INTERVAL '0 0:00:59.999' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '60' SECOND", "INTERVAL '0 0:01:00' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '61' SECOND", "INTERVAL '0 0:01:01' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '3661' SECOND", "INTERVAL '0 1:01:01' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '90061' SECOND", "INTERVAL '1 1:01:01' DAY TO SECOND"); + + testIntervalDayTimeRoundTrip("INTERVAL '0' MINUTE", "INTERVAL '0 0:00:00' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL -'0' MINUTE", "INTERVAL '0 0:00:00' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '25' MINUTE", "INTERVAL '0 0:25:00' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL -'25' MINUTE", "INTERVAL -'0 0:25:00' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '15:30' MINUTE TO SECOND", "INTERVAL '0 0:15:30' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '59:00.999' MINUTE TO SECOND", "INTERVAL '0 0:59:00.999' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '60' MINUTE", "INTERVAL '0 1:00:00' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '61' MINUTE", "INTERVAL '0 1:01:00' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '1500' MINUTE", "INTERVAL '1 1:00:00' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '1501' MINUTE", "INTERVAL '1 1:01:00' DAY TO SECOND"); + + testIntervalDayTimeRoundTrip("INTERVAL '0' HOUR", "INTERVAL '0 0:00:00' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL -'0' HOUR", "INTERVAL '0 0:00:00' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '8' HOUR", "INTERVAL '0 8:00:00' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL -'8' HOUR", "INTERVAL -'0 8:00:00' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '2:45' HOUR TO MINUTE", "INTERVAL '0 2:45:00' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '2:00:45' HOUR TO SECOND", "INTERVAL '0 2:00:45' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '1:30:45' HOUR TO SECOND", "INTERVAL '0 1:30:45' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '1:00:00.999' HOUR TO SECOND", "INTERVAL '0 1:00:00.999' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '24' HOUR", "INTERVAL '1 0:00:00' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '25' HOUR", "INTERVAL '1 1:00:00' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '17520' HOUR", "INTERVAL '730 0:00:00' DAY TO SECOND"); + + testIntervalDayTimeRoundTrip("INTERVAL '0' DAY", "INTERVAL '0 0:00:00' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL -'0' DAY", "INTERVAL '0 0:00:00' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '340' DAY", "INTERVAL '340 0:00:00' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL -'340' DAY", "INTERVAL -'340 0:00:00' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '2 6' DAY TO HOUR", "INTERVAL '2 6:00:00' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '3 0:30' DAY TO MINUTE", "INTERVAL '3 0:30:00' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '3 12:30' DAY TO MINUTE", "INTERVAL '3 12:30:00' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '1 0:00:15' DAY TO SECOND", "INTERVAL '1 0:00:15' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '1 4:20:15' DAY TO SECOND", "INTERVAL '1 4:20:15' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '1 0:00:00.999' DAY TO SECOND", "INTERVAL '1 0:00:00.999' DAY TO SECOND"); + testIntervalDayTimeRoundTrip("INTERVAL '1 23:59:59.999' DAY TO SECOND", "INTERVAL '1 23:59:59.999' DAY TO SECOND"); + } + + private void testIntervalDayTimeRoundTrip(@Language("SQL") String input, @Language("SQL") String expectedFormatted) + { + Result evaluatedResult = assertions.expression(input).evaluate(); + assertThat(evaluatedResult.type()).isEqualTo(IntervalDayTimeType.INTERVAL_DAY_TIME); + SqlIntervalDayTime originalInterval = (SqlIntervalDayTime) evaluatedResult.value(); + + Duration duration = Duration.ofMillis(originalInterval.getMillis()); + IntervalLiteral formattedLiteral = formatDayTimeInterval(duration); + String formatted = ExpressionFormatter.formatExpression(formattedLiteral); + assertThat(formatted).isEqualTo(expectedFormatted); + + Result reparsedResult = assertions.expression(formatted).evaluate(); + SqlIntervalDayTime reparsedInterval = (SqlIntervalDayTime) reparsedResult.value(); + assertThat(reparsedInterval).isEqualTo(originalInterval); + } } diff --git a/core/trino-server/src/main/provisio/trino.xml b/core/trino-server/src/main/provisio/trino.xml index 97a366b39290..e4a8e7bc8e4c 100644 --- a/core/trino-server/src/main/provisio/trino.xml +++ b/core/trino-server/src/main/provisio/trino.xml @@ -284,6 +284,12 @@ + + + + + + diff --git a/docs/src/main/sphinx/connector.md b/docs/src/main/sphinx/connector.md index a954b30cf059..1a77556400c3 100644 --- a/docs/src/main/sphinx/connector.md +++ b/docs/src/main/sphinx/connector.md @@ -42,6 +42,7 @@ SingleStore Snowflake SQL Server System +Teradata Thrift TPC-DS TPC-H diff --git a/docs/src/main/sphinx/connector/teradata.md b/docs/src/main/sphinx/connector/teradata.md new file mode 100644 index 000000000000..8936258ebce8 --- /dev/null +++ b/docs/src/main/sphinx/connector/teradata.md @@ -0,0 +1,192 @@ +# Teradata connector + +```{raw} html + +``` + +The Teradata connector allows querying and creating tables in an +external [Teradata](https://www.teradata.com/) database. This can be used to join data between +different systems like Teradata and Hive, or between different +Teradata instances. + +## Requirements + +To connect to Teradata, you need: + +- Teradata database 16.20 or higher. +- Network access from the Trino coordinator and workers to Teradata. +- Port 1025 is the default port. + +## Configuration + +The connector can query a database on a given Teradata instance. Create a catalog +properties file that specifies the Teradata connector by setting the +`connector.name` to `teradata`. + +For example, to access a database as the `example` catalog, create the file +`etc/catalog/example.properties`. Replace the connection properties as +appropriate for your setup: + +```properties +connector.name=teradata +connection-url=jdbc:teradata://example.teradata.com/CHARSET=UTF8,TMODE=ANSI,LOGMECH=TD2 +connection-user=root +connection-password=secret +``` + +The `connection-url` defines the connection information and parameters to pass +to the Teradata JDBC driver. The parameters for the URL are available in the +[Teradata JDBC documentation](https://teradata-docs.s3.amazonaws.com/doc/connectivity/jdbc/reference/current/jdbcug_chapter_2.html#BABJIHBJ). + +The `connection-user` and `connection-password` are typically required and +determine the user credentials for the connection, often a service user. You can +use {doc}`secrets ` to avoid actual values in the catalog +properties files. + +### Connection security + +If you have TLS configured with a globally-trusted certificate installed on +your data source, you can enable TLS between your cluster and the data +source by appending parameters to the JDBC connection string set in the +connection-url catalog configuration property. + +For example, to specify `SSLMODE`: + +```properties +connection-url=jdbc:teradata://example.teradata.com/SSLMODE=REQUIRED +``` + +For more information on TLS configuration options, see the +Teradata [JDBC documentation](https://teradata-docs.s3.amazonaws.com/doc/connectivity/jdbc/reference/current/jdbcug_chapter_2.html#URL_SSLMODE). + +```{include} jdbc-authentication.fragment +``` + +### Multiple Teradata databases + +The Teradata connector can only access a single Teradata database within +a single catalog. Thus, if you have multiple Teradata databases, +or want to connect to multiple Teradata instances, you must configure +multiple instances of the Teradata connector. + +To add another catalog, simply add another properties file to `etc/catalog` +with a different name, making sure it ends in `.properties`. For example, +if you name the property file `sales.properties`, Trino creates a +catalog named `sales` using the configured connector. + +## Type mapping + +Because Trino and Teradata each support types that the other does not, this +connector {ref}`modifies some types ` when reading data. +Refer to the following sections for type mapping when reading data from +Teradata to Trino. + +### Teradata type to Trino type mapping + +The connector maps Teradata types to the corresponding Trino types following +this table: + +:::{list-table} Teradata type to Trino type mapping +:widths: 40, 40, 20 +:header-rows: 1 + +* - Teradata type + - Trino type + - Notes +* - `TINYINT` + - `TINYINT` + - +* - `SMALLINT` + - `SMALLINT` + - +* - `INTEGER` + - `INTEGER` + - +* - `BIGINT` + - `BIGINT` + - +* - `REAL` + - `DOUBLE` + - +* - `DOUBLE` + - `DOUBLE` + - +* - `FLOAT` + - `DOUBLE` + - +* - `NUMBER(p, s)` + - `DECIMAL(p, s)` + - +* - `NUMERIC(p, s)` + - `DECIMAL(p, s)` + - +* - `DECIMAL(p, s)` + - `DECIMAL(p, s)` + - +* - `CHAR(n)` + - `CHAR(n)` + - +* - `CHARACTER(n)` + - `CHAR(n)` + - +* - `VARCHAR(n)` + - `VARCHAR(n)` + - +* - `DATE` + - `DATE` + - +::: + +No other types are supported. + +### Trino type to Teradata type mapping + +The connector maps Trino types to the corresponding Teradata types following +this table: + +:::{list-table} Trino type to Teradata type mapping +:widths: 40, 40, 20 +:header-rows: 1 + +* - Trino type + - Teradata type + - Notes +* - `TINYINT` + - `SMALLINT` + - +* - `SMALLINT` + - `SMALLINT` + - +* - `INTEGER` + - `INTEGER` + - +* - `BIGINT` + - `BIGINT` + - +* - `REAL` + - `FLOAT` + - +* - `DOUBLE` + - `DOUBLE` + - +* - `DATE` + - `DATE` + - +:::: + +No other types are supported. + +```{include} jdbc-type-mapping.fragment +``` + +## SQL support + +The connector provides read and limited write access to data and metadata in +a Teradata database. In addition to the [globally available](sql-globally-available) and +[read operation](sql-read-operations) statements, the connector supports the +following features: + +- [](/sql/create-schema) +- [](/sql/drop-schema) +- [](/sql/create-table) +- [](/sql/drop-table) diff --git a/plugin/trino-bigquery/src/main/java/io/trino/plugin/bigquery/BigQuerySplitSource.java b/plugin/trino-bigquery/src/main/java/io/trino/plugin/bigquery/BigQuerySplitSource.java index cd088d8ed883..632e641d7f41 100644 --- a/plugin/trino-bigquery/src/main/java/io/trino/plugin/bigquery/BigQuerySplitSource.java +++ b/plugin/trino-bigquery/src/main/java/io/trino/plugin/bigquery/BigQuerySplitSource.java @@ -155,7 +155,9 @@ private List getSplits( String query = buildNativeQuery(bigQueryQueryRelationHandle.getQuery(), filter, limit); TableId destinationTable = bigQueryQueryRelationHandle.getDestinationTableName().toTableId(); - TableInfo tableInfo = new ViewMaterializationCache.DestinationTableBuilder(bigQueryClientFactory.create(session), viewExpiration, query, destinationTable).get(); + BigQueryClient bigQueryClient = bigQueryClientFactory.create(session); + Optional bigQueryTable = bigQueryClient.getTable(destinationTable); + TableInfo tableInfo = bigQueryTable.orElseGet(() -> new ViewMaterializationCache.DestinationTableBuilder(bigQueryClient, viewExpiration, query, destinationTable).get()); log.debug("Using Storage API for running query: %s", query); remoteTableId = tableInfo.getTableId(); diff --git a/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/BaseBigQueryConnectorTest.java b/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/BaseBigQueryConnectorTest.java index 9072c195725e..41cba58b3e83 100644 --- a/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/BaseBigQueryConnectorTest.java +++ b/plugin/trino-bigquery/src/test/java/io/trino/plugin/bigquery/BaseBigQueryConnectorTest.java @@ -1122,6 +1122,13 @@ public void testNativeQuerySelectFromTestTable() } } + @Test // regression test for https://github.com/trinodb/trino/issues/27573 + public void testNativeQueryWhenResultReused() + { + assertThat(query("WITH t AS (SELECT * FROM TABLE(system.query('SELECT regionkey FROM tpch.region WHERE regionkey = 0'))) SELECT * FROM t, t")) + .matches("VALUES (BIGINT '0', BIGINT '0')"); + } + @Test public void testNativeQuerySelectUnsupportedType() { diff --git a/plugin/trino-pinot/pom.xml b/plugin/trino-pinot/pom.xml index 4ddfeadefda8..6793d8ef73ce 100644 --- a/plugin/trino-pinot/pom.xml +++ b/plugin/trino-pinot/pom.xml @@ -26,7 +26,7 @@ net.openhft chronicle-core - 2.27ea9 + 2.27ea10 net.openhft diff --git a/plugin/trino-teradata/README.md b/plugin/trino-teradata/README.md new file mode 100644 index 000000000000..e9e3817f684b --- /dev/null +++ b/plugin/trino-teradata/README.md @@ -0,0 +1,42 @@ +# Teradata Connector Developer Notes + +The Teradata connector module has both unit tests and integration tests. +The integration tests require access to a [Teradata ClearScape Analytics™ Experience](https://clearscape.teradata.com/sign-in). +You can follow the steps below to run the integration tests locally. + +## Prerequisites + +#### 1. Create a new ClearScape Analytics™ Experience account + +If you don't already have one, sign up at: + +[Teradata ClearScape Analytics™ Experience](https://www.teradata.com/getting-started/demos/clearscape-analytics) + +#### 2. Login + +Sign in with your new account at: + +[ClearScape Analytics™ Experience Login](https://clearscape.teradata.com/sign-in) + +#### 3. Collect the API Token + +Use the **Copy API Token** button in the UI to retrieve your token. + +#### 4. Define the following environment variables + +⚠️ **Note:** The Teradata database password must be **at least 8 characters long**. + +``` +export CLEARSCAPE_TOKEN= +export CLEARSCAPE_PASSWORD= +``` + +## Running Integration Tests + +Once the environment variables are set, run the integration tests with: + +⚠️ **Note:** Run the following command from the Trino parent directory. + +``` + ./mvnw clean install -pl :trino-teradata +``` diff --git a/plugin/trino-teradata/pom.xml b/plugin/trino-teradata/pom.xml new file mode 100644 index 000000000000..8c1430b19ba6 --- /dev/null +++ b/plugin/trino-teradata/pom.xml @@ -0,0 +1,327 @@ + + + 4.0.0 + + io.trino + trino-root + 479-SNAPSHOT + ../../pom.xml + + + trino-teradata + trino-plugin + ${project.artifactId} + Trino - Teradata connector + + + true + true + + + + + + com.google.guava + guava + + + + com.google.inject + guice + classes + + + + io.airlift + configuration + + + + io.airlift + log + + + + io.trino + trino-base-jdbc + + + + io.trino + trino-plugin-toolkit + + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + + io.airlift + slice + provided + + + + io.opentelemetry + opentelemetry-api + provided + + + + io.opentelemetry + opentelemetry-api-incubator + provided + + + + io.opentelemetry + opentelemetry-context + provided + + + + io.trino + trino-spi + provided + + + + org.openjdk.jol + jol-core + provided + + + + com.teradata.jdbc + terajdbc + 20.00.00.51 + runtime + + + + io.airlift + units + runtime + + + + com.fasterxml.jackson.core + jackson-core + test + + + + com.fasterxml.jackson.core + jackson-databind + test + + + + com.google.errorprone + error_prone_annotations + test + true + + + + io.airlift + configuration-testing + test + + + + io.airlift + json + test + + + + io.airlift + junit-extensions + test + + + + io.airlift + log-manager + test + + + + io.airlift + testing + test + + + + io.airlift + tracing + test + + + + io.trino + trino-base-jdbc + test-jar + test + + + + io.trino + trino-exchange-filesystem + test + + + + io.trino + trino-exchange-filesystem + test-jar + test + + + + io.trino + trino-jmx + test + + + + io.trino + trino-main + test + + + + io.trino + trino-main + test-jar + test + + + + io.trino + trino-parser + test + + + + io.trino + trino-testing + test + + + + io.trino + trino-testing-containers + test + + + + io.trino + trino-testing-services + test + + + + io.trino + trino-tpch + test + + + + io.trino.tpch + tpch + test + + + + org.assertj + assertj-core + test + + + + org.jetbrains + annotations + test + + + + org.junit.jupiter + junit-jupiter-api + test + + + + org.junit.jupiter + junit-jupiter-engine + test + + + + + + default + + true + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + **/TestTeradataConnectorTest.java + **/TestTeradataTypeMapping.java + + + + + + + + + clearscape-tests + + false + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + **/TestTeradataConnectorTest.java + **/TestTeradataTypeMapping.java + + long_run + + + + + + + + run-only-long-tests + + false + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + long_run + + + + + + + diff --git a/plugin/trino-teradata/src/main/java/io/trino/plugin/teradata/TeradataClient.java b/plugin/trino-teradata/src/main/java/io/trino/plugin/teradata/TeradataClient.java new file mode 100644 index 000000000000..a4eba4da7c7b --- /dev/null +++ b/plugin/trino-teradata/src/main/java/io/trino/plugin/teradata/TeradataClient.java @@ -0,0 +1,352 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.teradata; + +import com.google.common.collect.ImmutableMap; +import com.google.inject.Inject; +import io.airlift.log.Logger; +import io.trino.plugin.base.mapping.IdentifierMapping; +import io.trino.plugin.jdbc.BaseJdbcClient; +import io.trino.plugin.jdbc.BaseJdbcConfig; +import io.trino.plugin.jdbc.CaseSensitivity; +import io.trino.plugin.jdbc.ColumnMapping; +import io.trino.plugin.jdbc.ConnectionFactory; +import io.trino.plugin.jdbc.JdbcColumnHandle; +import io.trino.plugin.jdbc.JdbcOutputTableHandle; +import io.trino.plugin.jdbc.JdbcTableHandle; +import io.trino.plugin.jdbc.JdbcTypeHandle; +import io.trino.plugin.jdbc.QueryBuilder; +import io.trino.plugin.jdbc.RemoteTableName; +import io.trino.plugin.jdbc.WriteMapping; +import io.trino.plugin.jdbc.logging.RemoteQueryModifier; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.ColumnMetadata; +import io.trino.spi.connector.ColumnPosition; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.type.CharType; +import io.trino.spi.type.DecimalType; +import io.trino.spi.type.Decimals; +import io.trino.spi.type.Type; +import io.trino.spi.type.VarcharType; + +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.PreparedStatement; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.sql.Types; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.OptionalLong; + +import static io.trino.plugin.jdbc.CaseSensitivity.CASE_INSENSITIVE; +import static io.trino.plugin.jdbc.CaseSensitivity.CASE_SENSITIVE; +import static io.trino.plugin.jdbc.PredicatePushdownController.CASE_INSENSITIVE_CHARACTER_PUSHDOWN; +import static io.trino.plugin.jdbc.PredicatePushdownController.FULL_PUSHDOWN; +import static io.trino.plugin.jdbc.StandardColumnMappings.bigintColumnMapping; +import static io.trino.plugin.jdbc.StandardColumnMappings.bigintWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.charReadFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.charWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.dateColumnMappingUsingLocalDate; +import static io.trino.plugin.jdbc.StandardColumnMappings.dateWriteFunctionUsingLocalDate; +import static io.trino.plugin.jdbc.StandardColumnMappings.decimalColumnMapping; +import static io.trino.plugin.jdbc.StandardColumnMappings.doubleColumnMapping; +import static io.trino.plugin.jdbc.StandardColumnMappings.doubleWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.integerColumnMapping; +import static io.trino.plugin.jdbc.StandardColumnMappings.integerWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.longDecimalWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.realWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.shortDecimalWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.smallintColumnMapping; +import static io.trino.plugin.jdbc.StandardColumnMappings.smallintWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.tinyintColumnMapping; +import static io.trino.plugin.jdbc.StandardColumnMappings.tinyintWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.varcharReadFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.varcharWriteFunction; +import static io.trino.plugin.jdbc.TypeHandlingJdbcSessionProperties.getUnsupportedTypeHandling; +import static io.trino.plugin.jdbc.UnsupportedTypeHandling.CONVERT_TO_VARCHAR; +import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.CharType.createCharType; +import static io.trino.spi.type.DateType.DATE; +import static io.trino.spi.type.DecimalType.createDecimalType; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static io.trino.spi.type.IntegerType.INTEGER; +import static io.trino.spi.type.RealType.REAL; +import static io.trino.spi.type.SmallintType.SMALLINT; +import static io.trino.spi.type.TinyintType.TINYINT; +import static io.trino.spi.type.VarcharType.createVarcharType; +import static java.lang.String.format; + +public class TeradataClient + extends BaseJdbcClient +{ + private static final Logger log = Logger.get(TeradataClient.class); + + @Inject + public TeradataClient( + BaseJdbcConfig config, + ConnectionFactory connectionFactory, + QueryBuilder queryBuilder, + IdentifierMapping identifierMapping, + RemoteQueryModifier remoteQueryModifier) + { + super("\"", connectionFactory, queryBuilder, config.getJdbcTypesMappedToVarchar(), identifierMapping, remoteQueryModifier, true); + } + + @Override + protected void createSchema(ConnectorSession session, Connection connection, String remoteSchemaName) + { + execute(session, format("CREATE DATABASE %s AS PERMANENT = 60000000", quoted(remoteSchemaName))); + } + + @Override + protected void verifySchemaName(DatabaseMetaData databaseMetadata, String schemaName) + throws SQLException + { + int schemaNameLimit = databaseMetadata.getMaxSchemaNameLength(); + if (schemaName.length() > schemaNameLimit) { + throw new TrinoException( + NOT_SUPPORTED, + format("Schema name must be shorter than or equal to '%s' characters but got '%s'", schemaNameLimit, schemaName.length())); + } + } + + @Override + protected void verifyTableName(DatabaseMetaData databaseMetadata, String tableName) + throws SQLException + { + if (tableName.length() > databaseMetadata.getMaxTableNameLength()) { + throw new TrinoException( + NOT_SUPPORTED, + format("Table name must be shorter than or equal to '%s' characters but got '%s'", databaseMetadata.getMaxTableNameLength(), tableName.length())); + } + } + + @Override + protected void verifyColumnName(DatabaseMetaData databaseMetadata, String columnName) + throws SQLException + { + if (columnName.length() > databaseMetadata.getMaxColumnNameLength()) { + throw new TrinoException( + NOT_SUPPORTED, + format("Column name must be shorter than or equal to '%s' characters but got '%s': '%s'", databaseMetadata.getMaxColumnNameLength(), columnName.length(), columnName)); + } + } + + @Override + protected void dropSchema(ConnectorSession session, Connection connection, String remoteSchemaName, boolean cascade) + throws SQLException + { + if (cascade) { + throw new TrinoException( + NOT_SUPPORTED, + "This connector does not support dropping schemas with CASCADE option"); + } + String dropSchema = "DROP DATABASE " + quoted(remoteSchemaName); + execute(session, connection, dropSchema); + } + + @Override + public void renameSchema(ConnectorSession session, String schemaName, String newSchemaName) + { + throw new TrinoException(NOT_SUPPORTED, "This connector does not support renaming schemas"); + } + + @Override + public OptionalLong delete(ConnectorSession session, JdbcTableHandle handle) + { + throw new TrinoException(NOT_SUPPORTED, "This connector does not support modifying table rows"); + } + + @Override + public void truncateTable(ConnectorSession session, JdbcTableHandle handle) + { + throw new TrinoException(NOT_SUPPORTED, "This connector does not support truncating tables"); + } + + @Override + public void dropColumn(ConnectorSession session, JdbcTableHandle handle, JdbcColumnHandle column) + { + throw new TrinoException(NOT_SUPPORTED, "This connector does not support dropping columns"); + } + + @Override + public void renameColumn(ConnectorSession session, JdbcTableHandle handle, JdbcColumnHandle jdbcColumn, String newColumnName) + { + throw new TrinoException(NOT_SUPPORTED, "This connector does not support renaming columns"); + } + + @Override + public void renameTable(ConnectorSession session, JdbcTableHandle handle, SchemaTableName newTableName) + { + throw new TrinoException(NOT_SUPPORTED, "This connector does not support renaming tables"); + } + + @Override + public JdbcOutputTableHandle beginInsertTable(ConnectorSession session, JdbcTableHandle tableHandle, List columns) + { + throw new TrinoException(NOT_SUPPORTED, "This connector does not support inserts"); + } + + @Override + public void setColumnType(ConnectorSession session, JdbcTableHandle handle, JdbcColumnHandle column, Type type) + { + throw new TrinoException(NOT_SUPPORTED, "This connector does not support setting column types"); + } + + @Override + public void addColumn(ConnectorSession session, JdbcTableHandle handle, ColumnMetadata column, ColumnPosition position) + { + throw new TrinoException(NOT_SUPPORTED, "This connector does not support adding columns"); + } + + @Override + public void dropNotNullConstraint(ConnectorSession session, JdbcTableHandle handle, JdbcColumnHandle column) + { + throw new TrinoException(NOT_SUPPORTED, "This connector does not support dropping a not null constraint"); + } + + @Override + protected Map getCaseSensitivityForColumns(ConnectorSession session, Connection connection, SchemaTableName schemaTableName, RemoteTableName remoteTableName) + { + String sql = format("SELECT * FROM %s.%s WHERE 0=1", quoted(schemaTableName.getSchemaName()), quoted(schemaTableName.getTableName())); + try (PreparedStatement preparedStatement = connection.prepareStatement(sql)) { + ImmutableMap.Builder columns = ImmutableMap.builder(); + ResultSetMetaData metaData = preparedStatement.getMetaData(); + int columnCount = metaData.getColumnCount(); + + for (int i = 1; i <= columnCount; i++) { + columns.put( + metaData.getColumnName(i), + metaData.isCaseSensitive(i) ? CASE_SENSITIVE : CASE_INSENSITIVE); + } + + return columns.buildOrThrow(); + } + catch (SQLException e) { + return ImmutableMap.of(); + } + } + + @Override + public Optional toColumnMapping(ConnectorSession session, Connection connection, JdbcTypeHandle typeHandle) + { + Optional mapping = getForcedMappingToVarchar(typeHandle); + if (mapping.isPresent()) { + return mapping; + } + + switch (typeHandle.jdbcType()) { + case Types.TINYINT: + return Optional.of(tinyintColumnMapping()); + case Types.SMALLINT: + return Optional.of(smallintColumnMapping()); + case Types.INTEGER: + return Optional.of(integerColumnMapping()); + case Types.BIGINT: + return Optional.of(bigintColumnMapping()); + case Types.REAL: + case Types.DOUBLE: + case Types.FLOAT: + // FLOAT is a Teradata synonym for REAL and DOUBLE PRECISION + return Optional.of(doubleColumnMapping()); + case Types.NUMERIC: + case Types.DECIMAL: + return numberMapping(typeHandle); + case Types.CHAR: + return Optional.of(charColumnMapping(typeHandle.requiredColumnSize(), deriveCaseSensitivity(typeHandle.caseSensitivity().orElse(null)))); + case Types.VARCHAR: + return Optional.of(varcharColumnMapping(typeHandle.requiredColumnSize(), deriveCaseSensitivity(typeHandle.caseSensitivity().orElse(null)))); + case Types.DATE: + return Optional.of(dateColumnMappingUsingLocalDate()); + } + + if (getUnsupportedTypeHandling(session) == CONVERT_TO_VARCHAR) { + log.debug("Mapping unsupported Teradata type %s to VARCHAR", typeHandle); + return mapToUnboundedVarchar(typeHandle); + } + + return Optional.empty(); + } + + private static Optional numberMapping(JdbcTypeHandle typeHandle) + { + int precision = typeHandle.requiredColumnSize(); + int scale = typeHandle.requiredDecimalDigits(); + if (precision > Decimals.MAX_PRECISION) { + // this will trigger for number(*) as precision is 40 + return Optional.of(decimalColumnMapping(createDecimalType(Decimals.MAX_PRECISION, scale))); + } + return Optional.of(decimalColumnMapping(createDecimalType(precision, scale))); + } + + private static ColumnMapping charColumnMapping(int charLength, boolean isCaseSensitive) + { + // Teradata supports max of 64k for char type + CharType charType = createCharType(charLength); + return ColumnMapping.sliceMapping( + charType, + charReadFunction(charType), + charWriteFunction(), + isCaseSensitive ? FULL_PUSHDOWN : CASE_INSENSITIVE_CHARACTER_PUSHDOWN); + } + + private static ColumnMapping varcharColumnMapping(int varcharLength, boolean isCaseSensitive) + { + // Teradata supports max of 64k for varchar type + VarcharType varcharType = createVarcharType(varcharLength); + return ColumnMapping.sliceMapping( + varcharType, + varcharReadFunction(varcharType), + varcharWriteFunction(), + isCaseSensitive ? FULL_PUSHDOWN : CASE_INSENSITIVE_CHARACTER_PUSHDOWN); + } + + private boolean deriveCaseSensitivity(CaseSensitivity caseSensitivity) + { + return caseSensitivity == CASE_SENSITIVE; + } + + @Override + public WriteMapping toWriteMapping(ConnectorSession session, Type type) + { + return switch (type) { + case Type typeInstance when typeInstance == TINYINT -> WriteMapping.longMapping("smallint", tinyintWriteFunction()); + case Type typeInstance when typeInstance == SMALLINT -> WriteMapping.longMapping("smallint", smallintWriteFunction()); + case Type typeInstance when typeInstance == INTEGER -> WriteMapping.longMapping("integer", integerWriteFunction()); + case Type typeInstance when typeInstance == BIGINT -> WriteMapping.longMapping("bigint", bigintWriteFunction()); + case Type typeInstance when typeInstance == REAL -> WriteMapping.longMapping("FLOAT", realWriteFunction()); + case Type typeInstance when typeInstance == DOUBLE -> WriteMapping.doubleMapping("double precision", doubleWriteFunction()); + case Type typeInstance when typeInstance == DATE -> WriteMapping.longMapping("date", dateWriteFunctionUsingLocalDate()); + case DecimalType decimalTypeInstance -> { + String dataType = format("decimal(%s, %s)", decimalTypeInstance.getPrecision(), decimalTypeInstance.getScale()); + if (decimalTypeInstance.isShort()) { + yield WriteMapping.longMapping(dataType, shortDecimalWriteFunction(decimalTypeInstance)); + } + yield WriteMapping.objectMapping(dataType, longDecimalWriteFunction(decimalTypeInstance)); + } + case CharType charTypeInstance -> WriteMapping.sliceMapping("char(" + charTypeInstance.getLength() + ")", charWriteFunction()); + case VarcharType varcharTypeInstance -> { + String dataType = varcharTypeInstance.isUnbounded() + ? "clob" + : "varchar(" + varcharTypeInstance.getBoundedLength() + ")"; + yield WriteMapping.sliceMapping(dataType, varcharWriteFunction()); + } + default -> throw new TrinoException(NOT_SUPPORTED, "Unsupported column type: " + type.getDisplayName()); + }; + } +} diff --git a/plugin/trino-teradata/src/main/java/io/trino/plugin/teradata/TeradataClientModule.java b/plugin/trino-teradata/src/main/java/io/trino/plugin/teradata/TeradataClientModule.java new file mode 100644 index 000000000000..e286f0fed432 --- /dev/null +++ b/plugin/trino-teradata/src/main/java/io/trino/plugin/teradata/TeradataClientModule.java @@ -0,0 +1,62 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.teradata; + +import com.google.inject.Binder; +import com.google.inject.Provides; +import com.google.inject.Scopes; +import com.google.inject.Singleton; +import io.airlift.configuration.AbstractConfigurationAwareModule; +import io.opentelemetry.api.OpenTelemetry; +import io.trino.plugin.jdbc.BaseJdbcConfig; +import io.trino.plugin.jdbc.ConnectionFactory; +import io.trino.plugin.jdbc.DriverConnectionFactory; +import io.trino.plugin.jdbc.ForBaseJdbc; +import io.trino.plugin.jdbc.JdbcClient; +import io.trino.plugin.jdbc.JdbcJoinPushdownSupportModule; +import io.trino.plugin.jdbc.JdbcStatisticsConfig; +import io.trino.plugin.jdbc.credential.CredentialProvider; + +import java.sql.Driver; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.util.Properties; + +import static io.airlift.configuration.ConfigBinder.configBinder; + +public class TeradataClientModule + extends AbstractConfigurationAwareModule +{ + @Provides + @Singleton + @ForBaseJdbc + public static ConnectionFactory getConnectionFactory(BaseJdbcConfig config, CredentialProvider credentialProvider, OpenTelemetry openTelemetry) + throws SQLException + { + Driver driver = DriverManager.getDriver(config.getConnectionUrl()); + Properties connectionProperties = new Properties(); + connectionProperties.setProperty("LOGMECH", "TD2"); + return DriverConnectionFactory.builder(driver, config.getConnectionUrl(), credentialProvider) + .setConnectionProperties(connectionProperties) + .setOpenTelemetry(openTelemetry).build(); + } + + @Override + public void setup(Binder binder) + { + binder.bind(JdbcClient.class).annotatedWith(ForBaseJdbc.class).to(TeradataClient.class).in(Scopes.SINGLETON); + configBinder(binder).bindConfig(JdbcStatisticsConfig.class); + install(new JdbcJoinPushdownSupportModule()); + } +} diff --git a/plugin/trino-teradata/src/main/java/io/trino/plugin/teradata/TeradataPlugin.java b/plugin/trino-teradata/src/main/java/io/trino/plugin/teradata/TeradataPlugin.java new file mode 100644 index 000000000000..d11110edfbed --- /dev/null +++ b/plugin/trino-teradata/src/main/java/io/trino/plugin/teradata/TeradataPlugin.java @@ -0,0 +1,25 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.teradata; + +import io.trino.plugin.jdbc.JdbcPlugin; + +public class TeradataPlugin + extends JdbcPlugin +{ + public TeradataPlugin() + { + super("teradata", TeradataClientModule::new); + } +} diff --git a/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/TestTeradataPlugin.java b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/TestTeradataPlugin.java new file mode 100644 index 000000000000..857b215ebf28 --- /dev/null +++ b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/TestTeradataPlugin.java @@ -0,0 +1,40 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.teradata; + +import io.trino.plugin.jdbc.JdbcConnectorFactory; +import io.trino.spi.connector.ConnectorFactory; +import io.trino.testing.TestingConnectorContext; +import org.junit.jupiter.api.Test; + +import java.util.Map; + +import static com.google.common.collect.Iterables.getOnlyElement; +import static org.assertj.core.api.Assertions.assertThat; + +public class TestTeradataPlugin +{ + @Test + public void testCreateConnector() + { + TeradataPlugin plugin = new TeradataPlugin(); + ConnectorFactory factory = getOnlyElement(plugin.getConnectorFactories()); + assertThat(factory).isInstanceOf(JdbcConnectorFactory.class); + factory.create("test", + Map.of( + "connection-url", "jdbc:teradata://test/"), + new TestingConnectorContext()) + .shutdown(); + } +} diff --git a/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/AuthenticationConfig.java b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/AuthenticationConfig.java new file mode 100644 index 000000000000..605faaf3d0cb --- /dev/null +++ b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/AuthenticationConfig.java @@ -0,0 +1,24 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.teradata.integration; + +public record AuthenticationConfig( + String userName, + String password) +{ + public AuthenticationConfig() + { + this(null, null); + } +} diff --git a/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/DatabaseConfig.java b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/DatabaseConfig.java new file mode 100644 index 000000000000..ed0223a6c4c8 --- /dev/null +++ b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/DatabaseConfig.java @@ -0,0 +1,156 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.teradata.integration; + +import java.util.Map; + +public class DatabaseConfig +{ + private final String jdbcUrl; + private final String hostName; + private final String databaseName; + private final boolean useClearScape; + private final AuthenticationConfig authConfig; + private final String clearScapeEnvName; + private final Map jdbcProperties; + + private DatabaseConfig(Builder builder) + { + jdbcUrl = builder.jdbcUrl; + hostName = builder.hostName; + databaseName = builder.databaseName; + useClearScape = builder.useClearScape; + authConfig = builder.authConfig; + clearScapeEnvName = builder.clearScapeEnvName; + jdbcProperties = builder.jdbcProperties; + } + + public static Builder builder() + { + return new Builder(); + } + + public Builder toBuilder() + { + return builder() + .jdbcUrl(jdbcUrl) + .hostName(hostName) + .databaseName(databaseName) + .useClearScape(useClearScape) + .authConfig(authConfig) + .clearScapeEnvName(clearScapeEnvName) + .jdbcProperties(jdbcProperties); + } + + public String getJdbcUrl() + { + return jdbcUrl; + } + + public String getDatabaseName() + { + return databaseName; + } + + public boolean isUseClearScape() + { + return useClearScape; + } + + public AuthenticationConfig getAuthConfig() + { + return authConfig; + } + + public String getClearScapeEnvName() + { + return clearScapeEnvName; + } + + public Map getJdbcProperties() + { + return jdbcProperties; + } + + public String getHostName() + { + return hostName; + } + + public String getTMode() + { + if (jdbcProperties != null && jdbcProperties.containsKey("TMODE")) { + return jdbcProperties.get("TMODE"); + } + return "ANSI"; + } + + public static class Builder + { + private String jdbcUrl; + private String hostName; + private String databaseName = "trino"; + private boolean useClearScape; + private AuthenticationConfig authConfig = new AuthenticationConfig(); + private String clearScapeEnvName; + private Map jdbcProperties; + + public Builder jdbcUrl(String jdbcUrl) + { + this.jdbcUrl = jdbcUrl; + return this; + } + + public Builder databaseName(String databaseName) + { + this.databaseName = databaseName; + return this; + } + + public Builder useClearScape(boolean useClearScape) + { + this.useClearScape = useClearScape; + return this; + } + + public Builder authConfig(AuthenticationConfig authConfig) + { + this.authConfig = authConfig; + return this; + } + + public Builder clearScapeEnvName(String clearScapeEnvName) + { + this.clearScapeEnvName = clearScapeEnvName; + return this; + } + + public Builder jdbcProperties(Map jdbcProperties) + { + this.jdbcProperties = jdbcProperties; + return this; + } + + public Builder hostName(String hostName) + { + this.hostName = hostName; + return this; + } + + public DatabaseConfig build() + { + return new DatabaseConfig(this); + } + } +} diff --git a/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/DatabaseConfigFactory.java b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/DatabaseConfigFactory.java new file mode 100644 index 000000000000..4dbbb2d97742 --- /dev/null +++ b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/DatabaseConfigFactory.java @@ -0,0 +1,63 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.teradata.integration; + +import java.util.HashMap; +import java.util.Map; + +import static io.trino.testing.SystemEnvironmentUtils.isEnvSet; +import static io.trino.testing.SystemEnvironmentUtils.requireEnv; + +public final class DatabaseConfigFactory +{ + private DatabaseConfigFactory() {} + + public static DatabaseConfig create(String envName) + { + String userName; + String password; + String hostName = null; + + if (isEnvSet("CLEARSCAPE_TOKEN")) { + userName = TeradataTestConstants.CLEARSCAPE_USERNAME; + password = requireEnv("CLEARSCAPE_PASSWORD"); + } + else { + userName = requireEnv("TERADATA_USERNAME"); + password = requireEnv("TERADATA_PASSWORD"); + hostName = requireEnv("TERADATA_HOSTNAME"); + } + + String databaseName = envName.replace("-", "_"); + + AuthenticationConfig authConfig = createAuthConfig(userName, password); + Map jdbcProperties = new HashMap<>(); + jdbcProperties.put("TMODE", "ANSI"); + jdbcProperties.put("CHARSET", "UTF8"); + + return DatabaseConfig.builder() + .hostName(hostName) + .databaseName(databaseName) + .useClearScape(hostName == null) + .authConfig(authConfig) + .clearScapeEnvName(envName) + .jdbcProperties(jdbcProperties) + .build(); + } + + private static AuthenticationConfig createAuthConfig(String username, String password) + { + return new AuthenticationConfig(username, password); + } +} diff --git a/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/TeradataQueryRunner.java b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/TeradataQueryRunner.java new file mode 100644 index 000000000000..53c6eb6178ea --- /dev/null +++ b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/TeradataQueryRunner.java @@ -0,0 +1,130 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.teradata.integration; + +import com.google.common.collect.ImmutableList; +import com.google.errorprone.annotations.CanIgnoreReturnValue; +import io.airlift.log.Level; +import io.airlift.log.Logger; +import io.airlift.log.Logging; +import io.trino.Session; +import io.trino.metadata.QualifiedObjectName; +import io.trino.plugin.teradata.TeradataPlugin; +import io.trino.plugin.tpch.TpchPlugin; +import io.trino.testing.DistributedQueryRunner; +import io.trino.testing.QueryRunner; +import io.trino.tpch.TpchTable; +import org.intellij.lang.annotations.Language; + +import java.util.List; +import java.util.Locale; + +import static io.trino.plugin.tpch.TpchMetadata.TINY_SCHEMA_NAME; +import static io.trino.testing.TestingSession.testSessionBuilder; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; +import static org.assertj.core.api.Assertions.assertThat; + +public final class TeradataQueryRunner +{ + private TeradataQueryRunner() {} + + public static Builder builder(TestingTeradataServer server) + { + return new Builder(server); + } + + public static class Builder + extends DistributedQueryRunner.Builder + { + private final TestingTeradataServer server; + private List> initialTables = ImmutableList.of(); + + protected Builder(TestingTeradataServer server) + { + super(testSessionBuilder().setCatalog("teradata").setSchema(server.getDatabaseName()).build()); + this.server = requireNonNull(server, "server is null"); + } + + public void copyTable(QueryRunner queryRunner, QualifiedObjectName table, Session session) + { + @Language("SQL") String sql = format("CREATE TABLE %s AS SELECT * FROM %s", table.objectName(), table); + queryRunner.execute(session, sql); + assertThat(queryRunner.execute(session, "SELECT count(*) FROM " + table.objectName()) + .getOnlyValue()) + .as("Table is not loaded properly: %s", new Object[] {table.objectName()}) + .isEqualTo(queryRunner.execute(session, "SELECT count(*) FROM " + table).getOnlyValue()); + } + + public void copyTpchTables(QueryRunner queryRunner, String sourceCatalog, String sourceSchema, Session session, Iterable> tables) + { + for (TpchTable table : tables) { + copyTable(queryRunner, sourceCatalog, sourceSchema, table.getTableName().toLowerCase(Locale.ENGLISH), session); + } + } + + public void copyTpchTables(QueryRunner queryRunner, String sourceCatalog, String sourceSchema, Iterable> tables) + { + copyTpchTables(queryRunner, sourceCatalog, sourceSchema, queryRunner.getDefaultSession(), tables); + } + + public void copyTable(QueryRunner queryRunner, String sourceCatalog, String sourceSchema, String sourceTable, Session session) + { + QualifiedObjectName table = new QualifiedObjectName(sourceCatalog, sourceSchema, sourceTable); + if (!server.isTableExists(sourceTable)) { + copyTable(queryRunner, table, session); + } + } + + @CanIgnoreReturnValue + public Builder setInitialTables(Iterable> initialTables) + { + this.initialTables = ImmutableList.copyOf(requireNonNull(initialTables, "initialTables is null")); + return this; + } + + @Override + public DistributedQueryRunner build() + throws Exception + { + super.setAdditionalSetup(runner -> { + runner.installPlugin(new TpchPlugin()); + runner.createCatalog("tpch", "tpch"); + + runner.installPlugin(new TeradataPlugin()); + runner.createCatalog("teradata", "teradata", server.fetchCatalogProperties()); + + copyTpchTables(runner, "tpch", TINY_SCHEMA_NAME, initialTables); + }); + return super.build(); + } + + static void main() + throws Exception + { + Logging logger = Logging.initialize(); + logger.setLevel("io.trino.plugin.teradata", Level.DEBUG); + logger.setLevel("io.trino", Level.INFO); + TestingTeradataServer server = new TestingTeradataServer("TeradataQueryRunner", false); + QueryRunner queryRunner = builder(server) + .addCoordinatorProperty("http-server.http.port", "8080") + .setInitialTables(TpchTable.getTables()) + .build(); + + Logger log = Logger.get(TeradataQueryRunner.class); + log.info("======== SERVER STARTED ========"); + log.info("\n====\n%s\n====", queryRunner.getCoordinator().getBaseUrl()); + } + } +} diff --git a/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/TeradataTestConstants.java b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/TeradataTestConstants.java new file mode 100644 index 000000000000..df34c711ca95 --- /dev/null +++ b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/TeradataTestConstants.java @@ -0,0 +1,22 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.teradata.integration; + +public final class TeradataTestConstants +{ + public static final String CLEARSCAPE_URL = "https://api.clearscape.teradata.com"; + public static final String CLEARSCAPE_USERNAME = "demo_user"; + + private TeradataTestConstants() {} +} diff --git a/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/TestTeradataConnectorTest.java b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/TestTeradataConnectorTest.java new file mode 100644 index 000000000000..a0e114129599 --- /dev/null +++ b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/TestTeradataConnectorTest.java @@ -0,0 +1,600 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.teradata.integration; + +import com.google.common.collect.ImmutableList; +import io.trino.Session; +import io.trino.plugin.jdbc.BaseJdbcConnectorTest; +import io.trino.sql.query.QueryAssertions; +import io.trino.testing.QueryRunner; +import io.trino.testing.TestingConnectorBehavior; +import io.trino.testing.TestingNames; +import io.trino.testing.assertions.TrinoExceptionAssert; +import io.trino.testing.sql.SqlExecutor; +import io.trino.testing.sql.TestTable; +import org.assertj.core.api.AssertProvider; +import org.intellij.lang.annotations.Language; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.parallel.ResourceAccessMode; +import org.junit.jupiter.api.parallel.ResourceLock; + +import java.util.List; +import java.util.Optional; +import java.util.OptionalInt; +import java.util.function.Consumer; + +import static io.trino.plugin.teradata.integration.clearscape.ClearScapeEnvironmentUtils.generateUniqueEnvName; +import static io.trino.spi.connector.ConnectorMetadata.MODIFYING_ROWS_MESSAGE; +import static io.trino.testing.TestingNames.randomNameSuffix; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assumptions.abort; + +final class TestTeradataConnectorTest + extends BaseJdbcConnectorTest +{ + private static final int TERADATA_OBJECT_NAME_LIMIT = 128; + + private TestingTeradataServer database; + + @Override + protected SqlExecutor onRemoteDatabase() + { + return database; + } + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + database = closeAfterClass(new TestingTeradataServer(generateUniqueEnvName(getClass()), true)); + return TeradataQueryRunner.builder(database).setInitialTables(REQUIRED_TPCH_TABLES).build(); + } + + @Override + protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) + { + return switch (connectorBehavior) { + case SUPPORTS_ADD_COLUMN, + SUPPORTS_AGGREGATION_PUSHDOWN, + SUPPORTS_COMMENT_ON_COLUMN, + SUPPORTS_COMMENT_ON_TABLE, + SUPPORTS_CREATE_MATERIALIZED_VIEW, + SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT, + SUPPORTS_CREATE_TABLE_WITH_DATA, + SUPPORTS_CREATE_TABLE_WITH_TABLE_COMMENT, + SUPPORTS_CREATE_VIEW, + SUPPORTS_DELETE, + SUPPORTS_DEREFERENCE_PUSHDOWN, + SUPPORTS_DROP_COLUMN, + SUPPORTS_DROP_NOT_NULL_CONSTRAINT, + SUPPORTS_DROP_SCHEMA_CASCADE, + SUPPORTS_INSERT, + SUPPORTS_JOIN_PUSHDOWN, + SUPPORTS_JOIN_PUSHDOWN_WITH_DISTINCT_FROM, + SUPPORTS_JOIN_PUSHDOWN_WITH_VARCHAR_INEQUALITY, + SUPPORTS_LIMIT_PUSHDOWN, + SUPPORTS_MAP_TYPE, + SUPPORTS_MERGE, + SUPPORTS_NATIVE_QUERY, + SUPPORTS_NEGATIVE_DATE, + SUPPORTS_PREDICATE_ARITHMETIC_EXPRESSION_PUSHDOWN, + SUPPORTS_PREDICATE_EXPRESSION_PUSHDOWN, + SUPPORTS_PREDICATE_PUSHDOWN, + SUPPORTS_PREDICATE_PUSHDOWN_WITH_VARCHAR_INEQUALITY, + SUPPORTS_RENAME_COLUMN, + SUPPORTS_RENAME_SCHEMA, + SUPPORTS_RENAME_TABLE, + SUPPORTS_ROW_LEVEL_DELETE, + SUPPORTS_ROW_TYPE, + SUPPORTS_SET_COLUMN_TYPE, + SUPPORTS_TOPN_PUSHDOWN, + SUPPORTS_TOPN_PUSHDOWN_WITH_VARCHAR, + SUPPORTS_TRUNCATE, + SUPPORTS_UPDATE -> false; + case SUPPORTS_CREATE_SCHEMA, + SUPPORTS_CREATE_TABLE -> true; + default -> super.hasBehavior(connectorBehavior); + }; + } + + @AfterAll + public void cleanupTestDatabase() + { + database = null; + } + + @Override + // Override because Teradata Object name limit is 128 characters + protected OptionalInt maxSchemaNameLength() + { + return OptionalInt.of(TERADATA_OBJECT_NAME_LIMIT); + } + + @Override + protected void verifySchemaNameLengthFailurePermissible(Throwable e) + { + assertThat(e) + .hasMessage(format("Schema name must be shorter than or equal to '%s' characters but got '%s'", TERADATA_OBJECT_NAME_LIMIT, TERADATA_OBJECT_NAME_LIMIT + 1)); + } + + @Override + // Override because Teradata Object name limit is 128 characters + protected OptionalInt maxColumnNameLength() + { + return OptionalInt.of(TERADATA_OBJECT_NAME_LIMIT); + } + + @Override + protected void verifyColumnNameLengthFailurePermissible(Throwable e) + { + assertThat(e) + .hasMessageMatching(format("Column name must be shorter than or equal to '%s' characters but got '%s': '.*'", TERADATA_OBJECT_NAME_LIMIT, TERADATA_OBJECT_NAME_LIMIT + 1)); + } + + @Override + @Test + public void testDataMappingSmokeTest() + { + skipTestUnless(false); + } + + @Override + // Override because Teradata Table name limit is 128 characters + protected OptionalInt maxTableNameLength() + { + return OptionalInt.of(TERADATA_OBJECT_NAME_LIMIT); + } + + @Override + // Override because the expected error message is different + protected void verifyTableNameLengthFailurePermissible(Throwable e) + { + assertThat(e) + .hasMessageMatching(format("Table name must be shorter than or equal to '%s' characters but got '%s'", TERADATA_OBJECT_NAME_LIMIT, TERADATA_OBJECT_NAME_LIMIT + 1)); + } + + @Override + // Overriding this test case as Teradata defines varchar with a length. + @Test + public void testVarcharCastToDateInPredicate() + { + String tableName = "varchar_as_date_pred"; + try (TestTable table = newTrinoTable( + tableName, + "(a varchar(50))", + ImmutableList.of( + "'999-09-09'", + "'1005-09-09'", + "'2005-06-06'", "'2005-06-6'", "'2005-6-06'", "'2005-6-6'", "' 2005-06-06'", "'2005-06-06 '", "' +2005-06-06'", "'02005-06-06'", + "'2005-09-06'", "'2005-09-6'", "'2005-9-06'", "'2005-9-6'", "' 2005-09-06'", "'2005-09-06 '", "' +2005-09-06'", "'02005-09-06'", + "'2005-09-09'", "'2005-09-9'", "'2005-9-09'", "'2005-9-9'", "' 2005-09-09'", "'2005-09-09 '", "' +2005-09-09'", "'02005-09-09'", + "'2005-09-10'", "'2005-9-10'", "' 2005-09-10'", "'2005-09-10 '", "' +2005-09-10'", "'02005-09-10'", + "'2005-09-20'", "'2005-9-20'", "' 2005-09-20'", "'2005-09-20 '", "' +2005-09-20'", "'02005-09-20'", + "'9999-09-09'", + "'99999-09-09'"))) { + for (String date : ImmutableList.of("2005-09-06", "2005-09-09", "2005-09-10")) { + for (String operator : ImmutableList.of("=", "<=", "<", ">", ">=", "!=", "IS DISTINCT FROM", "IS NOT DISTINCT FROM")) { + assertThat(query("SELECT a FROM %s WHERE CAST(a AS date) %s DATE '%s'".formatted(table.getName(), operator, date))) + .hasCorrectResultsRegardlessOfPushdown(); + } + } + } + try (TestTable table = newTrinoTable( + tableName, + "(a varchar(50))", + ImmutableList.of("'2005-06-bad-date'", "'2005-09-10'"))) { + assertThat(query("SELECT a FROM %s WHERE CAST(a AS date) < DATE '2005-09-10'".formatted(table.getName()))) + .failure().hasMessage("Value cannot be cast to date: " + "2005-06-bad-date"); + verifyResultOrFailure( + query("SELECT a FROM %s WHERE CAST(a AS date) = DATE '2005-09-10'".formatted(table.getName())), + queryAssert -> queryAssert.skippingTypesCheck().matches("VALUES '2005-09-10'"), + failureAssert -> failureAssert + .hasMessage("Value cannot be cast to date: " + "2005-06-bad-date")); + } + try (TestTable table = newTrinoTable( + tableName, + "(a varchar(50))", + ImmutableList.of("'2005-09-10'"))) { + // 2005-09-01, when written as 2005-09-1, is a prefix of an existing data point: 2005-09-10 + assertThat(query("SELECT a FROM %s WHERE CAST(a AS date) != DATE '2005-09-01'".formatted(table.getName()))) + .skippingTypesCheck().matches("VALUES '2005-09-10'"); + } + } + + @Override + // Overridden to handle Teradata specific WITH DATA syntax for table creation + @Test + public void testCreateTableAsSelect() + { + String tableName = "test_ctas" + randomNameSuffix(); + assertUpdate( + "CREATE TABLE IF NOT EXISTS " + tableName + " AS SELECT name, regionkey FROM nation", + "SELECT count(*) FROM nation"); + assertTableColumnNames(tableName, "name", "regionkey"); + assertThat(getTableComment(tableName)).isNull(); + assertUpdate("DROP TABLE " + tableName); + + assertUpdate( + "CREATE TABLE IF NOT EXISTS nation AS SELECT nationkey, regionkey FROM nation", + 0); + assertTableColumnNames("nation", "nationkey", "name", "regionkey", "comment"); + + assertCreateTableAsSelect( + "SELECT nationkey, name, regionkey FROM nation", + "SELECT count(*) FROM nation"); + + assertCreateTableAsSelect( + "SELECT mktsegment, sum(acctbal) x FROM customer GROUP BY mktsegment", + "SELECT count(DISTINCT mktsegment) FROM customer"); + + assertCreateTableAsSelect( + "SELECT count(*) x FROM nation JOIN region ON nation.regionkey = region.regionkey", + "SELECT 1"); + + assertCreateTableAsSelect( + "SELECT nationkey FROM nation ORDER BY nationkey LIMIT 10", + "SELECT 10"); + + assertCreateTableAsSelect( + "SELECT name, nationkey, regionkey FROM nation WHERE nationkey % 2 = 0 UNION ALL " + + "SELECT name, nationkey, regionkey FROM nation WHERE nationkey % 2 = 1", + "SELECT name, nationkey, regionkey FROM nation", + "SELECT count(*) FROM nation"); + + assertCreateTableAsSelect( + Session.builder(getSession()).setSystemProperty("redistribute_writes", "true").build(), + "SELECT CAST(nationkey AS BIGINT) nationkey, regionkey FROM nation UNION ALL " + + "SELECT 1234567890, 123", + "SELECT nationkey, regionkey FROM nation UNION ALL " + + "SELECT 1234567890, 123", + "SELECT count(*) + 1 FROM nation"); + + assertCreateTableAsSelect( + Session.builder(getSession()).setSystemProperty("redistribute_writes", "false").build(), + "SELECT CAST(nationkey AS BIGINT) nationkey, regionkey FROM nation UNION ALL " + + "SELECT 1234567890, 123", + "SELECT nationkey, regionkey FROM nation UNION ALL " + + "SELECT 1234567890, 123", + "SELECT count(*) + 1 FROM nation"); + + tableName = "test_ctas" + randomNameSuffix(); + assertThat(query("EXPLAIN ANALYZE CREATE TABLE " + tableName + " AS SELECT name FROM nation")).succeeds(); + assertThat(query("SELECT * from " + tableName)).matches("SELECT name FROM nation"); + assertUpdate("DROP TABLE " + tableName); + } + + @Override + // Overriding this test case as Teradata does not support negative dates. + @Test + public void testDateYearOfEraPredicate() + { + assertQuery( + "SELECT orderdate FROM orders WHERE orderdate = DATE '1997-09-14'", + "VALUES DATE '1997-09-14'"); + } + + @Override + // Override this test case as Teradata has different syntax for creating tables with AS SELECT statement. + @Test + public void verifySupportsRowLevelUpdateDeclaration() + { + String testTableName = "test_supports_update"; + try (TestTable table = newTrinoTable(testTableName, "AS ( SELECT * FROM nation) WITH DATA")) { + assertQueryFails( + "UPDATE " + table.getName() + " SET nationkey = nationkey * 100 WHERE regionkey = 2", + MODIFYING_ROWS_MESSAGE); + } + } + + @Override + // Overriding this test case as Teradata doesn't have support to (k, v) AS VALUES in insert statement + @Test + public void testCharVarcharComparison() + { + String testTableName = "test_char_varchar"; + try (TestTable table = newTrinoTable( + testTableName, + "(k int, v char(3))", + ImmutableList.of( + "-1, CAST(NULL AS char(3))", + "3, CAST(' ' AS char(3))", + "6, CAST('x ' AS char(3))"))) { + assertQuery( + "SELECT k, v FROM " + table.getName() + " WHERE v = CAST(' ' AS varchar(2))", + "VALUES (3, ' ')"); + assertQuery( + "SELECT k, v FROM " + table.getName() + " WHERE v = CAST(' ' AS varchar(4))", + "VALUES (3, ' ')"); + assertQuery( + "SELECT k, v FROM " + table.getName() + " WHERE v = CAST('x ' AS varchar(2))", + "VALUES (6, 'x ')"); + } + } + + @Override + // Overriding this test case as Teradata doesn't have support to (k, v) AS VALUES in insert statement + @Test + public void testVarcharCharComparison() + { + try (TestTable table = newTrinoTable( + "test_varchar_char", + "(k int, v char(3))", + ImmutableList.of( + "-1, CAST(NULL AS varchar(3))", + "0, CAST('' AS varchar(3))", + "1, CAST(' ' AS varchar(3))", + "2, CAST(' ' AS varchar(3))", + "3, CAST(' ' AS varchar(3))", + "4, CAST('x' AS varchar(3))", + "5, CAST('x ' AS varchar(3))", + "6, CAST('x ' AS " + "varchar(3))"))) { + assertQuery( + "SELECT k, v FROM " + table.getName() + " WHERE v = CAST(' ' AS char(2))", + "VALUES (0, ' '), (1, ' '), (2, ' '), (3, ' ')"); + assertQuery( + "SELECT k, v FROM " + table.getName() + " WHERE v = CAST('x ' AS char(2))", + "VALUES (4, 'x '), (5, 'x '), (6, 'x ')"); + } + } + + @Override + // Overriding to add ResourceLock to run sequential this test along with other tests labeled with TERADATA_SCHEMA to avoid issue Concurrent change conflict on database + @Test + @ResourceLock(value = "TERADATA_SCHEMA", mode = ResourceAccessMode.READ_WRITE) + public void testShowCreateSchema() + { + super.testShowCreateSchema(); + } + + @Override + // Overriding to add ResourceLock to run sequential this test along with other tests labeled with TERADATA_SCHEMA to avoid issue Concurrent change conflict on database + @Test + @ResourceLock(value = "TERADATA_SCHEMA", mode = ResourceAccessMode.READ_WRITE) + public void testCreateSchema() + { + super.testCreateSchema(); + } + + @Override + // Overriding to add ResourceLock to run sequential this test along with other tests labeled with TERADATA_SCHEMA to avoid issue Concurrent change conflict on database + @Test + @ResourceLock(value = "TERADATA_SCHEMA", mode = ResourceAccessMode.READ_WRITE) + public void testCreateSchemaWithLongName() + { + super.testCreateSchemaWithLongName(); + } + + @Override + // Overriding as Teradata.query method allows SELECT statements + @Test + public void testExecuteProcedureWithInvalidQuery() + { + assertQuerySucceeds("CALL system.execute('SELECT 1')"); + assertQueryFails( + "CALL system.execute('invalid')", + ".*Syntax error: expected something between the beginning of the request and the word 'invalid'.*"); + } + + @Override + // Overriding to add ResourceLock to run sequential this test along with other tests labeled with TERADATA_SCHEMA to avoid issue Concurrent change conflict on database + @Test + @ResourceLock(value = "TERADATA_SCHEMA", mode = ResourceAccessMode.READ_WRITE) + public void testRenameSchemaToLongName() + { + super.testRenameSchemaToLongName(); + } + + @Override + // Overriding to add ResourceLock to run sequential this test along with other tests labeled with TERADATA_SCHEMA to avoid issue Concurrent change conflict on database + @Test + @ResourceLock(value = "TERADATA_SCHEMA", mode = ResourceAccessMode.READ_WRITE) + public void testRenameTableAcrossSchema() + throws Exception + { + super.testRenameTableAcrossSchema(); + } + + @Override + // Overriding to add ResourceLock to run sequential this test along with other tests labeled with TERADATA_SCHEMA to avoid issue Concurrent change conflict on database + @Test + @ResourceLock(value = "TERADATA_SCHEMA", mode = ResourceAccessMode.READ_WRITE) + public void testRenameTableToUnqualifiedPreservesSchema() + throws Exception + { + super.testRenameTableToUnqualifiedPreservesSchema(); + } + + @Override + // Overriding to tag this test as long_run test case to avoid running in clearscape_tests profile + @Test + @Tag("long_run") + public void testSelectInformationSchemaColumns() + { + super.testSelectInformationSchemaColumns(); + } + + @Override + // Overriding to tag this test as long_run test case to avoid running in clearscape_tests profile + @Test + @Tag("long_run") + public void testCaseSensitiveDataMapping() + { + super.testCaseSensitiveDataMapping(); + } + + @Override + // Overriding as Teradata does not support insert operations. Base implementation does not have check insert support before running the test. + @Test + public void testInsertIntoNotNullColumn() + { + abort("Skipping as connector does not support insert operations"); + } + + @Override + // Overriding as Teradata does not support insert operations. Base implementation does not have check insert support before running the test. + @Test + public void testInsertWithoutTemporaryTable() + { + abort("Skipping as connector does not support insert operations"); + } + + @Override + // Overriding as base test tyring to insert data but this connector not support insert operations. + @Test + public void testColumnName() + { + abort("Skipping as connector does not support column level write operations"); + } + + @Override + // Overriding as this connector does not support creating table with UNICODE characters + @Test + public void testCreateTableAsSelectWithUnicode() + { + abort("Skipping as connector does not support creating table with UNICODE characters"); + } + + @Override + // Overriding as Teradata does not support insert operations. Base implementation does not have check insert support before running the test. + @Test + public void testUpdateNotNullColumn() + { + abort("Skipping as connector does not support insert operations"); + } + + @Override + // Overriding as Teradata does not support insert operations. Base implementation does not have check insert support before running the test. + @Test + public void testWriteBatchSizeSessionProperty() + { + abort("Skipping as connector does not support insert operations"); + } + + @Override + // Overriding as Teradata does not support insert operations. Base implementation does not have check insert support before running the test. + @Test + public void testWriteTaskParallelismSessionProperty() + { + abort("Skipping as connector does not support insert operations"); + } + + @Test + void testTeradataNumberDataType() + { + try (TestTable table = newTrinoTable( + "test_number", + "(id INTEGER, number_col NUMBER(10,2), number_default NUMBER, number_large NUMBER(38,10))", + ImmutableList.of( + "1, CAST(12345.67 AS NUMBER(10,2)), CAST(999999999999999 AS NUMBER), CAST(1234567890123456789012345678.1234567890 AS NUMBER(38,10))", + "2, CAST(-99999.99 AS NUMBER(10,2)), CAST(-123456789012345 AS NUMBER), CAST(-9999999999999999999999999999.9999999999 AS NUMBER(38,10))", + "3, CAST(0.00 AS NUMBER(10,2)), CAST" + "(0 AS NUMBER), CAST(0.0000000000 AS NUMBER(38,10))"))) { + assertThat(query(format("SELECT number_col FROM %s WHERE id = 1", table.getName()))) + .matches("VALUES CAST(12345.67 AS DECIMAL(10,2))"); + assertThat(query(format("SELECT number_default FROM %s WHERE id = 1", table.getName()))) + .matches("VALUES CAST(999999999999999 AS DECIMAL(38,0))"); + assertThat(query(format("SELECT number_large FROM %s WHERE id = 1", table.getName()))) + .matches("VALUES CAST(1234567890123456789012345678.1234567890 AS DECIMAL(38,10))"); + assertThat(query(format("SELECT number_col FROM %s WHERE id = 2", table.getName()))) + .matches("VALUES CAST(-99999.99 AS DECIMAL(10,2))"); + assertThat(query(format("SELECT number_col FROM %s WHERE id = 3", table.getName()))) + .matches("VALUES CAST(0.00 AS DECIMAL(10,2))"); + } + } + + @Test + void testTeradataCharacterDataType() + { + try (TestTable table = newTrinoTable( + "test_character", + "(id INTEGER, char_col CHARACTER(5), char_default CHARACTER, char_large CHARACTER(100))", + ImmutableList.of( + "1, CAST('HELLO' AS CHARACTER(5)), CAST('A' AS CHARACTER), CAST('TERADATA' AS CHARACTER(100))", + "2, CAST('WORLD' AS CHARACTER(5)), CAST('B' AS CHARACTER), CAST('CHARACTER' AS CHARACTER(100))", + "3, CAST('' AS CHARACTER(5)), CAST('C' AS CHARACTER), CAST('' AS CHARACTER(100))"))) { + assertThat(query(format("SELECT char_col FROM %s WHERE id = 1", table.getName()))) + .matches("VALUES CAST('HELLO' AS CHAR(5))"); + assertThat(query(format("SELECT char_default FROM %s WHERE id = 1", table.getName()))) + .matches("VALUES CAST('A' AS CHAR(1))"); + assertThat(query(format("SELECT char_large FROM %s WHERE id = 1", table.getName()))) + .matches("VALUES CAST('TERADATA' AS CHAR(100))"); + assertThat(query(format("SELECT char_col FROM %s WHERE id = 3", table.getName()))) + .matches("VALUES CAST('' AS CHAR(5))"); + } + } + + @Override + // Overridden to exclude data types that Teradata doesn't support or handles differently + protected Optional filterDataMappingSmokeTestData(DataMappingTestSetup dataMappingTestSetup) + { + String typeName = dataMappingTestSetup.getTrinoTypeName(); + return switch (typeName) { + // skipping date as during julian->gregorian date is handled differently in Teradata. + // tinyint, double and varchar with unbounded (need to handle special characters) are skipped and will handle it while improving write functionalities. + case "boolean", + "tinyint", + "date", + "real", + "double", + "varchar", + "time", + "time(6)", + "timestamp", + "timestamp(6)", + "varbinary", + "timestamp(3) with time zone", + "timestamp(6) with time zone", + "U&'a \\000a newline'" -> Optional.empty(); + default -> Optional.of(dataMappingTestSetup); + }; + } + + @Override + // Overridden to use Teradata WITH DATA syntax for CREATE TABLE AS SELECT statements + protected void assertCreateTableAsSelect(Session session, String query, String expectedQuery, String rowCountQuery) + { + String table = "test_ctas_" + TestingNames.randomNameSuffix(); + assertUpdate(session, "CREATE TABLE " + table + " AS ( " + query + ") WITH DATA", rowCountQuery); + assertQuery(session, "SELECT * FROM " + table, expectedQuery); + assertUpdate(session, "DROP TABLE " + table); + assertThat(getQueryRunner().tableExists(session, table)).isFalse(); + } + + @Override + // Overridden to handle Teradata schema.table naming format and table creation syntax + protected TestTable newTrinoTable(String namePrefix, @Language("SQL") String tableDefinition, List rowsToInsert) + { + String tableName; + if (namePrefix.contains(".")) { + tableName = namePrefix; + } + else { + String schemaName = getSession().getSchema().orElseThrow(); + tableName = schemaName + "." + namePrefix; + } + return new TestTable(database, tableName, tableDefinition, rowsToInsert); + } + + private static void verifyResultOrFailure(AssertProvider queryAssertProvider, Consumer verifyResults, + Consumer verifyFailure) + { + requireNonNull(verifyResults, "verifyResults is null"); + requireNonNull(verifyFailure, "verifyFailure is null"); + QueryAssertions.QueryAssert queryAssert = assertThat(queryAssertProvider); + verifyResults.accept(queryAssert); + } +} diff --git a/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/TestTeradataTypeMapping.java b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/TestTeradataTypeMapping.java new file mode 100644 index 000000000000..8dc919c48759 --- /dev/null +++ b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/TestTeradataTypeMapping.java @@ -0,0 +1,281 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.teradata.integration; + +import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.QueryRunner; +import io.trino.testing.datatype.CreateAndInsertDataSetup; +import io.trino.testing.datatype.DataSetup; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Test; + +import java.sql.SQLException; + +import static io.trino.plugin.teradata.integration.clearscape.ClearScapeEnvironmentUtils.generateUniqueEnvName; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.CharType.createCharType; +import static io.trino.spi.type.DateType.DATE; +import static io.trino.spi.type.DecimalType.createDecimalType; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static io.trino.spi.type.IntegerType.INTEGER; +import static io.trino.spi.type.SmallintType.SMALLINT; +import static io.trino.spi.type.TinyintType.TINYINT; +import static io.trino.spi.type.VarcharType.createVarcharType; +import static io.trino.testing.datatype.SqlDataTypeTest.create; +import static java.lang.String.format; +import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy; + +final class TestTeradataTypeMapping + extends AbstractTestQueryFramework +{ + private TestingTeradataServer database; + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + database = closeAfterClass(new TestingTeradataServer(generateUniqueEnvName(getClass()), true)); + return TeradataQueryRunner.builder(database).build(); + } + + @AfterAll + void cleanupTestClass() + { + database = null; + } + + @Test + void testByteint() + { + create() + .addRoundTrip("byteint", "0", TINYINT, "CAST(0 AS TINYINT)") + .addRoundTrip("byteint", "127", TINYINT, "CAST(127 AS TINYINT)") + .addRoundTrip("byteint", "-128", TINYINT, "CAST(-128 AS TINYINT)") + .addRoundTrip("byteint", "null", TINYINT, "CAST(null AS TINYINT)") + .execute(getQueryRunner(), testInsertIntoNotNullColumn("byteint")); + } + + @Test + void testSmallint() + { + create() + .addRoundTrip("smallint", "0", SMALLINT, "CAST(0 AS SMALLINT)") + .addRoundTrip("smallint", "32767", SMALLINT, "CAST(32767 AS SMALLINT)") + .addRoundTrip("smallint", "-32768", SMALLINT, "CAST(-32768 AS SMALLINT)") + .addRoundTrip("smallint", "null", SMALLINT, "CAST(null AS SMALLINT)") + .execute(getQueryRunner(), testInsertIntoNotNullColumn("smallint")); + } + + @Test + void testInteger() + { + create() + .addRoundTrip("integer", "0", INTEGER, "0") + .addRoundTrip("integer", "2147483647", INTEGER, "2147483647") + .addRoundTrip("integer", "-2147483648", INTEGER, "-2147483648") + .addRoundTrip("integer", "NULL", INTEGER, "CAST(NULL AS INTEGER)") + .execute(getQueryRunner(), testInsertIntoNotNullColumn("integer")); + } + + @Test + void testBigint() + { + create() + .addRoundTrip("bigint", "0", BIGINT, "CAST(0 AS BIGINT)") + .addRoundTrip("bigint", "9223372036854775807", BIGINT, "9223372036854775807") + .addRoundTrip("bigint", "-9223372036854775808", BIGINT, "-9223372036854775808") + .addRoundTrip("bigint", "NULL", BIGINT, "CAST(NULL AS BIGINT)") + .execute(getQueryRunner(), testInsertIntoNotNullColumn("bigint")); + } + + @Test + void testFloat() + { + create() + .addRoundTrip("float", "0", DOUBLE, "CAST(0 AS DOUBLE)") + .addRoundTrip("real", "0", DOUBLE, "CAST(0 AS DOUBLE)") + .addRoundTrip("double precision", "0", DOUBLE, "CAST(0 AS DOUBLE)") + .addRoundTrip("float", "1.797e308", DOUBLE, "1.797e308") + .addRoundTrip("real", "1.797e308", DOUBLE, "1.797e308") + .addRoundTrip("double precision", "1.797e308", DOUBLE, "1.797e308") + .addRoundTrip("float", "2.226e-308", DOUBLE, "2.226e-308") + .addRoundTrip("real", "2.226e-308", DOUBLE, "2.226e-308") + .addRoundTrip("double precision", "2.226e-308", DOUBLE, "2.226e-308") + .addRoundTrip("float", "NULL", DOUBLE, "CAST(NULL AS DOUBLE)") + .addRoundTrip("real", "NULL", DOUBLE, "CAST(NULL AS DOUBLE)") + .addRoundTrip("double precision", "NULL", DOUBLE, "CAST(NULL AS DOUBLE)") + .execute(getQueryRunner(), testInsertIntoNotNullColumn("float")); + } + + @Test + void testDecimal() + { + create() + .addRoundTrip("decimal(3, 0)", "0", createDecimalType(3, 0), "CAST('0' AS decimal(3, 0))") + .addRoundTrip("numeric(3, 0)", "0", createDecimalType(3, 0), "CAST('0' AS decimal(3, 0))") + .addRoundTrip("decimal(3, 1)", "0.0", createDecimalType(3, 1), "CAST('0.0' AS decimal(3, 1))") + .addRoundTrip("numeric(3, 1)", "0.0", createDecimalType(3, 1), "CAST('0.0' AS decimal(3, 1))") + .addRoundTrip("decimal(1, 0)", "1", createDecimalType(1, 0), "CAST('1' AS decimal(1, 0))") + .addRoundTrip("numeric(1, 0)", "1", createDecimalType(1, 0), "CAST('1' AS decimal(1, 0))") + .addRoundTrip("decimal(1, 0)", "-1", createDecimalType(1, 0), "CAST('-1' AS decimal(1, 0))") + .addRoundTrip("numeric(1, 0)", "-1", createDecimalType(1, 0), "CAST('-1' AS decimal(1, 0))") + .addRoundTrip("decimal(3, 0)", "1", createDecimalType(3, 0), "CAST('1' AS decimal(3, 0))") + .addRoundTrip("numeric(3, 0)", "1", createDecimalType(3, 0), "CAST('1' AS decimal(3, 0))") + .addRoundTrip("decimal(3, 0)", "-1", createDecimalType(3, 0), "CAST('-1' AS decimal(3, 0))") + .addRoundTrip("numeric(3, 0)", "-1", createDecimalType(3, 0), "CAST('-1' AS decimal(3, 0))") + .addRoundTrip("decimal(3, 0)", "123", createDecimalType(3, 0), "CAST('123' AS decimal(3, 0))") + .addRoundTrip("numeric(3, 0)", "123", createDecimalType(3, 0), "CAST('123' AS decimal(3, 0))") + .addRoundTrip("decimal(3, 0)", "-123", createDecimalType(3, 0), "CAST('-123' AS decimal(3, 0))") + .addRoundTrip("numeric(3, 0)", "-123", createDecimalType(3, 0), "CAST('-123' AS decimal(3, 0))") + .addRoundTrip("decimal(3, 1)", "10.0", createDecimalType(3, 1), "CAST('10.0' AS decimal(3, 1))") + .addRoundTrip("numeric(3, 1)", "10.0", createDecimalType(3, 1), "CAST('10.0' AS decimal(3, 1))") + .addRoundTrip("decimal(3, 1)", "12.3", createDecimalType(3, 1), "CAST('12.3' AS decimal(3, 1))") + .addRoundTrip("numeric(3, 1)", "12.3", createDecimalType(3, 1), "CAST('12.3' AS decimal(3, 1))") + .addRoundTrip("decimal(3, 1)", "-12.3", createDecimalType(3, 1), "CAST('-12.3' AS decimal(3, 1))") + .addRoundTrip("numeric(3, 1)", "-12.3", createDecimalType(3, 1), "CAST('-12.3' AS decimal(3, 1))") + .addRoundTrip("decimal(38, 0)", "12345678901234567890123456789012345678", createDecimalType(38, 0), "CAST('12345678901234567890123456789012345678' AS decimal(38, 0))") + .addRoundTrip("numeric(38, 0)", "12345678901234567890123456789012345678", createDecimalType(38, 0), "CAST('12345678901234567890123456789012345678' AS decimal(38, 0))") + .addRoundTrip("decimal(38, 0)", "-12345678901234567890123456789012345678", createDecimalType(38, 0), "CAST('-12345678901234567890123456789012345678' AS decimal(38, 0))") + .addRoundTrip("numeric(38, 0)", "-12345678901234567890123456789012345678", createDecimalType(38, 0), "CAST('-12345678901234567890123456789012345678' AS decimal(38, 0))") + .addRoundTrip("decimal(1, 0)", "null", createDecimalType(1, 0), "CAST(null AS decimal(1, 0))") + .execute(getQueryRunner(), testInsertIntoNotNullColumn("decimal")); + } + + @Test + void testNumber() + { + create() + .addRoundTrip("numeric(3)", "0", createDecimalType(3, 0), "CAST('0' AS decimal(3, 0))") + .addRoundTrip("number(5,2)", "0", createDecimalType(5, 2), "CAST('0' AS decimal(5, 2))") + .addRoundTrip("number(38)", "0", createDecimalType(38, 0), "CAST('0' AS decimal(38, 0))") + .addRoundTrip("number(38,2)", "123456789012345678901234567890123456.78", createDecimalType(38, 2), "CAST('123456789012345678901234567890123456.78' AS decimal(38, 2))") + .addRoundTrip("numeric(38)", "12345678901234567890123456789012345678", createDecimalType(38, 0), "CAST('12345678901234567890123456789012345678' AS decimal(38, 0))") + .addRoundTrip("numeric(3)", "null", createDecimalType(3, 0), "CAST(null AS decimal(3, 0))") + .execute(getQueryRunner(), testInsertIntoNotNullColumn("number")); + } + + @Test + void testChar() + { + create() + .addRoundTrip("char(3)", "''", createCharType(3), "CAST('' AS char(3))") + .addRoundTrip("char(3)", "' '", createCharType(3), "CAST(' ' AS char(3))") + .addRoundTrip("char(3)", "' '", createCharType(3), "CAST(' ' AS char(3))") + .addRoundTrip("char(3)", "' '", createCharType(3), "CAST(' ' AS char(3))") + .addRoundTrip("char(3)", "'A'", createCharType(3), "CAST('A' AS char(3))") + .addRoundTrip("char(3)", "'A '", createCharType(3), "CAST('A ' AS char(3))") + .addRoundTrip("char(3)", "' B '", createCharType(3), "CAST(' B ' AS char(3))") + .addRoundTrip("char(3)", "' C'", createCharType(3), "CAST(' C' AS char(3))") + .addRoundTrip("char(3)", "'AB'", createCharType(3), "CAST('AB' AS char(3))") + .addRoundTrip("char(3)", "'ABC'", createCharType(3), "CAST('ABC' AS char(3))") + .addRoundTrip("char(3)", "'A C'", createCharType(3), "CAST('A C' AS char(3))") + .addRoundTrip("char(3)", "' BC'", createCharType(3), "CAST(' BC' AS char(3))") + .addRoundTrip("char(3)", "null", createCharType(3), "CAST(null AS char(3))") + .execute(getQueryRunner(), testInsertIntoNotNullColumn("char")); + String tmode = database.getTMode(); + if (tmode.equals("TERA")) { + // truncation + create() + .addRoundTrip("char(3)", "'ABCD'", createCharType(3), "CAST('ABCD' AS char(3))") + .execute(getQueryRunner(), testInsertIntoNotNullColumn("chart")); + } + else { + // Error on truncation + assertThatThrownBy(() -> + create() + .addRoundTrip("char(3)", "'ABCD'", createCharType(3), "CAST('ABCD' AS char(3))") + .execute(getQueryRunner(), testInsertIntoNotNullColumn("chart"))) + .isInstanceOf(RuntimeException.class) + .hasCauseInstanceOf(SQLException.class) + .cause() + .hasMessageContaining("Right truncation of string data"); + } + // max-size + create() + .addRoundTrip("char(64000)", "'max'", createCharType(64000), "CAST('max' AS char(64000))") + .execute(getQueryRunner(), testInsertIntoNotNullColumn("charl")); + } + + @Test + void testVarchar() + { + create() + .addRoundTrip("varchar(32)", "''", createVarcharType(32), "CAST('' AS varchar(32))") + .addRoundTrip("varchar(32)", "' '", createVarcharType(32), "CAST(' ' AS varchar(32))") + .addRoundTrip("varchar(32)", "' '", createVarcharType(32), "CAST(' ' AS varchar(32))") + .addRoundTrip("varchar(32)", "' '", createVarcharType(32), "CAST(' ' AS varchar(32))") + .addRoundTrip("varchar(32)", "' '", createVarcharType(32), "CAST(' ' AS varchar(32))") + .addRoundTrip("varchar(32)", "'A'", createVarcharType(32), "CAST('A' AS varchar(32))") + .addRoundTrip("varchar(32)", "'A '", createVarcharType(32), "CAST('A ' AS varchar(32))") + .addRoundTrip("varchar(32)", "' B '", createVarcharType(32), "CAST(' B ' AS varchar(32))") + .addRoundTrip("varchar(32)", "' C'", createVarcharType(32), "CAST(' C' AS varchar(32))") + .addRoundTrip("varchar(32)", "'AB'", createVarcharType(32), "CAST('AB' AS varchar(32))") + .addRoundTrip("varchar(32)", "'ABC'", createVarcharType(32), "CAST('ABC' AS varchar(32))") + .addRoundTrip("varchar(32)", "'A C'", createVarcharType(32), "CAST('A C' AS varchar(32))") + .addRoundTrip("varchar(32)", "' BC'", createVarcharType(32), "CAST(' BC' AS varchar(32))") + .addRoundTrip("varchar(32)", "null", createVarcharType(32), "CAST(null AS varchar(32))") + .execute(getQueryRunner(), testInsertIntoNotNullColumn("varchar")); + String teraMode = database.getTMode(); + if (teraMode.equals("TERA")) { + // truncation + create() + .addRoundTrip("varchar(3)", "'ABCD'", createVarcharType(3), "CAST('ABCD' AS varchar(3))") + .execute(getQueryRunner(), testInsertIntoNotNullColumn("varchart")); + } + else { + // Error on truncation + assertThatThrownBy(() -> + create() + .addRoundTrip("varchar(3)", "'ABCD'", createVarcharType(3), "CAST('ABCD' AS varchar(3))") + .execute(getQueryRunner(), testInsertIntoNotNullColumn("varchart"))) + .isInstanceOf(RuntimeException.class) + .hasCauseInstanceOf(SQLException.class) + .cause() + .hasMessageContaining("Right truncation of string data"); + } + // max-size + create() + .addRoundTrip("long varchar", "'max'", createVarcharType(64000), "CAST('max' AS varchar(64000))") + .execute(getQueryRunner(), testInsertIntoNotNullColumn("varcharl")); + } + + @Test + void testDate() + { + create() + .addRoundTrip("date", "DATE '0001-01-01'", DATE, "DATE '0001-01-01'") + .addRoundTrip("date", "DATE '0012-12-12'", DATE, "DATE '0012-12-12'") + .addRoundTrip("date", "DATE '1500-01-01'", DATE, "DATE '1500-01-01'") + .addRoundTrip("date", "DATE '1582-10-04'", DATE, "DATE '1582-10-04'") + .addRoundTrip("date", "DATE '1582-10-15'", DATE, "DATE '1582-10-15'") + .addRoundTrip("date", "DATE '1952-04-03'", DATE, "DATE '1952-04-03'") + .addRoundTrip("date", "DATE '1970-01-01'", DATE, "DATE '1970-01-01'") + .addRoundTrip("date", "DATE '1970-02-03'", DATE, "DATE '1970-02-03'") + .addRoundTrip("date", "DATE '1970-01-01'", DATE, "DATE '1970-01-01'") + .addRoundTrip("date", "DATE '1983-04-01'", DATE, "DATE '1983-04-01'") + .addRoundTrip("date", "DATE '1983-10-01'", DATE, "DATE '1983-10-01'") + .addRoundTrip("date", "DATE '2017-07-01'", DATE, "DATE '2017-07-01'") + .addRoundTrip("date", "DATE '2017-01-01'", DATE, "DATE '2017-01-01'") + .addRoundTrip("date", "DATE '2024-02-29'", DATE, "DATE '2024-02-29'") + .addRoundTrip("date", "DATE '9999-12-30'", DATE, "DATE '9999-12-30'") + .addRoundTrip("date", "NULL", DATE, "CAST(NULL AS DATE)") + .execute(getQueryRunner(), testInsertIntoNotNullColumn("date")); + } + + private DataSetup testInsertIntoNotNullColumn(String tableNamePrefix) + { + String prefix = format("%s.%s", database.getDatabaseName(), tableNamePrefix); + return new CreateAndInsertDataSetup(database, prefix); + } +} diff --git a/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/TestingTeradataServer.java b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/TestingTeradataServer.java new file mode 100644 index 000000000000..5ff677a70591 --- /dev/null +++ b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/TestingTeradataServer.java @@ -0,0 +1,399 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.teradata.integration; + +import io.trino.plugin.teradata.integration.clearscape.ClearScapeSetup; +import io.trino.plugin.teradata.integration.clearscape.EnvironmentResponse; +import io.trino.plugin.teradata.integration.clearscape.Model; +import io.trino.testing.sql.SqlExecutor; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; +import java.util.Random; + +import static io.trino.testing.SystemEnvironmentUtils.isEnvSet; +import static io.trino.testing.SystemEnvironmentUtils.requireEnv; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public final class TestingTeradataServer + implements AutoCloseable, SqlExecutor +{ + private static final int MAX_RETRIES = 5; + private static final long BASE_RETRY_DELAY_MS = 1500L; + private static final long MAX_RETRY_DELAY_MS = 10_000L; + private static final Random RANDOM = new Random(); + + private volatile Connection connection; + private DatabaseConfig config; + private ClearScapeSetup clearScapeSetup; + + public TestingTeradataServer(String envName, boolean destroyEnv) + { + requireNonNull(envName, "envName should not be null"); + config = DatabaseConfigFactory.create(envName); + String hostName = config.getHostName(); + + // Initialize ClearScape Instance and get hostname from ClearScape API when used + if (config.isUseClearScape()) { + if (isEnvSet("CLEARSCAPE_DESTROY_ENV")) { + destroyEnv = Boolean.parseBoolean(requireEnv("CLEARSCAPE_DESTROY_ENV")); + } + clearScapeSetup = new ClearScapeSetup( + requireEnv("CLEARSCAPE_TOKEN"), + requireEnv("CLEARSCAPE_PASSWORD"), + config.getClearScapeEnvName(), + destroyEnv, + requireEnv("CLEARSCAPE_REGION")); + Model model = clearScapeSetup.initialize(); + hostName = model.getHostName(); + } + String jdbcUrl = buildJdbcUrl(hostName); + config = config.toBuilder() + .hostName(hostName) + .jdbcUrl(jdbcUrl) + .build(); + // Recreate the connection with retries to handle transient ClearScape socket or connection closure issues. + connection = createConnectionWithRetries(); + createTestDatabaseIfAbsent(); + } + + public Map fetchCatalogProperties() + { + Map properties = new HashMap<>(); + properties.put("connection-url", config.getJdbcUrl()); + + AuthenticationConfig auth = config.getAuthConfig(); + properties.put("connection-user", auth.userName()); + properties.put("connection-password", auth.password()); + + return properties; + } + + public void createTestDatabaseIfAbsent() + { + executeWithRetry(() -> { + if (!schemaExists(config.getDatabaseName())) { + execute(format("CREATE DATABASE \"%s\" AS PERM=100e6;", config.getDatabaseName())); + } + }); + } + + public void dropTestDatabaseIfExists() + { + executeWithRetry(() -> { + if (schemaExists(config.getDatabaseName())) { + execute(format("DELETE DATABASE \"%s\"", config.getDatabaseName())); + execute(format("DROP DATABASE \"%s\"", config.getDatabaseName())); + } + }); + } + + public boolean isTableExists(String tableName) + { + ensureConnection(); + String query = "SELECT count(1) FROM DBC.TablesV WHERE DataBaseName = ? AND TableName = ?"; + try (PreparedStatement stmt = connection.prepareStatement(query)) { + stmt.setString(1, config.getDatabaseName()); + stmt.setString(2, tableName); + try (ResultSet rs = stmt.executeQuery()) { + return rs.next() && rs.getInt(1) > 0; + } + } + catch (SQLException e) { + if (isConnectionException(e)) { + connection = createConnectionWithRetries(); + try (PreparedStatement stmt = connection.prepareStatement(query)) { + stmt.setString(1, config.getDatabaseName()); + stmt.setString(2, tableName); + try (ResultSet rs = stmt.executeQuery()) { + return rs.next() && rs.getInt(1) > 0; + } + } + catch (SQLException ex) { + throw new RuntimeException("Failed to check table existence: " + ex.getMessage(), ex); + } + } + throw new RuntimeException("Failed to check table existence: " + e.getMessage(), e); + } + } + + @Override + public void execute(String sql) + { + executeWithRetry(() -> doExecute(sql)); + } + + public String getDatabaseName() + { + return config.getDatabaseName(); + } + + public String getTMode() + { + return config.getTMode(); + } + + @Override + public void close() + { + try { + if (config.isUseClearScape()) { + EnvironmentResponse.State state = clearScapeSetup.status(); + if (state == EnvironmentResponse.State.RUNNING) { + dropTestDatabaseIfExists(); + } + } + else { + dropTestDatabaseIfExists(); + } + } + finally { + try { + if (connection != null && !connection.isClosed()) { + connection.close(); + } + } + catch (SQLException ignored) { + } + connection = null; + if (clearScapeSetup != null) { + try { + clearScapeSetup.cleanup(); + } + catch (Exception ignored) { + } + } + } + } + + @Override + public boolean supportsMultiRowInsert() + { + return false; + } + + private String buildJdbcUrl(String hostName) + { + String baseUrl = format("jdbc:teradata://%s/", hostName); + String propertiesString = buildPropertiesString(); + return propertiesString.isEmpty() ? baseUrl : baseUrl + propertiesString; + } + + private String buildPropertiesString() + { + Map properties = config.getJdbcProperties(); + if (properties == null || properties.isEmpty()) { + return ""; + } + return properties.entrySet() + .stream() + .map(entry -> entry.getKey() + "=" + entry.getValue()) + .collect(java.util.stream.Collectors.joining(",")); + } + + private void doExecute(String sql) + { + ensureConnection(); + try (Statement stmt = connection.createStatement()) { + if (config.getDatabaseName() != null && schemaExists(config.getDatabaseName())) { + stmt.execute(format("DATABASE \"%s\"", config.getDatabaseName())); + } + stmt.execute(sql); + } + catch (SQLException e) { + throw new RuntimeException("SQL execution failed: " + sql, e); + } + } + + private boolean schemaExists(String schemaName) + { + ensureConnection(); + String query = "SELECT COUNT(1) FROM DBC.DatabasesV WHERE DatabaseName = ?"; + try (PreparedStatement stmt = connection.prepareStatement(query)) { + stmt.setString(1, schemaName); + try (ResultSet rs = stmt.executeQuery()) { + return rs.next() && rs.getInt(1) > 0; + } + } + catch (SQLException e) { + if (isConnectionException(e)) { + connection = createConnectionWithRetries(); + try (PreparedStatement stmt = connection.prepareStatement(query)) { + stmt.setString(1, schemaName); + try (ResultSet rs = stmt.executeQuery()) { + return rs.next() && rs.getInt(1) > 0; + } + } + catch (SQLException ex) { + throw new RuntimeException("Failed to check schema existence", ex); + } + } + throw new RuntimeException("Failed to check schema existence", e); + } + } + + private synchronized void ensureConnection() + { + try { + if (connection == null || connection.isClosed()) { + connection = createConnectionWithRetries(); + } + } + catch (SQLException e) { + connection = createConnectionWithRetries(); + } + } + + private void executeWithRetry(Runnable operation) + { + int attempt = 0; + + while (true) { + try { + operation.run(); + return; + } + catch (RuntimeException e) { + attempt++; + Throwable cause = e.getCause(); + + // Connection-related: recreate connection and retry + if (cause instanceof SQLException sqlEx && isConnectionException(sqlEx) && attempt < MAX_RETRIES) { + connection = createConnectionWithRetries(); + sleepUnchecked(computeBackoffDelay(attempt)); + continue; + } + + // Teradata transient concurrency error 3598: backoff & retry + if (isTeradataError3598(e) && attempt < MAX_RETRIES) { + long delay = computeBackoffDelay(attempt); + sleepUnchecked(delay); + continue; + } + throw e; + } + } + } + + private Connection createConnectionWithRetries() + { + int attempt = 0; + while (true) { + try { + return createConnection(); + } + catch (RuntimeException e) { + attempt++; + if (attempt >= MAX_RETRIES) { + throw new RuntimeException("Failed to create database connection after retries", e); + } + long delay = computeBackoffDelay(attempt); + sleepUnchecked(delay); + } + } + } + + private Connection createConnection() + { + try { + Class.forName("com.teradata.jdbc.TeraDriver"); + Properties props = buildConnectionProperties(config.getAuthConfig()); + return DriverManager.getConnection(config.getJdbcUrl(), props); + } + catch (SQLException | ClassNotFoundException e) { + throw new RuntimeException("Failed to create database connection", e); + } + } + + private boolean isTeradataError3598(Throwable t) + { + if (t == null) { + return false; + } + Throwable root = t; + while (root.getCause() != null && !(root instanceof SQLException)) { + root = root.getCause(); + } + if (root instanceof SQLException sqlEx) { + try { + if (sqlEx.getErrorCode() == 3598) { + return true; + } + } + catch (Exception ignored) { + } + } + return false; + } + + private boolean isConnectionException(SQLException e) + { + if (e == null) { + return false; + } + try { + int code = e.getErrorCode(); + if (code == 1095 || code == 804) { // 1095 == closed connection, 804 socket communication failure + return true; + } + } + catch (Exception ignored) { + } + + try { + return connection == null || connection.isClosed(); + } + catch (SQLException ignored) { + } + + return false; + } + + private static Properties buildConnectionProperties(AuthenticationConfig auth) + { + Properties props = new Properties(); + props.setProperty("logmech", "TD2"); + props.setProperty("username", auth.userName()); + props.setProperty("password", auth.password()); + return props; + } + + private static long computeBackoffDelay(int attempt) + { + // Calculates how long to wait before retrying an operation that failed + long base = BASE_RETRY_DELAY_MS * (1L << Math.max(0, attempt - 1)); + long jitter = (long) (RANDOM.nextDouble() * BASE_RETRY_DELAY_MS); + long delay = Math.min(base + jitter, MAX_RETRY_DELAY_MS); + return Math.max(delay, BASE_RETRY_DELAY_MS); + } + + private static void sleepUnchecked(long millis) + { + try { + Thread.sleep(millis); + } + catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted during retry wait", ie); + } + } +} diff --git a/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/ClearScapeEnvironmentUtils.java b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/ClearScapeEnvironmentUtils.java new file mode 100644 index 000000000000..b6174bd31309 --- /dev/null +++ b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/ClearScapeEnvironmentUtils.java @@ -0,0 +1,40 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.teradata.integration.clearscape; + +import java.util.concurrent.ThreadLocalRandom; + +import static java.util.Locale.ENGLISH; + +public final class ClearScapeEnvironmentUtils +{ + private static final int MAX_ENV_NAME_LENGTH = 20; + + private ClearScapeEnvironmentUtils() {} + + public static String generateUniqueEnvName(Class testClass) + { + String prefix = testClass.getSimpleName().toLowerCase(ENGLISH); + String suffix = Long.toString(ThreadLocalRandom.current().nextLong(Long.MAX_VALUE), 36); + int suffixLength = 6; + if (suffix.length() > suffixLength) { + suffix = suffix.substring(0, suffixLength); + } + int prefixLength = MAX_ENV_NAME_LENGTH - suffixLength - 1; + if (prefix.length() > prefixLength) { + prefix = prefix.substring(0, prefixLength); + } + return prefix + "-" + suffix; + } +} diff --git a/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/ClearScapeManager.java b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/ClearScapeManager.java new file mode 100644 index 000000000000..422ab8e17600 --- /dev/null +++ b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/ClearScapeManager.java @@ -0,0 +1,173 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.teradata.integration.clearscape; + +import io.airlift.log.Logger; +import io.trino.plugin.teradata.integration.TeradataTestConstants; + +import java.net.URISyntaxException; +import java.util.regex.Pattern; + +import static java.util.Objects.requireNonNull; + +public class ClearScapeManager +{ + private static final Logger log = Logger.get(ClearScapeManager.class); + private static final Pattern ALLOWED_URL_PATTERN = Pattern.compile("^(https?://)(www\\.)?api.clearscape.teradata\\.com.*"); + private final Model model; + + public ClearScapeManager(Model model) + { + requireNonNull(model, "model should not be null"); + this.model = model; + } + + public void setup() + { + createAndStartClearScapeInstance(); + } + + public void stop() + { + stopClearScapeInstance(); + } + + public EnvironmentResponse.State status() + { + return getClearScapeInstanceStatus(); + } + + public void teardown() + { + shutdownAndDestroyClearScapeInstance(); + } + + private EnvironmentResponse.State getClearScapeInstanceStatus() + { + try { + TeradataHttpClient teradataHttpClient = getTeradataHttpClient(); + + String token = model.getToken(); + String name = model.getEnvName(); + EnvironmentResponse response; + try { + response = teradataHttpClient.fetchEnvironment(new GetEnvironmentRequest(name), token); + } + catch (ClearScapeServiceException be) { + return EnvironmentResponse.State.TERMINATED; + } + + if (response != null) { + return response.state(); + } + return EnvironmentResponse.State.TERMINATED; + } + catch (Exception e) { + throw new RuntimeException("Failed to get status of ClearScape instance", e); + } + } + + private void createAndStartClearScapeInstance() + { + try { + TeradataHttpClient teradataHttpClient = getTeradataHttpClient(); + + String token = model.getToken(); + String name = model.getEnvName(); + EnvironmentResponse response = null; + try { + response = teradataHttpClient.fetchEnvironment(new GetEnvironmentRequest(name), token); + } + catch (ClearScapeServiceException be) { + log.info("Environment %s is not available. %s", name, be.getMessage()); + } + + if (response == null || response.ip() == null) { + CreateEnvironmentRequest request = new CreateEnvironmentRequest( + name, + model.getRegion(), + model.getPassword()); + response = teradataHttpClient.createEnvironment(request, token).get(); + } + else if (response.state() == EnvironmentResponse.State.STOPPED) { + EnvironmentRequest request = new EnvironmentRequest(name, new OperationRequest("start")); + teradataHttpClient.startEnvironment(request, token); + } + if (response != null) { + model.setHostName(response.ip()); + } + } + catch (Exception e) { + throw new RuntimeException("Failed to create and start ClearScape instance", e); + } + } + + private void stopClearScapeInstance() + { + try { + TeradataHttpClient teradataHttpClient = getTeradataHttpClient(); + String token = model.getToken(); + String name = model.getEnvName(); + + EnvironmentResponse response = null; + try { + response = teradataHttpClient.fetchEnvironment(new GetEnvironmentRequest(name), token); + } + catch (ClearScapeServiceException be) { + log.info("Environment %s is not available. %s", name, be.getMessage()); + } + if (response != null && + response.ip() != null && + response.state() == EnvironmentResponse.State.RUNNING) { + EnvironmentRequest request = new EnvironmentRequest(name, new OperationRequest("stop")); + teradataHttpClient.stopEnvironment(request, token); + } + } + catch (Exception e) { + throw new RuntimeException("Failed to stop ClearScape instance", e); + } + } + + private void shutdownAndDestroyClearScapeInstance() + { + try { + TeradataHttpClient teradataHttpClient = getTeradataHttpClient(); + String token = model.getToken(); + DeleteEnvironmentRequest request = new DeleteEnvironmentRequest(model.getEnvName()); + teradataHttpClient.deleteEnvironment(request, token).get(); + } + catch (ClearScapeServiceException be) { + log.info("Environment %s is not available. Error - %s", + model.getEnvName(), be.getMessage()); + } + catch (Exception e) { + throw new RuntimeException("Failed to shutdown and destroy ClearScape instance", e); + } + } + + private TeradataHttpClient getTeradataHttpClient() + throws URISyntaxException + { + String envUrl = TeradataTestConstants.CLEARSCAPE_URL; + if (isValidUrl(envUrl)) { + return new TeradataHttpClient(envUrl); + } + throw new URISyntaxException(envUrl, "Provide valid environment URL"); + } + + private static boolean isValidUrl(String url) + { + return ALLOWED_URL_PATTERN.matcher(url).matches(); + } +} diff --git a/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/ClearScapeServiceException.java b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/ClearScapeServiceException.java new file mode 100644 index 000000000000..1d4ee10dfd83 --- /dev/null +++ b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/ClearScapeServiceException.java @@ -0,0 +1,36 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.teradata.integration.clearscape; + +import static java.util.Objects.requireNonNull; + +public class ClearScapeServiceException + extends RuntimeException +{ + public ClearScapeServiceException(int statusCode, String body) + { + super(buildMessage(statusCode, requireNonNull(body, "body should not be null"))); + } + + private static String buildMessage(int statusCode, String body) + { + if (statusCode >= 400 && statusCode <= 499) { + return "Client error - " + statusCode + body; + } + if (statusCode >= 500 && statusCode <= 599) { + return "Server error - " + statusCode + body; + } + return "Unexpected error - " + statusCode + body; + } +} diff --git a/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/ClearScapeSetup.java b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/ClearScapeSetup.java new file mode 100644 index 000000000000..0a526bf157ee --- /dev/null +++ b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/ClearScapeSetup.java @@ -0,0 +1,91 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.teradata.integration.clearscape; + +import io.trino.plugin.teradata.integration.TeradataTestConstants; + +import static java.util.Objects.requireNonNull; + +public class ClearScapeSetup +{ + private final String token; + private final String password; + private final String envName; + private final String region; + private final boolean destroyEnv; + private ClearScapeManager manager; + + public ClearScapeSetup( + String token, + String password, + String envName, + boolean destroyEnv, + String region) + { + requireNonNull(token, "token is null"); + requireNonNull(password, "password is null"); + requireNonNull(envName, "envName is null"); + requireNonNull(region, "region is null"); + this.token = token; + this.password = password; + this.envName = envName; + this.region = region; + this.destroyEnv = destroyEnv; + } + + public Model initialize() + { + try { + Model model = createModel(); + manager = new ClearScapeManager(model); + manager.setup(); + return model; + } + catch (Exception e) { + throw new RuntimeException("Failed to initialize ClearScape environment: " + envName, e); + } + } + + private Model createModel() + { + Model model = new Model(); + model.setEnvName(envName); + model.setUserName(TeradataTestConstants.CLEARSCAPE_USERNAME); + model.setPassword(password); + model.setDatabaseName(TeradataTestConstants.CLEARSCAPE_USERNAME); + model.setToken(token); + model.setRegion(region); + return model; + } + + public void cleanup() + { + if (manager == null) { + return; + } + if (destroyEnv) { + manager.teardown(); + return; + } + manager.stop(); + } + + public EnvironmentResponse.State status() + { + if (manager == null) { + throw new IllegalStateException("ClearScape manager is not initialized"); + } + return manager.status(); + } +} diff --git a/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/CreateEnvironmentRequest.java b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/CreateEnvironmentRequest.java new file mode 100644 index 000000000000..8ac35a095493 --- /dev/null +++ b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/CreateEnvironmentRequest.java @@ -0,0 +1,29 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.teradata.integration.clearscape; + +import static java.util.Objects.requireNonNull; + +public record CreateEnvironmentRequest( + String name, + String region, + String password) +{ + public CreateEnvironmentRequest + { + requireNonNull(name, "name should not be null"); + requireNonNull(region, "region should not be null"); + requireNonNull(password, "password should not be null"); + } +} diff --git a/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/DeleteEnvironmentRequest.java b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/DeleteEnvironmentRequest.java new file mode 100644 index 000000000000..38320c87e226 --- /dev/null +++ b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/DeleteEnvironmentRequest.java @@ -0,0 +1,25 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.teradata.integration.clearscape; + +import static java.util.Objects.requireNonNull; + +public record DeleteEnvironmentRequest( + String name) +{ + public DeleteEnvironmentRequest + { + requireNonNull(name, "name should not be null"); + } +} diff --git a/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/EnvironmentRequest.java b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/EnvironmentRequest.java new file mode 100644 index 000000000000..a83993f4c24a --- /dev/null +++ b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/EnvironmentRequest.java @@ -0,0 +1,26 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.teradata.integration.clearscape; + +import static java.util.Objects.requireNonNull; + +public record EnvironmentRequest( + String name, + OperationRequest request) +{ + public EnvironmentRequest + { + requireNonNull(name, "name must not be null"); + } +} diff --git a/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/EnvironmentResponse.java b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/EnvironmentResponse.java new file mode 100644 index 000000000000..0ed7109f86e9 --- /dev/null +++ b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/EnvironmentResponse.java @@ -0,0 +1,38 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.teradata.integration.clearscape; + +import static java.util.Locale.ENGLISH; +import static java.util.Objects.requireNonNull; + +public record EnvironmentResponse( + State state, + String region, + String ip) +{ + public EnvironmentResponse + { + requireNonNull(state, "state must not be null"); + requireNonNull(region, "region must not be null"); + region = region.toUpperCase(ENGLISH); + } + + public enum State + { + RUNNING, + STOPPED, + TERMINATED, + STOPPING + } +} diff --git a/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/GetEnvironmentRequest.java b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/GetEnvironmentRequest.java new file mode 100644 index 000000000000..9f2ba3971ab2 --- /dev/null +++ b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/GetEnvironmentRequest.java @@ -0,0 +1,25 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.teradata.integration.clearscape; + +import static java.util.Objects.requireNonNull; + +public record GetEnvironmentRequest( + String name) +{ + public GetEnvironmentRequest + { + requireNonNull(name, "name should not be null"); + } +} diff --git a/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/Model.java b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/Model.java new file mode 100644 index 000000000000..605c60c3a191 --- /dev/null +++ b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/Model.java @@ -0,0 +1,85 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.teradata.integration.clearscape; + +public class Model +{ + String envName; + String hostName; + String userName; + String password; + String databaseName; + String token; + String region; + + public String getEnvName() + { + return envName; + } + + public void setEnvName(String envName) + { + this.envName = envName; + } + + public String getHostName() + { + return hostName; + } + + public void setHostName(String hostName) + { + this.hostName = hostName; + } + + public void setUserName(String userName) + { + this.userName = userName; + } + + public String getPassword() + { + return password; + } + + public void setPassword(String password) + { + this.password = password; + } + + public void setDatabaseName(String databaseName) + { + this.databaseName = databaseName; + } + + public String getToken() + { + return token; + } + + public void setToken(String token) + { + this.token = token; + } + + public String getRegion() + { + return region; + } + + public void setRegion(String region) + { + this.region = region; + } +} diff --git a/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/OperationRequest.java b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/OperationRequest.java new file mode 100644 index 000000000000..88e1e80d1530 --- /dev/null +++ b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/OperationRequest.java @@ -0,0 +1,25 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.teradata.integration.clearscape; + +import static java.util.Objects.requireNonNull; + +public record OperationRequest( + String operation) +{ + public OperationRequest + { + requireNonNull(operation, "operation should not be null"); + } +} diff --git a/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/TeradataHttpClient.java b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/TeradataHttpClient.java new file mode 100644 index 000000000000..29ed4851c3cf --- /dev/null +++ b/plugin/trino-teradata/src/test/java/io/trino/plugin/teradata/integration/clearscape/TeradataHttpClient.java @@ -0,0 +1,147 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.teradata.integration.clearscape; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.MapperFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.json.JsonMapper; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.net.URI; +import java.net.http.HttpClient; +import java.net.http.HttpRequest; +import java.net.http.HttpResponse; +import java.util.concurrent.CompletableFuture; + +import static com.google.common.net.HttpHeaders.AUTHORIZATION; +import static com.google.common.net.HttpHeaders.CONTENT_TYPE; +import static java.util.Objects.requireNonNull; + +public class TeradataHttpClient +{ + private static final String APPLICATION_JSON = "application/json"; + private static final String BEARER = "Bearer "; + + private final String baseUrl; + private final HttpClient httpClient; + private final ObjectMapper objectMapper; + + public TeradataHttpClient(String baseUrl) + { + requireNonNull(baseUrl, "baseUrl should not be null"); + this.baseUrl = baseUrl; + httpClient = HttpClient.newBuilder().version(HttpClient.Version.HTTP_1_1).build(); + objectMapper = JsonMapper.builder() + .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) + .configure(MapperFeature.ALLOW_FINAL_FIELDS_AS_MUTATORS, false) + .build(); + } + + public CompletableFuture createEnvironment(CreateEnvironmentRequest createEnvironmentRequest, String token) + { + String requestBody = handleCheckedException(() -> objectMapper.writeValueAsString(createEnvironmentRequest)); + HttpRequest httpRequest = HttpRequest.newBuilder(URI.create(baseUrl.concat("/environments"))) + .headers( + AUTHORIZATION, BEARER + token, + CONTENT_TYPE, APPLICATION_JSON) + .POST(HttpRequest.BodyPublishers.ofString(requestBody)) + .build(); + return httpClient.sendAsync(httpRequest, HttpResponse.BodyHandlers.ofString()) + .thenApply(httpResponse -> handleHttpResponse(httpResponse, new TypeReference<>() {})); + } + + public EnvironmentResponse fetchEnvironment(GetEnvironmentRequest getEnvironmentRequest, String token) + { + HttpRequest httpRequest = HttpRequest.newBuilder(URI.create(baseUrl + .concat("/environments/") + .concat(getEnvironmentRequest.name()))) + .headers(AUTHORIZATION, BEARER + token) + .GET() + .build(); + HttpResponse httpResponse = handleCheckedException(() -> httpClient.send(httpRequest, HttpResponse.BodyHandlers.ofString())); + return handleHttpResponse(httpResponse, new TypeReference<>() {}); + } + + public CompletableFuture deleteEnvironment(DeleteEnvironmentRequest deleteEnvironmentRequest, String token) + { + HttpRequest httpRequest = HttpRequest.newBuilder(URI.create(baseUrl + "/environments/" + deleteEnvironmentRequest.name())) + .headers(AUTHORIZATION, BEARER + token) + .DELETE() + .build(); + + httpClient.sendAsync(httpRequest, HttpResponse.BodyHandlers.ofString()); + return CompletableFuture.completedFuture(null); + } + + public void startEnvironment(EnvironmentRequest environmentRequest, String token) + { + String requestBody = handleCheckedException(() -> objectMapper.writeValueAsString(environmentRequest.request())); + getVoidCompletableFuture(environmentRequest.name(), token, requestBody); + } + + public void stopEnvironment(EnvironmentRequest environmentRequest, String token) + { + String requestBody = handleCheckedException(() -> objectMapper.writeValueAsString(environmentRequest.request())); + getVoidCompletableFuture(environmentRequest.name(), token, requestBody); + } + + private void getVoidCompletableFuture(String name, String token, String jsonPayLoadString) + { + HttpRequest.BodyPublisher publisher = HttpRequest.BodyPublishers.ofString(jsonPayLoadString); + HttpRequest httpRequest = HttpRequest.newBuilder(URI.create(baseUrl + "/environments/" + name)) + .headers(AUTHORIZATION, BEARER + token, CONTENT_TYPE, APPLICATION_JSON) + .method("PATCH", publisher) + .build(); + + httpClient.sendAsync(httpRequest, HttpResponse.BodyHandlers.ofString()); + } + + private T handleHttpResponse(HttpResponse httpResponse, TypeReference typeReference) + { + String body = httpResponse.body(); + if (httpResponse.statusCode() >= 200 && httpResponse.statusCode() <= 299) { + return handleCheckedException(() -> { + if (typeReference.getType().getTypeName().equals(Void.class.getTypeName())) { + return null; + } + return objectMapper.readValue(body, typeReference); + }); + } + throw new ClearScapeServiceException(httpResponse.statusCode(), body); + } + + private static T handleCheckedException(CheckedSupplier checkedSupplier) + { + try { + return checkedSupplier.get(); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } + + @FunctionalInterface + private interface CheckedSupplier + { + T get() + throws IOException, InterruptedException; + } +} diff --git a/pom.xml b/pom.xml index 16212dacd3f9..e79453636eca 100644 --- a/pom.xml +++ b/pom.xml @@ -5,7 +5,7 @@ io.airlift airbase - 332 + 334 io.trino @@ -113,6 +113,7 @@ plugin/trino-snowflake plugin/trino-spooling-filesystem plugin/trino-sqlserver + plugin/trino-teradata plugin/trino-teradata-functions plugin/trino-thrift plugin/trino-thrift-api @@ -191,7 +192,7 @@ 7.8.0 119 1.24 - 11.17.2 + 11.18.0 1.15.4 v22.14.0 11.2.0 @@ -228,7 +229,7 @@ com.azure azure-sdk-bom - 1.3.2 + 1.3.3 pom import @@ -300,7 +301,7 @@ software.amazon.awssdk bom - 2.39.5 + 2.39.6 pom import diff --git a/testing/trino-product-tests-groups/src/main/java/io/trino/tests/product/TestGroups.java b/testing/trino-product-tests-groups/src/main/java/io/trino/tests/product/TestGroups.java index b7df57f43ba4..ba67a93328bc 100644 --- a/testing/trino-product-tests-groups/src/main/java/io/trino/tests/product/TestGroups.java +++ b/testing/trino-product-tests-groups/src/main/java/io/trino/tests/product/TestGroups.java @@ -107,6 +107,7 @@ public final class TestGroups public static final String PARQUET = "parquet"; public static final String IGNITE = "ignite"; public static final String FAULT_TOLERANT = "fault-tolerant"; + public static final String TERADATA = "teradata"; private TestGroups() {} diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeAllConnectors.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeAllConnectors.java index ee3de89bd9f0..8fae4cc78e43 100644 --- a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeAllConnectors.java +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeAllConnectors.java @@ -75,6 +75,7 @@ public void extendEnvironment(Environment.Builder builder) "singlestore", "snowflake", "sqlserver", + "teradata", "tpcds", "trino_thrift", "vertica") diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeTeradata.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeTeradata.java new file mode 100644 index 000000000000..5a729b7a54b4 --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeTeradata.java @@ -0,0 +1,43 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.tests.product.launcher.env.environment; + +import com.google.inject.Inject; +import io.trino.tests.product.launcher.docker.DockerFiles; +import io.trino.tests.product.launcher.env.Environment.Builder; +import io.trino.tests.product.launcher.env.EnvironmentProvider; +import io.trino.tests.product.launcher.env.common.StandardMultinode; +import io.trino.tests.product.launcher.env.common.TestsEnvironment; + +import static org.testcontainers.utility.MountableFile.forHostPath; + +@TestsEnvironment +public class EnvMultinodeTeradata + extends EnvironmentProvider +{ + private final DockerFiles.ResourceProvider configDir; + + @Inject + public EnvMultinodeTeradata(StandardMultinode standardMultinode, DockerFiles dockerFiles) + { + super(standardMultinode); + this.configDir = dockerFiles.getDockerFilesHostDirectory("conf/environment/multinode-teradata"); + } + + @Override + public void extendEnvironment(Builder builder) + { + builder.addConnector("teradata", forHostPath(configDir.getPath("teradata.properties"))); + } +} diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/SuiteTeradata.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/SuiteTeradata.java new file mode 100644 index 000000000000..36632e2d1930 --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/SuiteTeradata.java @@ -0,0 +1,37 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.tests.product.launcher.suite.suites; + +import com.google.common.collect.ImmutableList; +import io.trino.tests.product.launcher.env.EnvironmentConfig; +import io.trino.tests.product.launcher.env.environment.EnvMultinodeTeradata; +import io.trino.tests.product.launcher.suite.Suite; +import io.trino.tests.product.launcher.suite.SuiteTestRun; + +import java.util.List; + +import static io.trino.tests.product.launcher.suite.SuiteTestRun.testOnEnvironment; + +public class SuiteTeradata + extends Suite +{ + @Override + public List getTestRuns(EnvironmentConfig config) + { + return ImmutableList.of( + testOnEnvironment(EnvMultinodeTeradata.class) + .withGroups("configured_features", "teradata") + .build()); + } +} diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-all/teradata.properties b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-all/teradata.properties new file mode 100644 index 000000000000..ede4d6db0d63 --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-all/teradata.properties @@ -0,0 +1,4 @@ +connector.name=teradata +connection-url=jdbc:teradata://host.invalid/ +connection-user=root +connection-password=secret diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-teradata/teradata.properties b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-teradata/teradata.properties new file mode 100644 index 000000000000..65d022bccb23 --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-teradata/teradata.properties @@ -0,0 +1,4 @@ +connector.name=teradata +connection-url=jdbc:teradata://${ENV:TERADATA_HOSTNAME}/ +connection-user=${ENV:TERADATA_USERNAME} +connection-password=${ENV:TERADATA_PASSWORD} diff --git a/testing/trino-product-tests/pom.xml b/testing/trino-product-tests/pom.xml index 5799c1a69a14..c9ff9c4e62da 100644 --- a/testing/trino-product-tests/pom.xml +++ b/testing/trino-product-tests/pom.xml @@ -264,6 +264,13 @@ runtime + + com.teradata.jdbc + terajdbc + 20.00.00.51 + runtime + + io.confluent kafka-protobuf-types diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/teradata/TestTeradata.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/teradata/TestTeradata.java new file mode 100644 index 000000000000..bc0316a21155 --- /dev/null +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/teradata/TestTeradata.java @@ -0,0 +1,46 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.tests.product.teradata; + +import io.trino.tempto.ProductTest; +import io.trino.tempto.query.QueryResult; +import org.testng.annotations.Test; + +import static io.trino.tempto.assertions.QueryAssert.Row.row; +import static io.trino.testing.TestingNames.randomNameSuffix; +import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; +import static io.trino.tests.product.TestGroups.TERADATA; +import static io.trino.tests.product.utils.QueryExecutors.onTrino; +import static org.assertj.core.api.Assertions.assertThat; + +public class TestTeradata + extends ProductTest +{ + @Test(groups = {TERADATA, PROFILE_SPECIFIC_TESTS}) + public void testCreateTableAsSelect() + { + String databaseName = "teradata.test_" + randomNameSuffix(); + String tableName = databaseName + ".nation_" + randomNameSuffix(); + onTrino().executeQuery("CREATE SCHEMA " + databaseName); + QueryResult result = onTrino().executeQuery("CREATE TABLE " + tableName + " AS SELECT * FROM tpch.tiny.nation"); + try { + assertThat(result).updatedRowsCountIsEqualTo(25); + assertThat(onTrino().executeQuery("SELECT COUNT(*) FROM " + tableName)).containsOnly(row(25)); + } + finally { + onTrino().executeQuery("DROP TABLE " + tableName); + onTrino().executeQuery("DROP SCHEMA " + databaseName); + } + } +} diff --git a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java index e1afcf56b66d..7378b28de8fc 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java @@ -1532,6 +1532,8 @@ public void testMaterializedViewGracePeriod() assertUpdate("CREATE MATERIALIZED VIEW " + viewName + " " + "GRACE PERIOD INTERVAL '1' HOUR " + "AS SELECT DISTINCT regionkey, format('%s', name) name FROM " + table.getName()); + assertThat((String) computeScalar("SHOW CREATE MATERIALIZED VIEW " + viewName)) + .matches("(?sm).*^GRACE PERIOD INTERVAL '0 1:00:00' DAY TO SECOND$.*"); String initialResults = "SELECT DISTINCT regionkey, CAST(name AS varchar) FROM region";