From 05b1226891c1eb3a7aac3c238b889679895153c9 Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Sun, 23 Aug 2020 11:16:18 +0900 Subject: [PATCH] Allow INSERT null for SQL Server varbinary type --- .../plugin/sqlserver/SqlServerClient.java | 49 ++++++++++++ .../sqlserver/TestSqlServerTypeMapping.java | 78 +++++++++++++++++++ 2 files changed, 127 insertions(+) create mode 100644 presto-sqlserver/src/test/java/io/prestosql/plugin/sqlserver/TestSqlServerTypeMapping.java diff --git a/presto-sqlserver/src/main/java/io/prestosql/plugin/sqlserver/SqlServerClient.java b/presto-sqlserver/src/main/java/io/prestosql/plugin/sqlserver/SqlServerClient.java index ac8973e75b20..840b1a538066 100644 --- a/presto-sqlserver/src/main/java/io/prestosql/plugin/sqlserver/SqlServerClient.java +++ b/presto-sqlserver/src/main/java/io/prestosql/plugin/sqlserver/SqlServerClient.java @@ -14,6 +14,7 @@ package io.prestosql.plugin.sqlserver; import com.google.common.base.Joiner; +import io.airlift.slice.Slice; import io.prestosql.plugin.jdbc.BaseJdbcClient; import io.prestosql.plugin.jdbc.BaseJdbcConfig; import io.prestosql.plugin.jdbc.ColumnMapping; @@ -24,6 +25,7 @@ import io.prestosql.plugin.jdbc.JdbcTypeHandle; import io.prestosql.plugin.jdbc.PredicatePushdownController; import io.prestosql.plugin.jdbc.PredicatePushdownController.DomainPushdownResult; +import io.prestosql.plugin.jdbc.SliceWriteFunction; import io.prestosql.plugin.jdbc.WriteMapping; import io.prestosql.spi.PrestoException; import io.prestosql.spi.connector.ConnectorSession; @@ -31,21 +33,28 @@ import io.prestosql.spi.predicate.Domain; import io.prestosql.spi.type.CharType; import io.prestosql.spi.type.Type; +import io.prestosql.spi.type.VarbinaryType; import io.prestosql.spi.type.VarcharType; import javax.inject.Inject; import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.SQLException; import java.util.List; import java.util.Optional; import java.util.function.BiFunction; import static com.google.common.base.Preconditions.checkArgument; +import static io.airlift.slice.Slices.wrappedBuffer; +import static io.prestosql.plugin.jdbc.ColumnMapping.DISABLE_PUSHDOWN; +import static io.prestosql.plugin.jdbc.JdbcErrorCode.JDBC_ERROR; import static io.prestosql.plugin.jdbc.StandardColumnMappings.booleanWriteFunction; import static io.prestosql.plugin.jdbc.StandardColumnMappings.charWriteFunction; import static io.prestosql.plugin.jdbc.StandardColumnMappings.varcharWriteFunction; import static io.prestosql.spi.StandardErrorCode.NOT_SUPPORTED; import static io.prestosql.spi.type.BooleanType.BOOLEAN; +import static io.prestosql.spi.type.VarbinaryType.VARBINARY; import static io.prestosql.spi.type.Varchars.isVarcharType; import static java.lang.String.format; import static java.util.stream.Collectors.joining; @@ -117,6 +126,14 @@ public Optional toPrestoType(ConnectorSession session, Connection if (mapping.isPresent()) { return mapping; } + + String jdbcTypeName = typeHandle.getJdbcTypeName() + .orElseThrow(() -> new PrestoException(JDBC_ERROR, "Type name is missing: " + typeHandle)); + + if (jdbcTypeName.equals("varbinary")) { + return Optional.of(varbinaryColumnMapping()); + } + // TODO (https://github.com/prestosql/presto/issues/4593) implement proper type mapping return super.toPrestoType(session, connection, typeHandle) .map(columnMapping -> new ColumnMapping( @@ -157,6 +174,10 @@ public WriteMapping toWriteMapping(ConnectorSession session, Type type) return WriteMapping.sliceMapping(dataType, charWriteFunction()); } + if (type instanceof VarbinaryType) { + return WriteMapping.sliceMapping("varbinary(max)", varbinaryWriteFunction()); + } + // TODO implement proper type mapping return super.toWriteMapping(session, type); } @@ -186,4 +207,32 @@ private static String singleQuote(String literal) { return "\'" + literal + "\'"; } + + public static ColumnMapping varbinaryColumnMapping() + { + return ColumnMapping.sliceMapping( + VARBINARY, + (resultSet, columnIndex) -> wrappedBuffer(resultSet.getBytes(columnIndex)), + varbinaryWriteFunction(), + DISABLE_PUSHDOWN); + } + + private static SliceWriteFunction varbinaryWriteFunction() + { + return new SliceWriteFunction() { + @Override + public void set(PreparedStatement statement, int index, Slice value) + throws SQLException + { + statement.setBytes(index, value.getBytes()); + } + + @Override + public void setNull(PreparedStatement statement, int index) + throws SQLException + { + statement.setBytes(index, null); + } + }; + } } diff --git a/presto-sqlserver/src/test/java/io/prestosql/plugin/sqlserver/TestSqlServerTypeMapping.java b/presto-sqlserver/src/test/java/io/prestosql/plugin/sqlserver/TestSqlServerTypeMapping.java new file mode 100644 index 000000000000..9fc4d4e38395 --- /dev/null +++ b/presto-sqlserver/src/test/java/io/prestosql/plugin/sqlserver/TestSqlServerTypeMapping.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.prestosql.plugin.sqlserver; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.prestosql.Session; +import io.prestosql.testing.AbstractTestQueryFramework; +import io.prestosql.testing.QueryRunner; +import io.prestosql.testing.datatype.CreateAsSelectDataSetup; +import io.prestosql.testing.datatype.DataSetup; +import io.prestosql.testing.datatype.DataTypeTest; +import io.prestosql.testing.sql.PrestoSqlExecutor; +import org.testng.annotations.AfterClass; +import org.testng.annotations.Test; + +import static io.prestosql.plugin.sqlserver.SqlServerQueryRunner.createSqlServerQueryRunner; +import static io.prestosql.testing.datatype.DataType.varbinaryDataType; +import static java.nio.charset.StandardCharsets.UTF_16LE; +import static java.nio.charset.StandardCharsets.UTF_8; + +public class TestSqlServerTypeMapping + extends AbstractTestQueryFramework +{ + private TestingSqlServer sqlServer; + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + sqlServer = new TestingSqlServer(); + sqlServer.start(); + return createSqlServerQueryRunner( + sqlServer, + ImmutableMap.of(), + ImmutableList.of()); + } + + @AfterClass(alwaysRun = true) + public final void destroy() + { + sqlServer.close(); + } + + @Test + public void testVarbinary() + { + DataTypeTest.create() + .addRoundTrip(varbinaryDataType(), null) + .addRoundTrip(varbinaryDataType(), "hello".getBytes(UTF_8)) + .addRoundTrip(varbinaryDataType(), "Piękna łąka w 東京都".getBytes(UTF_8)) + .addRoundTrip(varbinaryDataType(), "Bag full of 💰".getBytes(UTF_16LE)) + .addRoundTrip(varbinaryDataType(), new byte[] {}) + .addRoundTrip(varbinaryDataType(), new byte[] {0, 1, 2, 3, 4, 5, 6, 7, 8, 13, -7, 54, 122, -89, 0, 0, 0}) + .execute(getQueryRunner(), prestoCreateAsSelect("test_varbinary")); + } + + private DataSetup prestoCreateAsSelect(String tableNamePrefix) + { + return prestoCreateAsSelect(getSession(), tableNamePrefix); + } + + private DataSetup prestoCreateAsSelect(Session session, String tableNamePrefix) + { + return new CreateAsSelectDataSetup(new PrestoSqlExecutor(getQueryRunner(), session), tableNamePrefix); + } +}