From c2e5b7c61cd938d8551e0758119b6b7cbbaa38f3 Mon Sep 17 00:00:00 2001 From: academy-codex Date: Thu, 8 Jul 2021 01:08:26 +0530 Subject: [PATCH 01/11] Added trino-sybase connector --- plugin/trino-sybase/pom.xml | 189 +++++++++++++ .../plugin/sybase/ImplementAvgBigint.java | 64 +++++ .../sybase/ImplementSybaseStddevPop.java | 66 +++++ .../plugin/sybase/ImplementSybaseStdev.java | 66 +++++ .../sybase/ImplementSybaseVariance.java | 66 +++++ .../sybase/ImplementSybaseVariancePop.java | 66 +++++ .../io/trino/plugin/sybase/SybaseClient.java | 252 ++++++++++++++++++ .../plugin/sybase/SybaseClientModule.java | 54 ++++ .../io/trino/plugin/sybase/SybasePlugin.java | 25 ++ .../trino/plugin/sybase/SybaseDataLoader.java | 87 ++++++ .../plugin/sybase/SybaseQueryRunner.java | 78 ++++++ .../sybase/TestSybaseConnectorTest.java | 79 ++++++ .../trino/plugin/sybase/TestSybasePlugin.java | 33 +++ .../plugin/sybase/TestingSybaseServer.java | 111 ++++++++ .../container-license-acceptance.txt | 1 + 15 files changed, 1237 insertions(+) create mode 100644 plugin/trino-sybase/pom.xml create mode 100644 plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementAvgBigint.java create mode 100644 plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseStddevPop.java create mode 100644 plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseStdev.java create mode 100644 plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseVariance.java create mode 100644 plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseVariancePop.java create mode 100644 plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/SybaseClient.java create mode 100644 plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/SybaseClientModule.java create mode 100644 plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/SybasePlugin.java create mode 100644 plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/SybaseDataLoader.java create mode 100644 plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/SybaseQueryRunner.java create mode 100644 plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/TestSybaseConnectorTest.java create mode 100644 plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/TestSybasePlugin.java create mode 100644 plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/TestingSybaseServer.java create mode 100644 plugin/trino-sybase/src/test/resources/container-license-acceptance.txt diff --git a/plugin/trino-sybase/pom.xml b/plugin/trino-sybase/pom.xml new file mode 100644 index 0000000000000..33ddf6d985f8c --- /dev/null +++ b/plugin/trino-sybase/pom.xml @@ -0,0 +1,189 @@ + + + 4.0.0 + + + io.trino + trino-root + 359-SNAPSHOT + ../../pom.xml + + + trino-sybase + Trino - Sybase Connector + trino-plugin + + + ${project.parent.basedir} + + + + + io.trino + trino-base-jdbc + + + + io.trino + trino-matching + + + + io.airlift + configuration + + + + io.airlift + log + + + + io.airlift + units + + + + + + + + + com.google.guava + guava + + + + com.google.inject + guice + + + + javax.inject + javax.inject + + + + net.sourceforge.jtds + jtds + 1.3.1 + + + + + + + + + + net.jodah + failsafe + runtime + + + + io.trino + trino-spi + provided + + + + io.airlift + slice + provided + + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + + org.openjdk.jol + jol-core + provided + + + + + io.trino + trino-base-jdbc + test-jar + test + + + + io.trino + trino-main + test + + + + io.trino + trino-main + test-jar + test + + + + io.trino + trino-testing + test + + + + io.trino + trino-tpch + test + + + + io.trino.tpch + tpch + test + + + + io.airlift + testing + test + + + + org.assertj + assertj-core + test + + + + org.jetbrains + annotations + test + + + + org.testcontainers + jdbc + test + + + + org.testcontainers + mssqlserver + test + + + + org.testcontainers + testcontainers + test + + + + org.testng + testng + test + + + diff --git a/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementAvgBigint.java b/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementAvgBigint.java new file mode 100644 index 0000000000000..5f5a7d45a92c3 --- /dev/null +++ b/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementAvgBigint.java @@ -0,0 +1,64 @@ +/* + * 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.matching.Capture; +import io.trino.matching.Captures; +import io.trino.matching.Pattern; +import io.trino.plugin.jdbc.JdbcColumnHandle; +import io.trino.plugin.jdbc.JdbcExpression; +import io.trino.plugin.jdbc.JdbcTypeHandle; +import io.trino.plugin.jdbc.expression.AggregateFunctionRule; +import io.trino.spi.connector.AggregateFunction; +import io.trino.spi.expression.Variable; + +import java.sql.Types; +import java.util.Optional; + +import static com.google.common.base.Verify.verify; +import static io.trino.matching.Capture.newCapture; +import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.basicAggregation; +import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.expressionType; +import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.functionName; +import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.singleInput; +import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.variable; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static java.lang.String.format; + +public class ImplementAvgBigint + implements AggregateFunctionRule +{ + private static final Capture INPUT = newCapture(); + + @Override + public Pattern getPattern() + { + return basicAggregation() + .with(functionName().equalTo("avg")) + .with(singleInput().matching(variable().with(expressionType().equalTo(BIGINT)).capturedAs(INPUT))); + } + + @Override + public Optional rewrite(AggregateFunction aggregateFunction, Captures captures, RewriteContext context) + { + Variable input = captures.get(INPUT); + JdbcColumnHandle columnHandle = (JdbcColumnHandle) context.getAssignment(input.getName()); + verify(aggregateFunction.getOutputType() == DOUBLE); + + return Optional.of(new JdbcExpression( + format("avg(CAST(%s AS double precision))", context.getIdentifierQuote().apply(columnHandle.getColumnName())), + new JdbcTypeHandle(Types.DOUBLE, Optional.of("double"), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty()))); + } +} diff --git a/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseStddevPop.java b/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseStddevPop.java new file mode 100644 index 0000000000000..cd9a93229d5a1 --- /dev/null +++ b/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseStddevPop.java @@ -0,0 +1,66 @@ +/* + * 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.matching.Capture; +import io.trino.matching.Captures; +import io.trino.matching.Pattern; +import io.trino.plugin.jdbc.JdbcColumnHandle; +import io.trino.plugin.jdbc.JdbcExpression; +import io.trino.plugin.jdbc.expression.AggregateFunctionRule; +import io.trino.spi.connector.AggregateFunction; +import io.trino.spi.expression.Variable; +import io.trino.spi.type.DoubleType; + +import java.util.Optional; + +import static com.google.common.base.Verify.verify; +import static io.trino.matching.Capture.newCapture; +import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.basicAggregation; +import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.expressionType; +import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.functionName; +import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.singleInput; +import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.variable; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static java.lang.String.format; + +public class ImplementSybaseStddevPop + implements AggregateFunctionRule +{ + private static final Capture INPUT = newCapture(); + + @Override + public Pattern getPattern() + { + return basicAggregation() + .with(functionName().equalTo("stddev_pop")) + .with(singleInput().matching( + variable() + .with(expressionType().matching(DoubleType.class::isInstance)) + .capturedAs(INPUT))); + } + + @Override + public Optional rewrite(AggregateFunction aggregateFunction, Captures captures, AggregateFunctionRule.RewriteContext context) + { + Variable input = captures.get(INPUT); + JdbcColumnHandle columnHandle = (JdbcColumnHandle) context.getAssignment(input.getName()); + verify(columnHandle.getColumnType().equals(DOUBLE)); + verify(aggregateFunction.getOutputType().equals(DOUBLE)); + + return Optional.of(new JdbcExpression( + format("STDEVP(%s)", context.getIdentifierQuote().apply(columnHandle.getColumnName())), + columnHandle.getJdbcTypeHandle())); + } +} diff --git a/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseStdev.java b/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseStdev.java new file mode 100644 index 0000000000000..3482cae87890c --- /dev/null +++ b/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseStdev.java @@ -0,0 +1,66 @@ +/* + * 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.matching.Capture; +import io.trino.matching.Captures; +import io.trino.matching.Pattern; +import io.trino.plugin.jdbc.JdbcColumnHandle; +import io.trino.plugin.jdbc.JdbcExpression; +import io.trino.plugin.jdbc.expression.AggregateFunctionRule; +import io.trino.spi.connector.AggregateFunction; +import io.trino.spi.expression.Variable; +import io.trino.spi.type.DoubleType; + +import java.util.Optional; + +import static com.google.common.base.Verify.verify; +import static io.trino.matching.Capture.newCapture; +import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.basicAggregation; +import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.expressionType; +import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.functionName; +import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.singleInput; +import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.variable; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static java.lang.String.format; + +public class ImplementSybaseStdev + implements AggregateFunctionRule +{ + private static final Capture INPUT = newCapture(); + + @Override + public Pattern getPattern() + { + return basicAggregation() + .with(functionName().equalTo("stddev")) + .with(singleInput().matching( + variable() + .with(expressionType().matching(DoubleType.class::isInstance)) + .capturedAs(INPUT))); + } + + @Override + public Optional rewrite(AggregateFunction aggregateFunction, Captures captures, RewriteContext context) + { + Variable input = captures.get(INPUT); + JdbcColumnHandle columnHandle = (JdbcColumnHandle) context.getAssignment(input.getName()); + verify(columnHandle.getColumnType().equals(DOUBLE)); + verify(aggregateFunction.getOutputType().equals(DOUBLE)); + + return Optional.of(new JdbcExpression( + format("STDEV(%s)", context.getIdentifierQuote().apply(columnHandle.getColumnName())), + columnHandle.getJdbcTypeHandle())); + } +} diff --git a/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseVariance.java b/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseVariance.java new file mode 100644 index 0000000000000..f771b3806c4ba --- /dev/null +++ b/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseVariance.java @@ -0,0 +1,66 @@ +/* + * 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.matching.Capture; +import io.trino.matching.Captures; +import io.trino.matching.Pattern; +import io.trino.plugin.jdbc.JdbcColumnHandle; +import io.trino.plugin.jdbc.JdbcExpression; +import io.trino.plugin.jdbc.expression.AggregateFunctionRule; +import io.trino.spi.connector.AggregateFunction; +import io.trino.spi.expression.Variable; +import io.trino.spi.type.DoubleType; + +import java.util.Optional; + +import static com.google.common.base.Verify.verify; +import static io.trino.matching.Capture.newCapture; +import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.basicAggregation; +import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.expressionType; +import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.functionName; +import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.singleInput; +import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.variable; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static java.lang.String.format; + +public class ImplementSybaseVariance + implements AggregateFunctionRule +{ + private static final Capture INPUT = newCapture(); + + @Override + public Pattern getPattern() + { + return basicAggregation() + .with(functionName().equalTo("variance")) + .with(singleInput().matching( + variable() + .with(expressionType().matching(DoubleType.class::isInstance)) + .capturedAs(INPUT))); + } + + @Override + public Optional rewrite(AggregateFunction aggregateFunction, Captures captures, RewriteContext context) + { + Variable input = captures.get(INPUT); + JdbcColumnHandle columnHandle = (JdbcColumnHandle) context.getAssignment(input.getName()); + verify(columnHandle.getColumnType().equals(DOUBLE)); + verify(aggregateFunction.getOutputType().equals(DOUBLE)); + + return Optional.of(new JdbcExpression( + format("VAR(%s)", context.getIdentifierQuote().apply(columnHandle.getColumnName())), + columnHandle.getJdbcTypeHandle())); + } +} diff --git a/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseVariancePop.java b/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseVariancePop.java new file mode 100644 index 0000000000000..462605f4ee569 --- /dev/null +++ b/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseVariancePop.java @@ -0,0 +1,66 @@ +/* + * 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.matching.Capture; +import io.trino.matching.Captures; +import io.trino.matching.Pattern; +import io.trino.plugin.jdbc.JdbcColumnHandle; +import io.trino.plugin.jdbc.JdbcExpression; +import io.trino.plugin.jdbc.expression.AggregateFunctionRule; +import io.trino.spi.connector.AggregateFunction; +import io.trino.spi.expression.Variable; +import io.trino.spi.type.DoubleType; + +import java.util.Optional; + +import static com.google.common.base.Verify.verify; +import static io.trino.matching.Capture.newCapture; +import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.basicAggregation; +import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.expressionType; +import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.functionName; +import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.singleInput; +import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.variable; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static java.lang.String.format; + +public class ImplementSybaseVariancePop + implements AggregateFunctionRule +{ + private static final Capture INPUT = newCapture(); + + @Override + public Pattern getPattern() + { + return basicAggregation() + .with(functionName().equalTo("var_pop")) + .with(singleInput().matching( + variable() + .with(expressionType().matching(DoubleType.class::isInstance)) + .capturedAs(INPUT))); + } + + @Override + public Optional rewrite(AggregateFunction aggregateFunction, Captures captures, RewriteContext context) + { + Variable input = captures.get(INPUT); + JdbcColumnHandle columnHandle = (JdbcColumnHandle) context.getAssignment(input.getName()); + verify(columnHandle.getColumnType().equals(DOUBLE)); + verify(aggregateFunction.getOutputType().equals(DOUBLE)); + + return Optional.of(new JdbcExpression( + format("VARP(%s)", context.getIdentifierQuote().apply(columnHandle.getColumnName())), + columnHandle.getJdbcTypeHandle())); + } +} 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 100644 index 0000000000000..c5308e006ee30 --- /dev/null +++ b/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/SybaseClient.java @@ -0,0 +1,252 @@ +/* + * 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.base.Joiner; +import com.google.common.collect.ImmutableSet; +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.JdbcColumnHandle; +import io.trino.plugin.jdbc.JdbcExpression; +import io.trino.plugin.jdbc.JdbcTableHandle; +import io.trino.plugin.jdbc.JdbcTypeHandle; +import io.trino.plugin.jdbc.WriteMapping; +import io.trino.plugin.jdbc.expression.AggregateFunctionRewriter; +import io.trino.plugin.jdbc.expression.AggregateFunctionRule; +import io.trino.plugin.jdbc.expression.ImplementAvgDecimal; +import io.trino.plugin.jdbc.expression.ImplementAvgFloatingPoint; +import io.trino.plugin.jdbc.expression.ImplementCount; +import io.trino.plugin.jdbc.expression.ImplementCountAll; +import io.trino.plugin.jdbc.expression.ImplementMinMax; +import io.trino.plugin.jdbc.expression.ImplementSum; +import io.trino.plugin.jdbc.mapping.IdentifierMapping; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.AggregateFunction; +import io.trino.spi.connector.ColumnHandle; +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.Type; +import io.trino.spi.type.VarcharType; + +import javax.inject.Inject; + +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Types; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.BiFunction; + +import static io.trino.plugin.jdbc.JdbcErrorCode.JDBC_ERROR; +import static io.trino.plugin.jdbc.StandardColumnMappings.booleanWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.charWriteFunction; +import static io.trino.plugin.jdbc.StandardColumnMappings.varcharWriteFunction; +import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; +import static io.trino.spi.type.BooleanType.BOOLEAN; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.joining; + +public class SybaseClient + extends BaseJdbcClient +{ + // SqlServer supports 2100 parameters in prepared statement, let's create a space for about 4 big IN predicates + public static final int SYBASE_MAX_LIST_EXPRESSIONS = 500; + + private static final Joiner DOT_JOINER = Joiner.on("."); + + private final AggregateFunctionRewriter aggregateFunctionRewriter; + + @Override + public void createSchema(ConnectorSession session, String schemaName) + { + // TODO: Add try-catch for error message parsing (test cases patch) + super.createSchema(session, schemaName); + } + + @Inject + public SybaseClient(BaseJdbcConfig config, ConnectionFactory connectionFactory, IdentifierMapping identifierMapping) + { + super(config, "\"", connectionFactory, identifierMapping); + + requireNonNull(config, "sybaseConfig is null"); + + JdbcTypeHandle bigintTypeHandle = new JdbcTypeHandle(Types.BIGINT, Optional.of("bigint"), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty()); + this.aggregateFunctionRewriter = new AggregateFunctionRewriter( + this::quoted, + ImmutableSet.builder() + .add(new ImplementCountAll(bigintTypeHandle)) + .add(new ImplementCount(bigintTypeHandle)) + .add(new ImplementMinMax()) + .add(new ImplementSum(SybaseClient::toTypeHandle)) + .add(new ImplementAvgFloatingPoint()) + .add(new ImplementAvgDecimal()) + .add(new ImplementAvgBigint()) + .add(new ImplementSybaseStdev()) + .add(new ImplementSybaseStddevPop()) + .add(new ImplementSybaseVariance()) + .add(new ImplementSybaseVariancePop()) + .build()); + } + + @Override + protected void renameTable(ConnectorSession session, String catalogName, String schemaName, String tableName, SchemaTableName newTable) + { + if (!schemaName.equals(newTable.getSchemaName())) { + throw new TrinoException(NOT_SUPPORTED, "This connector does not support renaming tables across schemas"); + } + + String sql = format( + "sp_rename %s, %s", + singleQuote(catalogName, schemaName, tableName), + singleQuote(newTable.getTableName())); + execute(session, sql); + } + + @Override + public Collection listSchemas(Connection connection) + { + try (ResultSet resultSet = connection.getMetaData().getSchemas()) { + ImmutableSet.Builder schemaNames = ImmutableSet.builder(); + while (resultSet.next()) { + String schemaName = resultSet.getString("TABLE_SCHEM"); + // skip internal schemas + if (filterSchema(schemaName)) { + schemaNames.add(schemaName); + } + } + return schemaNames.build(); + } + catch (SQLException e) { + throw new TrinoException(JDBC_ERROR, e); + } + } + + @Override + public void renameColumn(ConnectorSession session, JdbcTableHandle handle, JdbcColumnHandle jdbcColumn, String newColumnName) + { + String sql = format( + "sp_rename %s, %s, 'COLUMN'", + singleQuote(handle.getCatalogName(), handle.getSchemaName(), handle.getTableName(), jdbcColumn.getColumnName()), + singleQuote(newColumnName)); + execute(session, sql); + } + + @Override + protected void copyTableSchema(Connection connection, String catalogName, String schemaName, String tableName, String newTableName, List columnNames) + { + String sql = format( + "SELECT %s INTO %s FROM %s WHERE 0 = 1", + columnNames.stream() + .map(this::quoted) + .collect(joining(", ")), + quoted(catalogName, schemaName, newTableName), + quoted(catalogName, schemaName, tableName)); + execute(connection, sql); + } + + @Override + public boolean schemaExists(ConnectorSession session, String schema) + { + return super.schemaExists(session, schema); + } + + @Override + public Optional toColumnMapping(ConnectorSession session, Connection connection, JdbcTypeHandle typeHandle) + { + return legacyColumnMapping(session, connection, typeHandle); + } + + @Override + public WriteMapping toWriteMapping(ConnectorSession session, Type type) + { + if (type == BOOLEAN) { + return WriteMapping.booleanMapping("bit", booleanWriteFunction()); + } + + if (type instanceof VarcharType) { + VarcharType varcharType = (VarcharType) type; + String dataType; + if (varcharType.isUnbounded() || varcharType.getBoundedLength() > 4000) { + dataType = "nvarchar(max)"; + } + else { + dataType = "nvarchar(" + varcharType.getBoundedLength() + ")"; + } + return WriteMapping.sliceMapping(dataType, varcharWriteFunction()); + } + + if (type instanceof CharType) { + CharType charType = (CharType) type; + String dataType; + if (charType.getLength() > 4000) { + dataType = "nvarchar(max)"; + } + else { + dataType = "nchar(" + charType.getLength() + ")"; + } + return WriteMapping.sliceMapping(dataType, charWriteFunction()); + } + + return legacyToWriteMapping(session, type); + } + + @Override + public Optional implementAggregation(ConnectorSession session, AggregateFunction aggregate, Map assignments) + { + // TODO support complex ConnectorExpressions + return aggregateFunctionRewriter.rewrite(session, aggregate, assignments); + } + + private static Optional toTypeHandle(DecimalType decimalType) + { + return Optional.of(new JdbcTypeHandle(Types.NUMERIC, Optional.of("decimal"), Optional.of(decimalType.getPrecision()), Optional.of(decimalType.getScale()), Optional.empty(), Optional.empty())); + } + + @Override + protected Optional> limitFunction() + { + return Optional.of((sql, limit) -> format("SELECT TOP %s * FROM (%s) o", limit, sql)); + } + + @Override + public boolean isLimitGuaranteed(ConnectorSession session) + { + return true; + } + + @Override + public void abortReadConnection(Connection connection) + throws SQLException + { + connection.close(); + } + + private static String singleQuote(String... objects) + { + return singleQuote(DOT_JOINER.join(objects)); + } + + private static String singleQuote(String literal) + { + return "\'" + literal + "\'"; + } +} 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 100644 index 0000000000000..eca421e08b032 --- /dev/null +++ b/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/SybaseClientModule.java @@ -0,0 +1,54 @@ +/* + * 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.Key; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.google.inject.Scopes; +import com.google.inject.Singleton; +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.MaxDomainCompactionThreshold; +import io.trino.plugin.jdbc.credential.CredentialProvider; +import net.sourceforge.jtds.jdbc.Driver; + +import static com.google.inject.multibindings.OptionalBinder.newOptionalBinder; +import static io.airlift.configuration.ConfigBinder.configBinder; +import static io.trino.plugin.sybase.SybaseClient.SYBASE_MAX_LIST_EXPRESSIONS; + +public class SybaseClientModule + implements Module +{ + @Override + public void configure(Binder binder) + { + configBinder(binder).bindConfig(BaseJdbcConfig.class); + binder.bind(JdbcClient.class).annotatedWith(ForBaseJdbc.class).to(SybaseClient.class).in(Scopes.SINGLETON); +// bindTablePropertiesProvider(binder, SqlServerTableProperties.class); + newOptionalBinder(binder, Key.get(int.class, MaxDomainCompactionThreshold.class)).setBinding().toInstance(SYBASE_MAX_LIST_EXPRESSIONS); + } + + @Provides + @Singleton + @ForBaseJdbc + public ConnectionFactory getConnectionFactory(BaseJdbcConfig config, CredentialProvider credentialProvider) + { + return new DriverConnectionFactory(new Driver(), config, credentialProvider); + } +} 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 100644 index 0000000000000..60be0781e177b --- /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", new SybaseClientModule()); + } +} diff --git a/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/SybaseDataLoader.java b/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/SybaseDataLoader.java new file mode 100644 index 0000000000000..2ffc1f05a38ef --- /dev/null +++ b/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/SybaseDataLoader.java @@ -0,0 +1,87 @@ +/* + * 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.log.Logger; +import io.trino.Session; +import io.trino.metadata.QualifiedObjectName; +import io.trino.testing.QueryRunner; +import io.trino.tpch.TpchTable; +import org.intellij.lang.annotations.Language; + +import java.util.concurrent.atomic.AtomicReference; + +import static io.airlift.units.Duration.nanosSince; +import static java.lang.String.format; +import static java.util.Locale.ENGLISH; +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.assertj.core.api.Assertions.assertThat; + +public class SybaseDataLoader +{ + private static final Logger log = Logger.get(SybaseDataLoader.class); + + private SybaseDataLoader() {} + + public static void copyTpchTables( + QueryRunner queryRunner, + String sourceCatalog, + String sourceSchema, + Session session, + Iterable> tables) + { + log.info("Loading data from %s.%s...", sourceCatalog, sourceSchema); + long startTime = System.nanoTime(); + for (TpchTable table : tables) { + copyTable(queryRunner, sourceCatalog, sourceSchema, table.getTableName().toLowerCase(ENGLISH), session); + } + log.info("Loading from %s.%s complete in %s", sourceCatalog, sourceSchema, nanosSince(startTime).toString(SECONDS)); + } + + public static void copyTable(QueryRunner queryRunner, String sourceCatalog, String sourceSchema, String sourceTable, Session session) + { + QualifiedObjectName table = new QualifiedObjectName(sourceCatalog, sourceSchema, sourceTable); + copyTable(queryRunner, table, session); + } + + public static void copyTable(QueryRunner queryRunner, QualifiedObjectName table, Session session) + { + long start = System.nanoTime(); + log.info("Initialising tables..."); + initTable(queryRunner, table); + log.info("Running import for %s", table.getObjectName()); + @Language("SQL") String sql = format("INSERT INTO %s SELECT * FROM %s", table.getObjectName(), table); + long rows = (Long) queryRunner.execute(session, sql).getMaterializedRows().get(0).getField(0); + log.info("Imported %s rows for %s in %s", rows, table.getObjectName(), nanosSince(start).convertToMostSuccinctTimeUnit()); + + assertThat(queryRunner.execute(session, "SELECT count(*) FROM " + table).getOnlyValue()) + .as("Table is not loaded properly: %s", table) + .isEqualTo(queryRunner.execute(session, "SELECT count(*) FROM " + table.getObjectName()).getOnlyValue()); + } + + private static void initTable(QueryRunner queryRunner, QualifiedObjectName table) + { + String createDdl = "CREATE TABLE IF NOT EXISTS " + table.getObjectName() + " (%s)"; + AtomicReference schemaStr = new AtomicReference<>(); + queryRunner.execute("describe " + table).getMaterializedRows().forEach(row -> { + String colName = (String) row.getField(0); + String dataType = (String) row.getField(1); + schemaStr.set(schemaStr + " " + colName + " " + dataType + ","); + }); + + String schema = schemaStr.get().substring(5, schemaStr.get().length() - 1); + @Language("SQL") String sqlToExecute = String.format(createDdl, schema); + queryRunner.execute(sqlToExecute); + } +} diff --git a/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/SybaseQueryRunner.java b/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/SybaseQueryRunner.java new file mode 100644 index 0000000000000..abee2b978a49c --- /dev/null +++ b/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/SybaseQueryRunner.java @@ -0,0 +1,78 @@ +/* + * 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.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.trino.Session; +import io.trino.plugin.tpch.TpchPlugin; +import io.trino.testing.DistributedQueryRunner; +import io.trino.testing.QueryRunner; +import io.trino.tpch.TpchTable; + +import java.util.HashMap; +import java.util.Map; + +import static io.airlift.testing.Closeables.closeAllSuppress; +import static io.trino.plugin.sybase.SybaseDataLoader.copyTpchTables; +import static io.trino.plugin.tpch.TpchMetadata.TINY_SCHEMA_NAME; +import static io.trino.testing.TestingSession.testSessionBuilder; + +public class SybaseQueryRunner +{ + private SybaseQueryRunner() {} + + private static final String TPCH_SCHEMA = "tpch"; + + public static QueryRunner createSybaseQueryRunner(TestingSybaseServer server, TpchTable... tables) + throws Exception + { + return createSybaseQueryRunner(server, ImmutableMap.of(), ImmutableList.copyOf(tables)); + } + + public static QueryRunner createSybaseQueryRunner(TestingSybaseServer server, Map connectorProperties, Iterable> tables) + throws Exception + { + DistributedQueryRunner queryRunner = DistributedQueryRunner.builder(createSession()).build(); + try { + queryRunner.installPlugin(new TpchPlugin()); + queryRunner.createCatalog("tpch", "tpch"); + + connectorProperties = new HashMap<>(ImmutableMap.copyOf(connectorProperties)); + connectorProperties.putIfAbsent("connection-url", server.getJdbcUrl()); + connectorProperties.putIfAbsent("connection-user", server.getUsername()); + connectorProperties.putIfAbsent("connection-password", server.getPassword()); + connectorProperties.putIfAbsent("allow-drop-table", "true"); + + queryRunner.installPlugin(new SybasePlugin()); + queryRunner.createCatalog("sybase", "sybase", connectorProperties); + + copyTpchTables(queryRunner, "tpch", TINY_SCHEMA_NAME, createSession(), tables); + + return queryRunner; + } + catch (Throwable e) { + closeAllSuppress(e, queryRunner); + throw e; + } + } + + private static Session createSession() + { + return testSessionBuilder() + .setCatalog("sybase") + .setSchema("dbo") + .build(); + } +} diff --git a/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/TestSybaseConnectorTest.java b/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/TestSybaseConnectorTest.java new file mode 100644 index 0000000000000..4f6e782c499de --- /dev/null +++ b/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/TestSybaseConnectorTest.java @@ -0,0 +1,79 @@ +/* + * 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.plugin.jdbc.BaseJdbcConnectorTest; +import io.trino.testing.QueryRunner; +import io.trino.testing.TestingConnectorBehavior; +import io.trino.testing.sql.SqlExecutor; +import org.testng.annotations.AfterClass; + +import static io.trino.plugin.sybase.SybaseQueryRunner.createSybaseQueryRunner; + +public class TestSybaseConnectorTest + extends BaseJdbcConnectorTest +{ + protected TestingSybaseServer sybaseServer; + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + sybaseServer = new TestingSybaseServer(); + return createSybaseQueryRunner(sybaseServer, ImmutableMap.of(), REQUIRED_TPCH_TABLES); + } + + @AfterClass(alwaysRun = true) + public final void destroy() + { + sybaseServer.close(); + } + + @Override + protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) + { + switch (connectorBehavior) { + case SUPPORTS_CREATE_TABLE: + case SUPPORTS_JOIN_PUSHDOWN_WITH_FULL_JOIN: + case SUPPORTS_JOIN_PUSHDOWN_WITH_DISTINCT_FROM: + case SUPPORTS_JOIN_PUSHDOWN: + case SUPPORTS_COMMENT_ON_TABLE: + case SUPPORTS_COMMENT_ON_COLUMN: + case SUPPORTS_ARRAY: + case SUPPORTS_RENAME_TABLE_ACROSS_SCHEMAS: + case SUPPORTS_CREATE_TABLE_WITH_DATA: + case SUPPORTS_PREDICATE_PUSHDOWN_WITH_VARCHAR_INEQUALITY: + case SUPPORTS_INSERT: + case SUPPORTS_CREATE_SCHEMA: + case SUPPORTS_AGGREGATION_PUSHDOWN: + case SUPPORTS_TOPN_PUSHDOWN: + return false; + + default: + return super.hasBehavior(connectorBehavior); + } + } + + private void execute(String sql) + { + sybaseServer.execute(sql); + } + + @Override + protected SqlExecutor onRemoteDatabase() + { + return sybaseServer::execute; + } +} 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 100644 index 0000000000000..0f463e61c510e --- /dev/null +++ b/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/TestSybasePlugin.java @@ -0,0 +1,33 @@ +/* + * 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; + +public class TestSybasePlugin +{ + @Test + public void testCreateConnector() + { + Plugin plugin = new SybasePlugin(); + ConnectorFactory factory = getOnlyElement(plugin.getConnectorFactories()); + factory.create("test", ImmutableMap.of("connection-url", "jdbc:mariadb://test"), new TestingConnectorContext()).shutdown(); + } +} diff --git a/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/TestingSybaseServer.java b/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/TestingSybaseServer.java new file mode 100644 index 0000000000000..e9685f5878b78 --- /dev/null +++ b/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/TestingSybaseServer.java @@ -0,0 +1,111 @@ +/* + * 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 org.testcontainers.containers.JdbcDatabaseContainer; +import org.testcontainers.utility.DockerImageName; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; + +public class TestingSybaseServer + extends JdbcDatabaseContainer +{ + public static final String DEFAULT_TAG = "datagrip/sybase"; + private static final int SYBASE_PORT = 5000; + + public TestingSybaseServer() + { + this(DEFAULT_TAG); + } + + public TestingSybaseServer(String dockerImageName) + { + super(DockerImageName.parse(dockerImageName)); + start(); + } + + @Override + public String getDriverClassName() + { + return "net.sourceforge.jtds.jdbc.Driver"; + } + + @Override + public String getJdbcUrl() + { + return "jdbc:jtds:sybase://" + getContainerIpAddress() + ":" + getMappedPort(SYBASE_PORT) + "/testdb"; + } + + @Override + public String getUsername() + { + return "sa"; + } + + @Override + public String getPassword() + { + return "myPassword"; + } + + @Override + protected String getTestQueryString() + { + return "SELECT 1"; + } + + public void execute(String sql) + { + execute(sql, getUsername(), getPassword()); + } + + public void execute(String sql, String user, String password) + { + try { + Class.forName("net.sourceforge.jtds.jdbc.Driver"); + } + catch (ClassNotFoundException e) { + e.printStackTrace(); + } + try (Connection connection = DriverManager.getConnection(getJdbcUrl(), user, password); + Statement statement = connection.createStatement()) { + statement.execute(sql); + } + catch (SQLException e) { + throw new RuntimeException(e); + } + } + + public ResultSet executeQuery(String sql) + { + try { + Class.forName("net.sourceforge.jtds.jdbc.Driver"); + } + catch (ClassNotFoundException e) { + e.printStackTrace(); + } + try (Connection connection = DriverManager.getConnection(getJdbcUrl(), getUsername(), getPassword()); + Statement statement = connection.createStatement()) { + return connection.getMetaData().getColumns("testdb", null, "Persons2", null); +// return statement.executeQuery(sql); + } + catch (SQLException e) { + throw new RuntimeException(e); + } + } +} diff --git a/plugin/trino-sybase/src/test/resources/container-license-acceptance.txt b/plugin/trino-sybase/src/test/resources/container-license-acceptance.txt new file mode 100644 index 0000000000000..ab843953ef7ff --- /dev/null +++ b/plugin/trino-sybase/src/test/resources/container-license-acceptance.txt @@ -0,0 +1 @@ +microsoft/mssql-server-linux:2017-CU13 From 55ff4dea5bd2c4ef57d6a190ab83a13ad31c9f05 Mon Sep 17 00:00:00 2001 From: academy-codex Date: Thu, 8 Jul 2021 12:48:02 +0530 Subject: [PATCH 02/11] Removing aggregation pushdown impl; --- .../plugin/sybase/ImplementAvgBigint.java | 64 ------------------ .../sybase/ImplementSybaseStddevPop.java | 66 ------------------- .../plugin/sybase/ImplementSybaseStdev.java | 66 ------------------- .../sybase/ImplementSybaseVariance.java | 66 ------------------- .../sybase/ImplementSybaseVariancePop.java | 66 ------------------- .../io/trino/plugin/sybase/SybaseClient.java | 38 ----------- 6 files changed, 366 deletions(-) delete mode 100644 plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementAvgBigint.java delete mode 100644 plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseStddevPop.java delete mode 100644 plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseStdev.java delete mode 100644 plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseVariance.java delete mode 100644 plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseVariancePop.java diff --git a/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementAvgBigint.java b/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementAvgBigint.java deleted file mode 100644 index 5f5a7d45a92c3..0000000000000 --- a/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementAvgBigint.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * 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.matching.Capture; -import io.trino.matching.Captures; -import io.trino.matching.Pattern; -import io.trino.plugin.jdbc.JdbcColumnHandle; -import io.trino.plugin.jdbc.JdbcExpression; -import io.trino.plugin.jdbc.JdbcTypeHandle; -import io.trino.plugin.jdbc.expression.AggregateFunctionRule; -import io.trino.spi.connector.AggregateFunction; -import io.trino.spi.expression.Variable; - -import java.sql.Types; -import java.util.Optional; - -import static com.google.common.base.Verify.verify; -import static io.trino.matching.Capture.newCapture; -import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.basicAggregation; -import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.expressionType; -import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.functionName; -import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.singleInput; -import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.variable; -import static io.trino.spi.type.BigintType.BIGINT; -import static io.trino.spi.type.DoubleType.DOUBLE; -import static java.lang.String.format; - -public class ImplementAvgBigint - implements AggregateFunctionRule -{ - private static final Capture INPUT = newCapture(); - - @Override - public Pattern getPattern() - { - return basicAggregation() - .with(functionName().equalTo("avg")) - .with(singleInput().matching(variable().with(expressionType().equalTo(BIGINT)).capturedAs(INPUT))); - } - - @Override - public Optional rewrite(AggregateFunction aggregateFunction, Captures captures, RewriteContext context) - { - Variable input = captures.get(INPUT); - JdbcColumnHandle columnHandle = (JdbcColumnHandle) context.getAssignment(input.getName()); - verify(aggregateFunction.getOutputType() == DOUBLE); - - return Optional.of(new JdbcExpression( - format("avg(CAST(%s AS double precision))", context.getIdentifierQuote().apply(columnHandle.getColumnName())), - new JdbcTypeHandle(Types.DOUBLE, Optional.of("double"), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty()))); - } -} diff --git a/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseStddevPop.java b/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseStddevPop.java deleted file mode 100644 index cd9a93229d5a1..0000000000000 --- a/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseStddevPop.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * 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.matching.Capture; -import io.trino.matching.Captures; -import io.trino.matching.Pattern; -import io.trino.plugin.jdbc.JdbcColumnHandle; -import io.trino.plugin.jdbc.JdbcExpression; -import io.trino.plugin.jdbc.expression.AggregateFunctionRule; -import io.trino.spi.connector.AggregateFunction; -import io.trino.spi.expression.Variable; -import io.trino.spi.type.DoubleType; - -import java.util.Optional; - -import static com.google.common.base.Verify.verify; -import static io.trino.matching.Capture.newCapture; -import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.basicAggregation; -import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.expressionType; -import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.functionName; -import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.singleInput; -import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.variable; -import static io.trino.spi.type.DoubleType.DOUBLE; -import static java.lang.String.format; - -public class ImplementSybaseStddevPop - implements AggregateFunctionRule -{ - private static final Capture INPUT = newCapture(); - - @Override - public Pattern getPattern() - { - return basicAggregation() - .with(functionName().equalTo("stddev_pop")) - .with(singleInput().matching( - variable() - .with(expressionType().matching(DoubleType.class::isInstance)) - .capturedAs(INPUT))); - } - - @Override - public Optional rewrite(AggregateFunction aggregateFunction, Captures captures, AggregateFunctionRule.RewriteContext context) - { - Variable input = captures.get(INPUT); - JdbcColumnHandle columnHandle = (JdbcColumnHandle) context.getAssignment(input.getName()); - verify(columnHandle.getColumnType().equals(DOUBLE)); - verify(aggregateFunction.getOutputType().equals(DOUBLE)); - - return Optional.of(new JdbcExpression( - format("STDEVP(%s)", context.getIdentifierQuote().apply(columnHandle.getColumnName())), - columnHandle.getJdbcTypeHandle())); - } -} diff --git a/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseStdev.java b/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseStdev.java deleted file mode 100644 index 3482cae87890c..0000000000000 --- a/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseStdev.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * 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.matching.Capture; -import io.trino.matching.Captures; -import io.trino.matching.Pattern; -import io.trino.plugin.jdbc.JdbcColumnHandle; -import io.trino.plugin.jdbc.JdbcExpression; -import io.trino.plugin.jdbc.expression.AggregateFunctionRule; -import io.trino.spi.connector.AggregateFunction; -import io.trino.spi.expression.Variable; -import io.trino.spi.type.DoubleType; - -import java.util.Optional; - -import static com.google.common.base.Verify.verify; -import static io.trino.matching.Capture.newCapture; -import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.basicAggregation; -import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.expressionType; -import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.functionName; -import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.singleInput; -import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.variable; -import static io.trino.spi.type.DoubleType.DOUBLE; -import static java.lang.String.format; - -public class ImplementSybaseStdev - implements AggregateFunctionRule -{ - private static final Capture INPUT = newCapture(); - - @Override - public Pattern getPattern() - { - return basicAggregation() - .with(functionName().equalTo("stddev")) - .with(singleInput().matching( - variable() - .with(expressionType().matching(DoubleType.class::isInstance)) - .capturedAs(INPUT))); - } - - @Override - public Optional rewrite(AggregateFunction aggregateFunction, Captures captures, RewriteContext context) - { - Variable input = captures.get(INPUT); - JdbcColumnHandle columnHandle = (JdbcColumnHandle) context.getAssignment(input.getName()); - verify(columnHandle.getColumnType().equals(DOUBLE)); - verify(aggregateFunction.getOutputType().equals(DOUBLE)); - - return Optional.of(new JdbcExpression( - format("STDEV(%s)", context.getIdentifierQuote().apply(columnHandle.getColumnName())), - columnHandle.getJdbcTypeHandle())); - } -} diff --git a/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseVariance.java b/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseVariance.java deleted file mode 100644 index f771b3806c4ba..0000000000000 --- a/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseVariance.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * 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.matching.Capture; -import io.trino.matching.Captures; -import io.trino.matching.Pattern; -import io.trino.plugin.jdbc.JdbcColumnHandle; -import io.trino.plugin.jdbc.JdbcExpression; -import io.trino.plugin.jdbc.expression.AggregateFunctionRule; -import io.trino.spi.connector.AggregateFunction; -import io.trino.spi.expression.Variable; -import io.trino.spi.type.DoubleType; - -import java.util.Optional; - -import static com.google.common.base.Verify.verify; -import static io.trino.matching.Capture.newCapture; -import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.basicAggregation; -import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.expressionType; -import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.functionName; -import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.singleInput; -import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.variable; -import static io.trino.spi.type.DoubleType.DOUBLE; -import static java.lang.String.format; - -public class ImplementSybaseVariance - implements AggregateFunctionRule -{ - private static final Capture INPUT = newCapture(); - - @Override - public Pattern getPattern() - { - return basicAggregation() - .with(functionName().equalTo("variance")) - .with(singleInput().matching( - variable() - .with(expressionType().matching(DoubleType.class::isInstance)) - .capturedAs(INPUT))); - } - - @Override - public Optional rewrite(AggregateFunction aggregateFunction, Captures captures, RewriteContext context) - { - Variable input = captures.get(INPUT); - JdbcColumnHandle columnHandle = (JdbcColumnHandle) context.getAssignment(input.getName()); - verify(columnHandle.getColumnType().equals(DOUBLE)); - verify(aggregateFunction.getOutputType().equals(DOUBLE)); - - return Optional.of(new JdbcExpression( - format("VAR(%s)", context.getIdentifierQuote().apply(columnHandle.getColumnName())), - columnHandle.getJdbcTypeHandle())); - } -} diff --git a/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseVariancePop.java b/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseVariancePop.java deleted file mode 100644 index 462605f4ee569..0000000000000 --- a/plugin/trino-sybase/src/main/java/io/trino/plugin/sybase/ImplementSybaseVariancePop.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * 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.matching.Capture; -import io.trino.matching.Captures; -import io.trino.matching.Pattern; -import io.trino.plugin.jdbc.JdbcColumnHandle; -import io.trino.plugin.jdbc.JdbcExpression; -import io.trino.plugin.jdbc.expression.AggregateFunctionRule; -import io.trino.spi.connector.AggregateFunction; -import io.trino.spi.expression.Variable; -import io.trino.spi.type.DoubleType; - -import java.util.Optional; - -import static com.google.common.base.Verify.verify; -import static io.trino.matching.Capture.newCapture; -import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.basicAggregation; -import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.expressionType; -import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.functionName; -import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.singleInput; -import static io.trino.plugin.jdbc.expression.AggregateFunctionPatterns.variable; -import static io.trino.spi.type.DoubleType.DOUBLE; -import static java.lang.String.format; - -public class ImplementSybaseVariancePop - implements AggregateFunctionRule -{ - private static final Capture INPUT = newCapture(); - - @Override - public Pattern getPattern() - { - return basicAggregation() - .with(functionName().equalTo("var_pop")) - .with(singleInput().matching( - variable() - .with(expressionType().matching(DoubleType.class::isInstance)) - .capturedAs(INPUT))); - } - - @Override - public Optional rewrite(AggregateFunction aggregateFunction, Captures captures, RewriteContext context) - { - Variable input = captures.get(INPUT); - JdbcColumnHandle columnHandle = (JdbcColumnHandle) context.getAssignment(input.getName()); - verify(columnHandle.getColumnType().equals(DOUBLE)); - verify(aggregateFunction.getOutputType().equals(DOUBLE)); - - return Optional.of(new JdbcExpression( - format("VARP(%s)", context.getIdentifierQuote().apply(columnHandle.getColumnName())), - columnHandle.getJdbcTypeHandle())); - } -} 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 index c5308e006ee30..acc677be678ec 100644 --- 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 @@ -20,22 +20,11 @@ import io.trino.plugin.jdbc.ColumnMapping; import io.trino.plugin.jdbc.ConnectionFactory; import io.trino.plugin.jdbc.JdbcColumnHandle; -import io.trino.plugin.jdbc.JdbcExpression; import io.trino.plugin.jdbc.JdbcTableHandle; import io.trino.plugin.jdbc.JdbcTypeHandle; import io.trino.plugin.jdbc.WriteMapping; -import io.trino.plugin.jdbc.expression.AggregateFunctionRewriter; -import io.trino.plugin.jdbc.expression.AggregateFunctionRule; -import io.trino.plugin.jdbc.expression.ImplementAvgDecimal; -import io.trino.plugin.jdbc.expression.ImplementAvgFloatingPoint; -import io.trino.plugin.jdbc.expression.ImplementCount; -import io.trino.plugin.jdbc.expression.ImplementCountAll; -import io.trino.plugin.jdbc.expression.ImplementMinMax; -import io.trino.plugin.jdbc.expression.ImplementSum; import io.trino.plugin.jdbc.mapping.IdentifierMapping; import io.trino.spi.TrinoException; -import io.trino.spi.connector.AggregateFunction; -import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.SchemaTableName; import io.trino.spi.type.CharType; @@ -51,7 +40,6 @@ import java.sql.Types; import java.util.Collection; import java.util.List; -import java.util.Map; import java.util.Optional; import java.util.function.BiFunction; @@ -73,8 +61,6 @@ public class SybaseClient private static final Joiner DOT_JOINER = Joiner.on("."); - private final AggregateFunctionRewriter aggregateFunctionRewriter; - @Override public void createSchema(ConnectorSession session, String schemaName) { @@ -88,23 +74,6 @@ public SybaseClient(BaseJdbcConfig config, ConnectionFactory connectionFactory, super(config, "\"", connectionFactory, identifierMapping); requireNonNull(config, "sybaseConfig is null"); - - JdbcTypeHandle bigintTypeHandle = new JdbcTypeHandle(Types.BIGINT, Optional.of("bigint"), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty()); - this.aggregateFunctionRewriter = new AggregateFunctionRewriter( - this::quoted, - ImmutableSet.builder() - .add(new ImplementCountAll(bigintTypeHandle)) - .add(new ImplementCount(bigintTypeHandle)) - .add(new ImplementMinMax()) - .add(new ImplementSum(SybaseClient::toTypeHandle)) - .add(new ImplementAvgFloatingPoint()) - .add(new ImplementAvgDecimal()) - .add(new ImplementAvgBigint()) - .add(new ImplementSybaseStdev()) - .add(new ImplementSybaseStddevPop()) - .add(new ImplementSybaseVariance()) - .add(new ImplementSybaseVariancePop()) - .build()); } @Override @@ -209,13 +178,6 @@ public WriteMapping toWriteMapping(ConnectorSession session, Type type) return legacyToWriteMapping(session, type); } - @Override - public Optional implementAggregation(ConnectorSession session, AggregateFunction aggregate, Map assignments) - { - // TODO support complex ConnectorExpressions - return aggregateFunctionRewriter.rewrite(session, aggregate, assignments); - } - private static Optional toTypeHandle(DecimalType decimalType) { return Optional.of(new JdbcTypeHandle(Types.NUMERIC, Optional.of("decimal"), Optional.of(decimalType.getPrecision()), Optional.of(decimalType.getScale()), Optional.empty(), Optional.empty())); From 45ba3c7f314ca380d29be6d7de3e2c1c4f3808c2 Mon Sep 17 00:00:00 2001 From: academy-codex Date: Sat, 17 Jul 2021 12:35:43 +0530 Subject: [PATCH 03/11] review comments fixes; --- .../io/trino/plugin/sybase/SybaseClient.java | 7 ---- .../plugin/sybase/SybaseQueryRunner.java | 35 +++++++++++++------ .../sybase/TestSybaseConnectorTest.java | 2 +- .../trino/plugin/sybase/TestSybasePlugin.java | 2 +- pom.xml | 1 + 5 files changed, 28 insertions(+), 19 deletions(-) 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 index acc677be678ec..863a237c4a0d1 100644 --- 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 @@ -61,13 +61,6 @@ public class SybaseClient private static final Joiner DOT_JOINER = Joiner.on("."); - @Override - public void createSchema(ConnectorSession session, String schemaName) - { - // TODO: Add try-catch for error message parsing (test cases patch) - super.createSchema(session, schemaName); - } - @Inject public SybaseClient(BaseJdbcConfig config, ConnectionFactory connectionFactory, IdentifierMapping identifierMapping) { diff --git a/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/SybaseQueryRunner.java b/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/SybaseQueryRunner.java index abee2b978a49c..fc6fb0b6a763a 100644 --- a/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/SybaseQueryRunner.java +++ b/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/SybaseQueryRunner.java @@ -13,12 +13,12 @@ */ package io.trino.plugin.sybase; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import io.airlift.log.Logger; +import io.airlift.log.Logging; import io.trino.Session; import io.trino.plugin.tpch.TpchPlugin; import io.trino.testing.DistributedQueryRunner; -import io.trino.testing.QueryRunner; import io.trino.tpch.TpchTable; import java.util.HashMap; @@ -35,16 +35,15 @@ private SybaseQueryRunner() {} private static final String TPCH_SCHEMA = "tpch"; - public static QueryRunner createSybaseQueryRunner(TestingSybaseServer server, TpchTable... tables) + public static DistributedQueryRunner createSybaseQueryRunner(TestingSybaseServer server, + Map extraProperties, + Map connectorProperties, + Iterable> tables) throws Exception { - return createSybaseQueryRunner(server, ImmutableMap.of(), ImmutableList.copyOf(tables)); - } - - public static QueryRunner createSybaseQueryRunner(TestingSybaseServer server, Map connectorProperties, Iterable> tables) - throws Exception - { - DistributedQueryRunner queryRunner = DistributedQueryRunner.builder(createSession()).build(); + DistributedQueryRunner queryRunner = DistributedQueryRunner.builder(createSession()) + .setExtraProperties(extraProperties) + .build(); try { queryRunner.installPlugin(new TpchPlugin()); queryRunner.createCatalog("tpch", "tpch"); @@ -75,4 +74,20 @@ private static Session createSession() .setSchema("dbo") .build(); } + + public static void main(String[] args) + throws Exception + { + Logging.initialize(); + + DistributedQueryRunner queryRunner = createSybaseQueryRunner( + new TestingSybaseServer(), + ImmutableMap.of("http-server.http.port", "8080"), + ImmutableMap.of(), + TpchTable.getTables()); + + Logger log = Logger.get(SybaseQueryRunner.class); + log.info("======== SERVER STARTED ========"); + log.info("\n====\n%s\n====", queryRunner.getCoordinator().getBaseUrl()); + } } diff --git a/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/TestSybaseConnectorTest.java b/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/TestSybaseConnectorTest.java index 4f6e782c499de..5e96856b60af3 100644 --- a/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/TestSybaseConnectorTest.java +++ b/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/TestSybaseConnectorTest.java @@ -32,7 +32,7 @@ protected QueryRunner createQueryRunner() throws Exception { sybaseServer = new TestingSybaseServer(); - return createSybaseQueryRunner(sybaseServer, ImmutableMap.of(), REQUIRED_TPCH_TABLES); + return createSybaseQueryRunner(sybaseServer, ImmutableMap.of(), ImmutableMap.of(), REQUIRED_TPCH_TABLES); } @AfterClass(alwaysRun = true) 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 index 0f463e61c510e..2cdd2dd32318c 100644 --- 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 @@ -28,6 +28,6 @@ public void testCreateConnector() { Plugin plugin = new SybasePlugin(); ConnectorFactory factory = getOnlyElement(plugin.getConnectorFactories()); - factory.create("test", ImmutableMap.of("connection-url", "jdbc:mariadb://test"), new TestingConnectorContext()).shutdown(); + factory.create("test", ImmutableMap.of("connection-url", "jdbc:jtds:sybase://testdb"), new TestingConnectorContext()).shutdown(); } } diff --git a/pom.xml b/pom.xml index badf018378081..c40e38c84e137 100644 --- a/pom.xml +++ b/pom.xml @@ -140,6 +140,7 @@ plugin/trino-resource-group-managers plugin/trino-session-property-managers plugin/trino-sqlserver + plugin/trino-sybase plugin/trino-teradata-functions plugin/trino-thrift plugin/trino-thrift-api From 618cce618797e2292121ba4537d745737066a5bf Mon Sep 17 00:00:00 2001 From: academy-codex Date: Sat, 17 Jul 2021 12:37:39 +0530 Subject: [PATCH 04/11] removed license --- .../src/test/resources/container-license-acceptance.txt | 1 - 1 file changed, 1 deletion(-) delete mode 100644 plugin/trino-sybase/src/test/resources/container-license-acceptance.txt diff --git a/plugin/trino-sybase/src/test/resources/container-license-acceptance.txt b/plugin/trino-sybase/src/test/resources/container-license-acceptance.txt deleted file mode 100644 index ab843953ef7ff..0000000000000 --- a/plugin/trino-sybase/src/test/resources/container-license-acceptance.txt +++ /dev/null @@ -1 +0,0 @@ -microsoft/mssql-server-linux:2017-CU13 From aff265a67105ee0b813863599d7f60c3101bfff9 Mon Sep 17 00:00:00 2001 From: Siddhant Chadha Date: Sat, 17 Jul 2021 12:40:47 +0530 Subject: [PATCH 05/11] Update pom.xml sybase trino version update --- plugin/trino-sybase/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/plugin/trino-sybase/pom.xml b/plugin/trino-sybase/pom.xml index 33ddf6d985f8c..ebaa101bd74f0 100644 --- a/plugin/trino-sybase/pom.xml +++ b/plugin/trino-sybase/pom.xml @@ -5,7 +5,7 @@ io.trino trino-root - 359-SNAPSHOT + 360-SNAPSHOT ../../pom.xml From 9bfc73adeb3a5eb4d701affd689e98dd8d53229b Mon Sep 17 00:00:00 2001 From: academy-codex Date: Sat, 17 Jul 2021 12:44:01 +0530 Subject: [PATCH 06/11] remove unused method; --- .../plugin/sybase/TestingSybaseServer.java | 18 ------------------ 1 file changed, 18 deletions(-) diff --git a/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/TestingSybaseServer.java b/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/TestingSybaseServer.java index e9685f5878b78..abf230c9e732f 100644 --- a/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/TestingSybaseServer.java +++ b/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/TestingSybaseServer.java @@ -90,22 +90,4 @@ public void execute(String sql, String user, String password) throw new RuntimeException(e); } } - - public ResultSet executeQuery(String sql) - { - try { - Class.forName("net.sourceforge.jtds.jdbc.Driver"); - } - catch (ClassNotFoundException e) { - e.printStackTrace(); - } - try (Connection connection = DriverManager.getConnection(getJdbcUrl(), getUsername(), getPassword()); - Statement statement = connection.createStatement()) { - return connection.getMetaData().getColumns("testdb", null, "Persons2", null); -// return statement.executeQuery(sql); - } - catch (SQLException e) { - throw new RuntimeException(e); - } - } } From 49ebdb0c19ea9c4ea5912eb4f3906cc1f84b0ab5 Mon Sep 17 00:00:00 2001 From: academy-codex Date: Sat, 17 Jul 2021 12:48:05 +0530 Subject: [PATCH 07/11] removed unneeded comment; --- .../src/main/java/io/trino/plugin/sybase/SybaseClient.java | 1 - 1 file changed, 1 deletion(-) 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 index 863a237c4a0d1..5313d11dd3164 100644 --- 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 @@ -56,7 +56,6 @@ public class SybaseClient extends BaseJdbcClient { - // SqlServer supports 2100 parameters in prepared statement, let's create a space for about 4 big IN predicates public static final int SYBASE_MAX_LIST_EXPRESSIONS = 500; private static final Joiner DOT_JOINER = Joiner.on("."); From 1ce0844cc9f540269aa3fa71a6961f8eb3b2a130 Mon Sep 17 00:00:00 2001 From: academy-codex Date: Tue, 20 Jul 2021 12:05:58 +0530 Subject: [PATCH 08/11] fix CI; fix query runner; --- .../java/io/trino/plugin/sybase/SybaseQueryRunner.java | 9 +++++++-- .../java/io/trino/plugin/sybase/TestingSybaseServer.java | 1 - 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/SybaseQueryRunner.java b/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/SybaseQueryRunner.java index fc6fb0b6a763a..f49cbdca7c52e 100644 --- a/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/SybaseQueryRunner.java +++ b/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/SybaseQueryRunner.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.sybase; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.airlift.log.Logger; import io.airlift.log.Logging; @@ -28,6 +29,10 @@ import static io.trino.plugin.sybase.SybaseDataLoader.copyTpchTables; import static io.trino.plugin.tpch.TpchMetadata.TINY_SCHEMA_NAME; import static io.trino.testing.TestingSession.testSessionBuilder; +import static io.trino.tpch.TpchTable.CUSTOMER; +import static io.trino.tpch.TpchTable.NATION; +import static io.trino.tpch.TpchTable.ORDERS; +import static io.trino.tpch.TpchTable.REGION; public class SybaseQueryRunner { @@ -82,9 +87,9 @@ public static void main(String[] args) DistributedQueryRunner queryRunner = createSybaseQueryRunner( new TestingSybaseServer(), - ImmutableMap.of("http-server.http.port", "8080"), ImmutableMap.of(), - TpchTable.getTables()); + ImmutableMap.of(), + ImmutableList.of(CUSTOMER, NATION, ORDERS, REGION)); Logger log = Logger.get(SybaseQueryRunner.class); log.info("======== SERVER STARTED ========"); diff --git a/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/TestingSybaseServer.java b/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/TestingSybaseServer.java index abf230c9e732f..84097e89b403e 100644 --- a/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/TestingSybaseServer.java +++ b/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/TestingSybaseServer.java @@ -18,7 +18,6 @@ import java.sql.Connection; import java.sql.DriverManager; -import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; From e17e2a3efd84df00df497abe141ef2a59ab7d03f Mon Sep 17 00:00:00 2001 From: academy-codex Date: Tue, 20 Jul 2021 12:07:28 +0530 Subject: [PATCH 09/11] fix unused varuable TPCH_SCHEMA; --- .../src/test/java/io/trino/plugin/sybase/SybaseQueryRunner.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/SybaseQueryRunner.java b/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/SybaseQueryRunner.java index f49cbdca7c52e..143fe2233d830 100644 --- a/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/SybaseQueryRunner.java +++ b/plugin/trino-sybase/src/test/java/io/trino/plugin/sybase/SybaseQueryRunner.java @@ -38,8 +38,6 @@ public class SybaseQueryRunner { private SybaseQueryRunner() {} - private static final String TPCH_SCHEMA = "tpch"; - public static DistributedQueryRunner createSybaseQueryRunner(TestingSybaseServer server, Map extraProperties, Map connectorProperties, From 38bc78ac2123554e8b2b5fb74bef547b7e456bc8 Mon Sep 17 00:00:00 2001 From: academy-codex Date: Tue, 20 Jul 2021 12:51:55 +0530 Subject: [PATCH 10/11] dependency fix; --- plugin/trino-sybase/pom.xml | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/plugin/trino-sybase/pom.xml b/plugin/trino-sybase/pom.xml index 33ddf6d985f8c..694bf194a88a7 100644 --- a/plugin/trino-sybase/pom.xml +++ b/plugin/trino-sybase/pom.xml @@ -40,13 +40,13 @@ io.airlift - units + log-manager - - - - + + io.airlift + units + com.google.guava From 785ca7eb367a6a5e0fbe2d078c97aafedef8b144 Mon Sep 17 00:00:00 2001 From: academy-codex Date: Tue, 20 Jul 2021 14:27:31 +0530 Subject: [PATCH 11/11] dependency fix; --- plugin/trino-sybase/pom.xml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/plugin/trino-sybase/pom.xml b/plugin/trino-sybase/pom.xml index 694bf194a88a7..a295d38c9fa06 100644 --- a/plugin/trino-sybase/pom.xml +++ b/plugin/trino-sybase/pom.xml @@ -23,10 +23,10 @@ trino-base-jdbc - - io.trino - trino-matching - + + + + io.airlift