diff --git a/core/trino-server/src/main/provisio/trino.xml b/core/trino-server/src/main/provisio/trino.xml index fb37ff50fa76..94592459444b 100644 --- a/core/trino-server/src/main/provisio/trino.xml +++ b/core/trino-server/src/main/provisio/trino.xml @@ -287,6 +287,12 @@ + + + + + + diff --git a/plugin/trino-sybase/pom.xml b/plugin/trino-sybase/pom.xml new file mode 100644 index 000000000000..5663d335329d --- /dev/null +++ b/plugin/trino-sybase/pom.xml @@ -0,0 +1,156 @@ + + + 4.0.0 + + io.trino + trino-root + 473-SNAPSHOT + ../../pom.xml + + + trino-sybase + trino-plugin + Trino - Sybase Connector + + + + + com.google.guava + guava + + + + com.google.inject + guice + + + + io.airlift + configuration + + + io.trino + trino-base-jdbc + + + + io.trino + trino-plugin-toolkit + + + + javax.validation + validation-api + 1.1.0.Final + + + + jdbc.sybase + jconn4 + 16.0 + + + + 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 + + + + io.airlift + configuration-testing + test + + + + io.airlift + testing + test + + + + + io.trino + trino-main + test + + + + io.trino + trino-tests + test + + + + io.trino + trino-tpch + test + + + + io.trino.tpch + tpch + test + + + + org.assertj + assertj-core + test + + + + org.testng + testng + test + + + + + + + false + + JumpMind + https://maven.jumpmind.com/repo/ + + + + diff --git a/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/SybaseClient.java b/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/SybaseClient.java new file mode 100755 index 000000000000..60bc4e6e756c --- /dev/null +++ b/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/SybaseClient.java @@ -0,0 +1,391 @@ +/* + * 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.sybase; + +import com.google.inject.Inject; +import io.trino.plugin.base.mapping.IdentifierMapping; +import io.trino.plugin.jdbc.BaseJdbcClient; +import io.trino.plugin.jdbc.BaseJdbcConfig; +import io.trino.plugin.jdbc.ColumnMapping; +import io.trino.plugin.jdbc.ConnectionFactory; +import io.trino.plugin.jdbc.JdbcSplit; +import io.trino.plugin.jdbc.JdbcTableHandle; +import io.trino.plugin.jdbc.JdbcTypeHandle; +import io.trino.plugin.jdbc.LongReadFunction; +import io.trino.plugin.jdbc.ObjectReadFunction; +import io.trino.plugin.jdbc.ObjectWriteFunction; +import io.trino.plugin.jdbc.QueryBuilder; +import io.trino.plugin.jdbc.WriteMapping; +import io.trino.plugin.jdbc.logging.RemoteQueryModifier; +import io.trino.spi.TrinoException; +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.LongTimestamp; +import io.trino.spi.type.TimestampType; +import io.trino.spi.type.Type; +import io.trino.spi.type.VarcharType; + +import java.sql.Connection; +import java.sql.SQLException; +import java.sql.Timestamp; +import java.sql.Types; +import java.util.List; +import java.util.Optional; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Verify.verify; +import static io.trino.plugin.jdbc.JdbcErrorCode.JDBC_ERROR; +import static io.trino.plugin.jdbc.StandardColumnMappings.bigintColumnMapping; +import static io.trino.plugin.jdbc.StandardColumnMappings.bigintWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.booleanColumnMapping; +import static io.trino.plugin.jdbc.StandardColumnMappings.booleanWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.charWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.dateColumnMappingUsingLocalDate; +import static io.trino.plugin.jdbc.StandardColumnMappings.dateWriteFunctionUsingSqlDate; +import static io.trino.plugin.jdbc.StandardColumnMappings.decimalColumnMapping; +import static io.trino.plugin.jdbc.StandardColumnMappings.defaultCharColumnMapping; +import static io.trino.plugin.jdbc.StandardColumnMappings.defaultVarcharColumnMapping; +import static io.trino.plugin.jdbc.StandardColumnMappings.doubleColumnMapping; +import static io.trino.plugin.jdbc.StandardColumnMappings.doubleWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.fromLongTrinoTimestamp; +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.realColumnMapping; +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.timeColumnMappingUsingSqlTime; +import static io.trino.plugin.jdbc.StandardColumnMappings.timestampWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.timestampWriteFunctionUsingSqlTimestamp; +import static io.trino.plugin.jdbc.StandardColumnMappings.tinyintColumnMapping; +import static io.trino.plugin.jdbc.StandardColumnMappings.tinyintWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.toLongTrinoTimestamp; +import static io.trino.plugin.jdbc.StandardColumnMappings.varbinaryColumnMapping; +import static io.trino.plugin.jdbc.StandardColumnMappings.varbinaryWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.varcharColumnMapping; +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.BooleanType.BOOLEAN; +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.TimestampType.TIMESTAMP_MICROS; +import static io.trino.spi.type.TinyintType.TINYINT; +import static io.trino.spi.type.VarbinaryType.VARBINARY; +import static io.trino.spi.type.VarcharType.createUnboundedVarcharType; +import static java.lang.Math.max; +import static java.lang.String.format; +import static java.time.ZoneOffset.UTC; +import static java.util.Locale.ENGLISH; +import static java.util.stream.Collectors.joining; + +public class SybaseClient + extends BaseJdbcClient +{ + private final int varcharMaxLength; + private static final int SYBASE_MAX_SUPPORTED_TIMESTAMP_PRECISION = 6; + private static final int MAX_LOCAL_DATE_TIME_PRECISION = 6; + private static final String VARCHAR_FORMAT = "VARCHAR(%d)"; + + @Inject + public SybaseClient( + BaseJdbcConfig config, + SybaseConfig sybaseConfig, + ConnectionFactory connectionFactory, + QueryBuilder queryBuilder, + IdentifierMapping identifierMapping, + RemoteQueryModifier remoteQueryModifier) + throws SQLException + { + super("\"", connectionFactory, queryBuilder, config.getJdbcTypesMappedToVarchar(), identifierMapping, remoteQueryModifier, true); + this.varcharMaxLength = sybaseConfig.getVarcharMaxLength(); + + // http://stackoverflow.com/questions/16910791/getting-error-code-4220-with-null-sql-state + System.setProperty("sybase.jcc.charsetDecoderEncoder", "3"); + } + + @Override + public Connection getConnection(ConnectorSession session, JdbcSplit split, JdbcTableHandle tableHandle) + throws SQLException + { + Connection connection = super.getConnection(session, split, tableHandle); + try { + // TRANSACTION_READ_UNCOMMITTED = Uncommitted read + // http://www.ibm.com/developerworks/data/library/techarticle/dm-0509schuetz/ + connection.setTransactionIsolation(Connection.TRANSACTION_READ_UNCOMMITTED); + } + catch (SQLException e) { + connection.close(); + throw e; + } + return connection; + } + + @Override + public Optional toColumnMapping(ConnectorSession session, Connection connection, JdbcTypeHandle typeHandle) + { + Optional mapping = getForcedMappingToVarchar(typeHandle); + if (mapping.isPresent()) { + return mapping; + } + + switch (typeHandle.jdbcType()) { + case Types.BIT: + case Types.BOOLEAN: + return Optional.of(booleanColumnMapping()); + + 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: + return Optional.of(realColumnMapping()); + + case Types.FLOAT: + case Types.DOUBLE: + return Optional.of(doubleColumnMapping()); + + case Types.NUMERIC: + case Types.DECIMAL: + int decimalDigits = typeHandle.requiredDecimalDigits(); + int precision = typeHandle.requiredColumnSize() + max(-decimalDigits, 0); // Map decimal(p, -s) (negative scale) to decimal(p+s, 0). + if (precision > Decimals.MAX_PRECISION) { + break; + } + return Optional.of(decimalColumnMapping(createDecimalType(precision, max(decimalDigits, 0)))); + + case Types.CHAR: + case Types.NCHAR: + return Optional.of(defaultCharColumnMapping(typeHandle.requiredColumnSize(), false)); + + case Types.VARCHAR: + int columnSize = typeHandle.requiredColumnSize(); + if (columnSize == -1) { + return Optional.of(varcharColumnMapping(createUnboundedVarcharType(), true)); + } + return Optional.of(defaultVarcharColumnMapping(columnSize, true)); + + case Types.NVARCHAR: + case Types.LONGVARCHAR: + case Types.LONGNVARCHAR: + return Optional.of(defaultVarcharColumnMapping(typeHandle.requiredColumnSize(), false)); + + case Types.BINARY: + case Types.VARBINARY: + case Types.LONGVARBINARY: + return Optional.of(varbinaryColumnMapping()); + + case Types.DATE: + return Optional.of(dateColumnMappingUsingLocalDate()); + + case Types.TIME: + return Optional.of(timeColumnMappingUsingSqlTime()); + + case Types.TIMESTAMP: + TimestampType timestampType = typeHandle.decimalDigits() + .map(TimestampType::createTimestampType) + .orElse(TIMESTAMP_MICROS); + return Optional.of(timestampColumnMapping(timestampType)); + } + + if (getUnsupportedTypeHandling(session) == CONVERT_TO_VARCHAR) { + return mapToUnboundedVarchar(typeHandle); + } + return Optional.empty(); + } + + public static ColumnMapping timestampColumnMapping(TimestampType timestampType) + { + if (timestampType.getPrecision() <= TimestampType.MAX_SHORT_PRECISION) { + return ColumnMapping.longMapping( + timestampType, + timestampReadFunction(timestampType), + timestampWriteFunctionUsingSqlTimestamp(timestampType)); + } + checkArgument(timestampType.getPrecision() <= MAX_LOCAL_DATE_TIME_PRECISION, "Precision is out of range: %s", timestampType.getPrecision()); + return ColumnMapping.objectMapping( + timestampType, + longtimestampReadFunction(timestampType), + longTimestampWriteFunction(timestampType)); + } + + public static LongReadFunction timestampReadFunction(TimestampType timestampType) + { + checkArgument(timestampType.getPrecision() <= TimestampType.MAX_SHORT_PRECISION, "Precision is out of range: %s", timestampType.getPrecision()); + return (resultSet, columnIndex) -> resultSet.getTimestamp(columnIndex).toLocalDateTime().toEpochSecond(UTC) * 1000000; + } + + private static ObjectReadFunction longtimestampReadFunction(TimestampType timestampType) + { + checkArgument(timestampType.getPrecision() <= MAX_LOCAL_DATE_TIME_PRECISION, + "Precision is out of range: %s", timestampType.getPrecision()); + return ObjectReadFunction.of( + LongTimestamp.class, + (resultSet, columnIndex) -> toLongTrinoTimestamp(timestampType, resultSet.getTimestamp(columnIndex).toLocalDateTime())); + } + + private static ObjectWriteFunction longTimestampWriteFunction(TimestampType timestampType) + { + checkArgument(timestampType.getPrecision() > TimestampType.MAX_SHORT_PRECISION, "Precision is out of range: %s", timestampType.getPrecision()); + return ObjectWriteFunction.of( + LongTimestamp.class, + (statement, index, value) -> statement.setTimestamp(index, Timestamp.valueOf(fromLongTrinoTimestamp(value, timestampType.getPrecision())))); + } + + /** + * To map data types when generating SQL. + */ + @Override + public WriteMapping toWriteMapping(ConnectorSession session, Type type) + { + if (type instanceof VarcharType) { + VarcharType varcharType = (VarcharType) type; + String dataType; + + if (varcharType.isUnbounded()) { + dataType = format(VARCHAR_FORMAT, this.varcharMaxLength); + } + else if (varcharType.getBoundedLength() > this.varcharMaxLength) { + dataType = format("CLOB(%d)", varcharType.getBoundedLength()); + } + else if (varcharType.getBoundedLength() < this.varcharMaxLength) { + dataType = format(VARCHAR_FORMAT, varcharType.getBoundedLength()); + } + else { + dataType = format(VARCHAR_FORMAT, this.varcharMaxLength); + } + + return WriteMapping.sliceMapping(dataType, varcharWriteFunction()); + } + + if (type instanceof TimestampType timestampType) { + verify(timestampType.getPrecision() <= SYBASE_MAX_SUPPORTED_TIMESTAMP_PRECISION); + return WriteMapping.longMapping(format("timestamp(%s)", timestampType.getPrecision()), timestampWriteFunction(timestampType)); + } + + return this.legacyToWriteMapping(session, type); + } + + protected WriteMapping legacyToWriteMapping(ConnectorSession session, Type type) + { + if (type instanceof VarcharType) { + VarcharType varcharType = (VarcharType) type; + String dataType; + if (varcharType.isUnbounded()) { + dataType = "varchar"; + } + else { + dataType = "varchar(" + varcharType.getBoundedLength() + ")"; + } + return WriteMapping.sliceMapping(dataType, varcharWriteFunction()); + } + if (type instanceof CharType) { + return WriteMapping.sliceMapping("char(" + ((CharType) type).getLength() + ")", charWriteFunction()); + } + if (type instanceof DecimalType) { + DecimalType decimalType = (DecimalType) type; + String dataType = format("decimal(%s, %s)", decimalType.getPrecision(), decimalType.getScale()); + if (decimalType.isShort()) { + return WriteMapping.longMapping(dataType, shortDecimalWriteFunction(decimalType)); + } + return WriteMapping.objectMapping(dataType, longDecimalWriteFunction(decimalType)); + } + + if (type == BOOLEAN) { + return WriteMapping.booleanMapping("boolean", booleanWriteFunction()); + } + if (type == TINYINT) { + return WriteMapping.longMapping("tinyint", tinyintWriteFunction()); + } + if (type == SMALLINT) { + return WriteMapping.longMapping("smallint", smallintWriteFunction()); + } + if (type == INTEGER) { + return WriteMapping.longMapping("integer", integerWriteFunction()); + } + if (type == BIGINT) { + return WriteMapping.longMapping("bigint", bigintWriteFunction()); + } + if (type == REAL) { + return WriteMapping.longMapping("real", realWriteFunction()); + } + if (type == DOUBLE) { + return WriteMapping.doubleMapping("double precision", doubleWriteFunction()); + } + if (type == VARBINARY) { + return WriteMapping.sliceMapping("varbinary", varbinaryWriteFunction()); + } + if (type == DATE) { + WriteMapping.longMapping("date", dateWriteFunctionUsingSqlDate()); + } + throw new TrinoException(NOT_SUPPORTED, "Unsupported column type: " + type.getDisplayName()); + } + + @Override + protected void renameTable(ConnectorSession session, String catalogName, String schemaName, String tableName, SchemaTableName newTable) + { + try (Connection connection = connectionFactory.openConnection(session)) { + String newTableName = newTable.getTableName(); + if (connection.getMetaData().storesUpperCaseIdentifiers()) { + newTableName = newTableName.toUpperCase(ENGLISH); + } + // Specifies the new name for the table without a schema name + String sql = format( + "RENAME TABLE %s TO %s", + quoted(catalogName, schemaName, tableName), + quoted(newTableName)); + execute(session, connection, sql); + } + catch (SQLException e) { + throw new TrinoException(JDBC_ERROR, e); + } + } + + @Override + protected void copyTableSchema(ConnectorSession session, Connection connection, String catalogName, String schemaName, String tableName, String newTableName, List columnNames) + { + String sql = format( + "CREATE TABLE %s AS (SELECT %s FROM %s) WITH NO DATA", + quoted(catalogName, schemaName, newTableName), + columnNames.stream() + .map(this::quoted) + .collect(joining(", ")), + quoted(catalogName, schemaName, tableName)); + try { + execute(session, connection, sql); + } + catch (SQLException e) { + throw new TrinoException(JDBC_ERROR, e); + } + } +} diff --git a/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/SybaseClientModule.java b/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/SybaseClientModule.java new file mode 100755 index 000000000000..6645493fa81b --- /dev/null +++ b/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/SybaseClientModule.java @@ -0,0 +1,71 @@ +/* + * 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.sybase; + +import com.google.inject.Binder; +import com.google.inject.Provides; +import com.google.inject.Scopes; +import com.google.inject.Singleton; +import com.sybase.jdbc4.jdbc.SybDriver; +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.DecimalModule; +import io.trino.plugin.jdbc.DriverConnectionFactory; +import io.trino.plugin.jdbc.ForBaseJdbc; +import io.trino.plugin.jdbc.JdbcClient; +import io.trino.plugin.jdbc.TypeHandlingJdbcConfig; +import io.trino.plugin.jdbc.credential.CredentialProvider; + +import java.util.Properties; + +import static io.airlift.configuration.ConfigBinder.configBinder; + +public class SybaseClientModule + extends AbstractConfigurationAwareModule +{ + @Override + public void setup(Binder binder) + { + binder.bind(JdbcClient.class).annotatedWith(ForBaseJdbc.class).to(SybaseClient.class).in(Scopes.SINGLETON); + configBinder(binder).bindConfig(BaseJdbcConfig.class); + configBinder(binder).bindConfig(SybaseConfig.class); + configBinder(binder).bindConfig(TypeHandlingJdbcConfig.class); + binder.install(new DecimalModule()); + } + + @Provides + @Singleton + @ForBaseJdbc + public static ConnectionFactory getConnectionFactory(BaseJdbcConfig config, CredentialProvider credentialProvider, SybaseConfig sybaseConfig, OpenTelemetry openTelemetry) + { + Properties connectionProperties = new Properties(); + // https://www-01.ibm.com/support/knowledgecenter/ssw_ibm_i_72/rzaha/conprop.htm + // block size (a.k.a fetch size), default 32 + connectionProperties.setProperty("block size", "512"); + + // use IAM authentication when using API key + if (sybaseConfig.getApiKey() != null) { + connectionProperties.setProperty("apiKey", sybaseConfig.getApiKey()); + connectionProperties.setProperty("securityMechanism", "15"); + connectionProperties.setProperty("pluginName", "IBMIAMauth"); + } + + return DriverConnectionFactory.builder(new SybDriver(), config.getConnectionUrl(), credentialProvider) + .setConnectionProperties(connectionProperties) + .setOpenTelemetry(openTelemetry) + .build(); + } +} diff --git a/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/SybaseConfig.java b/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/SybaseConfig.java new file mode 100755 index 000000000000..3395f44b1339 --- /dev/null +++ b/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/SybaseConfig.java @@ -0,0 +1,56 @@ +/* + * 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.sybase; + +import io.airlift.configuration.Config; +import io.airlift.configuration.ConfigDescription; +import io.airlift.configuration.ConfigSecuritySensitive; + +import javax.validation.constraints.Min; + +public class SybaseConfig +{ + // this value comes from the official document + private int varcharMaxLength = 32672; + // this value is for IAM authentication + private String apiKey; + + @Min(1) + public int getVarcharMaxLength() + { + return varcharMaxLength; + } + + @Config("sybase.varchar-max-length") + @ConfigDescription("Max length of varchar type in CREATE TABLE statement") + public SybaseConfig setVarcharMaxLength(int varcharMaxLength) + { + this.varcharMaxLength = varcharMaxLength; + return this; + } + + public String getApiKey() + { + return apiKey; + } + + @Config("sybase.iam-api-key") + @ConfigSecuritySensitive + @ConfigDescription("API key for IAM authentication") + public SybaseConfig setApiKey(String apiKey) + { + this.apiKey = apiKey; + return this; + } +} diff --git a/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/SybasePlugin.java b/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/SybasePlugin.java new file mode 100755 index 000000000000..639be1dbd706 --- /dev/null +++ b/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/SybasePlugin.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.sybase; + +import io.trino.plugin.jdbc.JdbcPlugin; + +public class SybasePlugin + extends JdbcPlugin +{ + public SybasePlugin() + { + super("sybase", SybaseClientModule::new); + } +} diff --git a/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/TestSybaseConfig.java b/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/TestSybaseConfig.java new file mode 100755 index 000000000000..0c339d5df87f --- /dev/null +++ b/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/TestSybaseConfig.java @@ -0,0 +1,52 @@ +/* + * 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.sybase; + +import com.google.common.collect.ImmutableMap; +import org.testng.annotations.Test; + +import java.util.Map; + +import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; +import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; +import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; + +public class TestSybaseConfig +{ + @Test + public void testDefaults() + { + assertRecordedDefaults(recordDefaults(SybaseConfig.class) + .setVarcharMaxLength(32672) + .setApiKey(null)); + } + + @Test + public void testExplicitPropertyMappings() + { + int testVarcharLength = 30000; + String testApiKey = "xyz"; + + Map properties = ImmutableMap.builder() + .put("sybase.varchar-max-length", String.valueOf(testVarcharLength)) + .put("sybase.iam-api-key", testApiKey) + .buildOrThrow(); + + SybaseConfig expected = new SybaseConfig() + .setVarcharMaxLength(testVarcharLength) + .setApiKey(testApiKey); + + assertFullMapping(properties, expected); + } +} diff --git a/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/TestSybasePlugin.java b/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/TestSybasePlugin.java new file mode 100755 index 000000000000..6efbf3e24c1a --- /dev/null +++ b/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/TestSybasePlugin.java @@ -0,0 +1,35 @@ +/* + * 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.sybase; + +import com.google.common.collect.ImmutableMap; +import io.trino.spi.Plugin; +import io.trino.spi.connector.ConnectorFactory; +import io.trino.testing.TestingConnectorContext; +import org.testng.annotations.Test; + +import static com.google.common.collect.Iterables.getOnlyElement; +import static org.assertj.core.api.Assertions.assertThat; + +public class TestSybasePlugin +{ + @Test + public void testCreateConnector() + { + Plugin plugin = new SybasePlugin(); + ConnectorFactory factory = getOnlyElement(plugin.getConnectorFactories()); + assertThat(factory.getName()).isEqualTo("sybase"); + factory.create("test", ImmutableMap.of("connection-url", "jdbc:sybase:test"), new TestingConnectorContext()); + } +} diff --git a/pom.xml b/pom.xml index 1b1151d37bd2..677c962d6d58 100644 --- a/pom.xml +++ b/pom.xml @@ -112,6 +112,7 @@ plugin/trino-snowflake plugin/trino-spooling-filesystem plugin/trino-sqlserver + plugin/trino-sybase plugin/trino-teradata-functions plugin/trino-thrift plugin/trino-thrift-api