diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml
index 99cd1b95208e..c68f6e70421e 100644
--- a/.github/workflows/ci.yml
+++ b/.github/workflows/ci.yml
@@ -370,6 +370,7 @@ jobs:
!:trino-hdfs,
!:trino-hive,
!:trino-hive-formats,
+ !:trino-hsqldb,
!:trino-hudi,
!:trino-iceberg,
!:trino-ignite,
@@ -491,6 +492,7 @@ jobs:
- { modules: plugin/trino-hive }
- { modules: plugin/trino-hive, profile: fte-tests }
- { modules: plugin/trino-hive, profile: test-parquet }
+ - { modules: plugin/trino-hsqldb }
- { modules: plugin/trino-hudi }
- { modules: plugin/trino-iceberg }
- { modules: plugin/trino-iceberg, profile: cloud-tests }
diff --git a/core/trino-server/src/main/provisio/trino.xml b/core/trino-server/src/main/provisio/trino.xml
index 97a366b39290..61607e4dfed7 100644
--- a/core/trino-server/src/main/provisio/trino.xml
+++ b/core/trino-server/src/main/provisio/trino.xml
@@ -100,6 +100,12 @@
+
+
+
+
+
+
diff --git a/docs/src/main/sphinx/connector.md b/docs/src/main/sphinx/connector.md
index a954b30cf059..c94de33a065e 100644
--- a/docs/src/main/sphinx/connector.md
+++ b/docs/src/main/sphinx/connector.md
@@ -20,6 +20,7 @@ Exasol
Faker
Google Sheets
Hive
+HsqlDB
Hudi
Iceberg
Ignite
diff --git a/docs/src/main/sphinx/connector/hsqldb.md b/docs/src/main/sphinx/connector/hsqldb.md
new file mode 100644
index 000000000000..8443cb803393
--- /dev/null
+++ b/docs/src/main/sphinx/connector/hsqldb.md
@@ -0,0 +1,327 @@
+---
+myst:
+ substitutions:
+ default_domain_compaction_threshold: '`256`'
+---
+
+# HsqlDB connector
+
+```{raw} html
+
+```
+
+The HsqlDB connector allows querying and creating tables in an external HsqlDB
+database.
+
+## Requirements
+
+To connect to HsqlDB, you need:
+
+- HsqlDB version 2.7.4 or higher.
+- Network access from the Trino coordinator and workers to HsqlDB. Port
+ 9001 is the default port.
+
+## Configuration
+
+To configure the HsqlDB connector, create a catalog properties file in
+`etc/catalog` named, for example, `example.properties`, to mount the HsqlDB
+connector as the `example` catalog. Create the file with the following
+contents, replacing the connection properties as appropriate for your setup:
+
+```text
+connector.name=hsqldb
+connection-url=jdbc:hsqldb:hsql://localhost:9001/
+connection-user=SA
+connection-password=
+```
+
+The `connection-user` and `connection-password` are typically required and
+determine the user credentials for the connection, often a service user. You can
+use {doc}`secrets ` to avoid actual values in the catalog
+properties files.
+
+```{include} jdbc-authentication.fragment
+```
+
+```{include} jdbc-common-configurations.fragment
+```
+
+```{include} jdbc-domain-compaction-threshold.fragment
+```
+
+```{include} jdbc-case-insensitive-matching.fragment
+```
+
+## Querying HsqlDB
+
+The HsqlDB connector provides a catalog for every HsqlDB *database* (ie: every catalog properties file).
+You can see the available HsqlDB databases by running `SHOW CATALOGS`:
+
+```
+SHOW CATALOGS;
+```
+
+If you have a HsqlDB schema named `web`, you can view the tables
+in this schema by running `SHOW TABLES`:
+
+```
+SHOW TABLES FROM example.web;
+```
+
+You can see a list of the columns in the `clicks` table in the `web`
+schema using either of the following:
+
+```
+DESCRIBE example.web.clicks;
+SHOW COLUMNS FROM example.web.clicks;
+```
+
+Finally, you can access the `clicks` table in the `web` database:
+
+```
+SELECT * FROM example.web.clicks;
+```
+
+If you used a different name for your catalog properties file, use
+that catalog name instead of `example` in the above examples.
+
+% hsqldb-type-mapping:
+
+## Type mapping
+
+Because Trino and HsqlDB each support types that the other does not, this
+connector {ref}`modifies some types ` when reading or
+writing data. Data types may not map the same way in both directions between
+Trino and the data source. Refer to the following sections for type mapping in
+each direction.
+
+### HsqlDB type to Trino type mapping
+
+The connector maps HsqlDB types to the corresponding Trino types according
+to the following table:
+
+:::{list-table} HsqlDB type to Trino type mapping
+:widths: 30, 30, 50
+:header-rows: 1
+
+* - HsqlDB type
+ - Trino type
+ - Notes
+* - `BOOLEAN`
+ - `BOOLEAN`
+ -
+* - `TINYINT`
+ - `TINYINT`
+ -
+* - `SMALLINT`
+ - `SMALLINT`
+ -
+* - `INTEGER` or `INT`
+ - `INTEGER`
+ -
+* - `BIGINT`
+ - `BIGINT`
+ -
+* - `DOUBLE` or `FLOAT`
+ - `DOUBLE`
+ -
+* - `DECIMAL(p,s)`
+ - `DECIMAL(p,s)`
+ -
+* - `CHAR(n)`
+ - `CHAR(n)`
+ -
+* - `VARCHAR(n)`
+ - `VARCHAR(n)`
+ -
+* - `CLOB(n)`
+ - `VARCHAR(n)`
+ -
+* - `BINARY(n)`
+ - `VARBINARY`
+ -
+* - `VARBINARY(n)`
+ - `VARBINARY`
+ -
+* - `BLOB(n)`
+ - `VARBINARY`
+ -
+* - `UUID`
+ - `UUID`
+ -
+* - `DATE`
+ - `DATE`
+ -
+* - `TIME(n)`
+ - `TIME(n)`
+ -
+* - `TIME(n) WITH TIME ZONE`
+ - `TIME(n) WITH TIME ZONE`
+ -
+* - `TIMESTAMP(n)`
+ - `TIMESTAMP(n)`
+ -
+* - `TIMESTAMP(n) WITH TIME ZONE`
+ - `TIMESTAMP(n) WITH TIME ZONE`
+ -
+* - `INTERVAL`
+ - `INTERVAL`
+ -
+:::
+
+No other types are supported.
+
+### Trino type mapping to HsqlDB type mapping
+
+The connector maps Trino types to the corresponding HsqlDB types according
+to the following table:
+
+:::{list-table} Trino type mapping to HsqlDB type mapping
+:widths: 30, 25, 50
+:header-rows: 1
+
+* - Trino type
+ - HsqlDB type
+ - Notes
+* - `BOOLEAN`
+ - `BOOLEAN`
+ -
+* - `TINYINT`
+ - `TINYINT`
+ -
+* - `SMALLINT`
+ - `SMALLINT`
+ -
+* - `INTEGER`
+ - `INTEGER`
+ -
+* - `BIGINT`
+ - `BIGINT`
+ -
+* - `DOUBLE`
+ - `DOUBLE`
+ -
+* - `DECIMAL(p,s)`
+ - `DECIMAL(p,s)`
+ -
+* - `CHAR(n)`
+ - `CHAR(n)`
+ -
+* - `VARCHAR(n)`
+ - `VARCHAR(n)`
+ -
+* - `VARBINARY`
+ - `VARBINARY(32768)`
+ -
+* - `UUID`
+ - `UUID`
+ -
+* - `DATE`
+ - `DATE`
+ -
+* - `TIME(n)`
+ - `TIME(n)`
+ -
+* - `TIME(n) WITH TIME ZONE`
+ - `TIME(n) WITH TIME ZONE`
+ -
+* - `TIMESTAMP(n)`
+ - `TIMESTAMP(n)`
+ -
+* - `TIMESTAMP(n) WITH TIME ZONE`
+ - `TIMESTAMP(n) WITH TIME ZONE`
+ -
+* - `INTERVAL`
+ - `INTERVAL`
+ -
+:::
+
+No other types are supported.
+
+Complete list of [HsqlDB data types](https://hsqldb.org/doc/2.0/guide/guide.html#sgc_data_type_guide).
+
+```{include} jdbc-type-mapping.fragment
+```
+
+(hsqldb-sql-support)=
+## SQL support
+
+The connector provides read access and write access to data and metadata in a
+HsqlDB database. In addition to the [globally
+available](sql-globally-available) and [read operation](sql-read-operations)
+statements, the connector supports the following features:
+
+- [](/sql/insert), see also [](hsqldb-insert)
+- [](/sql/update), see also [](hsqldb-update)
+- [](/sql/delete), see also [](hsqldb-delete)
+- [](/sql/truncate)
+- [](/sql/create-table)
+- [](/sql/create-table-as)
+- [](/sql/drop-table)
+- [](/sql/alter-table)
+- [](/sql/create-schema)
+- [](/sql/drop-schema)
+- [](hsqldb-procedures)
+- [](hsqldb-table-functions)
+
+(hsqldb-insert)=
+```{include} non-transactional-insert.fragment
+```
+
+(hsqldb-update)=
+```{include} sql-update-limitation.fragment
+```
+
+(hsqldb-delete)=
+```{include} sql-delete-limitation.fragment
+```
+
+(hsqldb-procedures)=
+### Procedures
+
+```{include} jdbc-procedures-flush.fragment
+```
+```{include} procedures-execute.fragment
+```
+
+(hsqldb-table-functions)=
+### Table functions
+
+The connector provides specific {doc}`table functions ` to
+access HsqlDB.
+
+(hsqldb-query-function)=
+#### `query(varchar) -> table`
+
+The `query` function allows you to query the underlying database directly. It
+requires syntax native to HsqlDB, because the full query is pushed down and
+processed in HsqlDB. This can be useful for accessing native features which are
+not available in Trino or for improving query performance in situations where
+running a query natively may be faster.
+
+```{include} query-passthrough-warning.fragment
+```
+
+As an example, query the `example` catalog and select the age of employees in `public` schema by
+using `TIMESTAMPDIFF` and `CURRENT_DATE`:
+
+```
+SELECT
+ age
+FROM
+ TABLE(
+ example.system.query(
+ query => 'SELECT
+ TIMESTAMPDIFF(
+ SQL_TSI_YEAR,
+ date_of_birth,
+ CURRENT_DATE
+ ) AS age
+ FROM
+ example.public.employees'
+ )
+ );
+```
+
+```{include} query-table-function-ordering.fragment
+```
diff --git a/docs/src/main/sphinx/static/img/hsqldb.png b/docs/src/main/sphinx/static/img/hsqldb.png
new file mode 100644
index 000000000000..e64caeaaea38
Binary files /dev/null and b/docs/src/main/sphinx/static/img/hsqldb.png differ
diff --git a/plugin/trino-hsqldb/pom.xml b/plugin/trino-hsqldb/pom.xml
new file mode 100644
index 000000000000..8e0624de0c82
--- /dev/null
+++ b/plugin/trino-hsqldb/pom.xml
@@ -0,0 +1,225 @@
+
+
+ 4.0.0
+
+
+ io.trino
+ trino-root
+ 479-SNAPSHOT
+ ../../pom.xml
+
+
+ trino-hsqldb
+ trino-plugin
+ Trino - HsqlDB connector
+
+
+
+ com.google.guava
+ guava
+
+
+
+ com.google.inject
+ guice
+ classes
+
+
+
+ io.airlift
+ configuration
+
+
+
+ io.trino
+ trino-base-jdbc
+
+
+
+ io.trino
+ trino-plugin-toolkit
+
+
+
+ jakarta.validation
+ jakarta.validation-api
+
+
+
+ org.hsqldb
+ hsqldb
+ 2.7.4
+
+
+
+ com.fasterxml.jackson.core
+ jackson-annotations
+ provided
+
+
+
+ io.airlift
+ slice
+ provided
+
+
+
+ io.opentelemetry
+ opentelemetry-api
+ provided
+
+
+
+ io.opentelemetry
+ opentelemetry-api-incubator
+ provided
+
+
+
+ io.opentelemetry
+ opentelemetry-context
+ provided
+
+
+
+ io.trino
+ trino-spi
+ provided
+
+
+
+ org.openjdk.jol
+ jol-core
+ provided
+
+
+
+ com.google.errorprone
+ error_prone_annotations
+ runtime
+
+
+
+ io.airlift
+ log
+ runtime
+
+
+
+ io.airlift
+ log-manager
+ runtime
+
+
+
+ io.airlift
+ units
+ runtime
+
+
+
+ io.airlift
+ junit-extensions
+ test
+
+
+
+ io.airlift
+ testing
+ test
+
+
+
+ io.trino
+ trino-base-jdbc
+ test-jar
+ test
+
+
+
+ io.trino
+ trino-exchange-filesystem
+ test
+
+
+
+ io.trino
+ trino-main
+ test
+
+
+
+ io.trino
+ trino-main
+ test-jar
+ test
+
+
+
+ io.trino
+ trino-plugin-toolkit
+ test-jar
+ test
+
+
+
+ io.trino
+ trino-testing
+ test
+
+
+
+ io.trino
+ trino-testing-containers
+ test
+
+
+
+ io.trino
+ trino-testing-services
+ test
+
+
+
+ io.trino
+ trino-tpch
+ test
+
+
+
+ io.trino.tpch
+ tpch
+ test
+
+
+
+ org.assertj
+ assertj-core
+ test
+
+
+
+ org.jetbrains
+ annotations
+ test
+
+
+
+ org.junit.jupiter
+ junit-jupiter-api
+ test
+
+
+
+ org.junit.jupiter
+ junit-jupiter-engine
+ test
+
+
+
+ org.testcontainers
+ testcontainers
+ test
+
+
+
diff --git a/plugin/trino-hsqldb/src/main/java/io/trino/plugin/hsqldb/HsqlDbClient.java b/plugin/trino-hsqldb/src/main/java/io/trino/plugin/hsqldb/HsqlDbClient.java
new file mode 100644
index 000000000000..4db42d4f904f
--- /dev/null
+++ b/plugin/trino-hsqldb/src/main/java/io/trino/plugin/hsqldb/HsqlDbClient.java
@@ -0,0 +1,566 @@
+/*
+ * 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.hsqldb;
+
+import com.google.common.collect.ImmutableList;
+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.CaseSensitivity;
+import io.trino.plugin.jdbc.ColumnMapping;
+import io.trino.plugin.jdbc.ConnectionFactory;
+import io.trino.plugin.jdbc.JdbcColumnHandle;
+import io.trino.plugin.jdbc.JdbcJoinCondition;
+import io.trino.plugin.jdbc.JdbcSortItem;
+import io.trino.plugin.jdbc.JdbcTableHandle;
+import io.trino.plugin.jdbc.JdbcTypeHandle;
+import io.trino.plugin.jdbc.LongReadFunction;
+import io.trino.plugin.jdbc.LongWriteFunction;
+import io.trino.plugin.jdbc.PredicatePushdownController;
+import io.trino.plugin.jdbc.QueryBuilder;
+import io.trino.plugin.jdbc.RemoteTableName;
+import io.trino.plugin.jdbc.WriteMapping;
+import io.trino.plugin.jdbc.logging.RemoteQueryModifier;
+import io.trino.spi.TrinoException;
+import io.trino.spi.connector.ConnectorSession;
+import io.trino.spi.connector.ConnectorTableMetadata;
+import io.trino.spi.connector.SchemaTableName;
+import io.trino.spi.predicate.Domain;
+import io.trino.spi.predicate.ValueSet;
+import io.trino.spi.statistics.TableStatistics;
+import io.trino.spi.type.CharType;
+import io.trino.spi.type.DecimalType;
+import io.trino.spi.type.Decimals;
+import io.trino.spi.type.TimeType;
+import io.trino.spi.type.TimeWithTimeZoneType;
+import io.trino.spi.type.Type;
+import io.trino.spi.type.VarcharType;
+
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.SQLSyntaxErrorException;
+import java.sql.Types;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.OffsetTime;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.stream.Stream;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Verify.verify;
+import static io.trino.plugin.jdbc.CaseSensitivity.CASE_INSENSITIVE;
+import static io.trino.plugin.jdbc.CaseSensitivity.CASE_SENSITIVE;
+import static io.trino.plugin.jdbc.DecimalConfig.DecimalMapping.ALLOW_OVERFLOW;
+import static io.trino.plugin.jdbc.DecimalSessionSessionProperties.getDecimalDefaultScale;
+import static io.trino.plugin.jdbc.DecimalSessionSessionProperties.getDecimalRounding;
+import static io.trino.plugin.jdbc.DecimalSessionSessionProperties.getDecimalRoundingMode;
+import static io.trino.plugin.jdbc.JdbcErrorCode.JDBC_ERROR;
+import static io.trino.plugin.jdbc.JdbcMetadataSessionProperties.getDomainCompactionThreshold;
+import static io.trino.plugin.jdbc.PredicatePushdownController.CASE_INSENSITIVE_CHARACTER_PUSHDOWN;
+import static io.trino.plugin.jdbc.PredicatePushdownController.DISABLE_PUSHDOWN;
+import static io.trino.plugin.jdbc.PredicatePushdownController.FULL_PUSHDOWN;
+import static io.trino.plugin.jdbc.StandardColumnMappings.bigintColumnMapping;
+import static io.trino.plugin.jdbc.StandardColumnMappings.bigintWriteFunction;
+import static io.trino.plugin.jdbc.StandardColumnMappings.booleanColumnMapping;
+import static io.trino.plugin.jdbc.StandardColumnMappings.booleanWriteFunction;
+import static io.trino.plugin.jdbc.StandardColumnMappings.charReadFunction;
+import static io.trino.plugin.jdbc.StandardColumnMappings.charWriteFunction;
+import static io.trino.plugin.jdbc.StandardColumnMappings.dateReadFunctionUsingLocalDate;
+import static io.trino.plugin.jdbc.StandardColumnMappings.decimalColumnMapping;
+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.integerColumnMapping;
+import static io.trino.plugin.jdbc.StandardColumnMappings.integerWriteFunction;
+import static io.trino.plugin.jdbc.StandardColumnMappings.longDecimalWriteFunction;
+import static io.trino.plugin.jdbc.StandardColumnMappings.realWriteFunction;
+import static io.trino.plugin.jdbc.StandardColumnMappings.shortDecimalWriteFunction;
+import static io.trino.plugin.jdbc.StandardColumnMappings.smallintColumnMapping;
+import static io.trino.plugin.jdbc.StandardColumnMappings.smallintWriteFunction;
+import static io.trino.plugin.jdbc.StandardColumnMappings.timeColumnMapping;
+import static io.trino.plugin.jdbc.StandardColumnMappings.timeWriteFunction;
+import static io.trino.plugin.jdbc.StandardColumnMappings.tinyintColumnMapping;
+import static io.trino.plugin.jdbc.StandardColumnMappings.tinyintWriteFunction;
+import static io.trino.plugin.jdbc.StandardColumnMappings.varbinaryColumnMapping;
+import static io.trino.plugin.jdbc.StandardColumnMappings.varbinaryWriteFunction;
+import static io.trino.plugin.jdbc.StandardColumnMappings.varcharReadFunction;
+import static io.trino.plugin.jdbc.StandardColumnMappings.varcharWriteFunction;
+import static io.trino.plugin.jdbc.TypeHandlingJdbcSessionProperties.getUnsupportedTypeHandling;
+import static io.trino.plugin.jdbc.UnsupportedTypeHandling.CONVERT_TO_VARCHAR;
+import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED;
+import static io.trino.spi.type.BigintType.BIGINT;
+import static io.trino.spi.type.BooleanType.BOOLEAN;
+import static io.trino.spi.type.CharType.createCharType;
+import static io.trino.spi.type.DateTimeEncoding.packTimeWithTimeZone;
+import static io.trino.spi.type.DateTimeEncoding.unpackOffsetMinutes;
+import static io.trino.spi.type.DateTimeEncoding.unpackTimeNanos;
+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.TimeType.createTimeType;
+import static io.trino.spi.type.TimeWithTimeZoneType.createTimeWithTimeZoneType;
+import static io.trino.spi.type.Timestamps.NANOSECONDS_PER_DAY;
+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 io.trino.spi.type.VarcharType.createVarcharType;
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static java.lang.String.format;
+import static java.lang.String.join;
+import static java.util.Objects.requireNonNull;
+import static java.util.stream.Collectors.joining;
+
+public class HsqlDbClient
+ extends BaseJdbcClient
+{
+ private static final int MAX_SUPPORTED_DATE_TIME_PRECISION = 9;
+
+ // HsqlDB driver returns width of time types instead of precision.
+ private static final int ZERO_PRECISION_TIME_COLUMN_SIZE = 8;
+ private static final int ZERO_PRECISION_TIME_WITH_TIME_ZONE_COLUMN_SIZE = 14;
+
+ private static final int DEFAULT_VARCHAR_LENGTH = 2_000_000_000;
+
+ private static final DateTimeFormatter DATE_FORMATTER = DateTimeFormatter.ofPattern("uuuu-MM-dd");
+
+ private static final PredicatePushdownController HSQLDB_CHARACTER_PUSHDOWN = (session, domain) -> {
+ if (domain.isNullableSingleValue()) {
+ return FULL_PUSHDOWN.apply(session, domain);
+ }
+
+ Domain simplifiedDomain = domain.simplify(getDomainCompactionThreshold(session));
+ if (!simplifiedDomain.getValues().isDiscreteSet()) {
+ // Push down inequality predicate
+ ValueSet complement = simplifiedDomain.getValues().complement();
+ if (complement.isDiscreteSet()) {
+ return FULL_PUSHDOWN.apply(session, simplifiedDomain);
+ }
+ // Domain#simplify can turn a discrete set into a range predicate
+ // Push down of range predicate for varchar/char types could lead to incorrect results
+ // when the remote database is case insensitive
+ return DISABLE_PUSHDOWN.apply(session, domain);
+ }
+ return FULL_PUSHDOWN.apply(session, simplifiedDomain);
+ };
+
+ @Inject
+ public HsqlDbClient(
+ BaseJdbcConfig config,
+ ConnectionFactory connectionFactory,
+ QueryBuilder queryBuilder,
+ IdentifierMapping identifierMapping,
+ RemoteQueryModifier remoteQueryModifier)
+ {
+ super("\"", connectionFactory, queryBuilder, config.getJdbcTypesMappedToVarchar(), identifierMapping, remoteQueryModifier, true);
+ }
+
+ @Override
+ public void setColumnComment(ConnectorSession session, JdbcTableHandle handle, JdbcColumnHandle column, Optional comment)
+ {
+ String sql = format(
+ "COMMENT ON COLUMN %s.%s IS %s",
+ quoted(handle.asPlainTable().getRemoteTableName()),
+ quoted(column.getColumnName()),
+ comment.map(BaseJdbcClient::varcharLiteral).orElse("NULL"));
+ execute(session, sql);
+ }
+
+ @Override
+ protected ResultSet getAllTableColumns(Connection connection, Optional remoteSchemaName)
+ throws SQLException
+ {
+ DatabaseMetaData metadata = connection.getMetaData();
+ return metadata.getColumns(
+ metadata.getConnection().getCatalog(),
+ escapeObjectNameForMetadataQuery(remoteSchemaName, metadata.getSearchStringEscape()).orElse(null),
+ null,
+ null);
+ }
+
+ @Override
+ public void setTableComment(ConnectorSession session, JdbcTableHandle handle, Optional comment)
+ {
+ execute(session, buildTableCommentSql(handle.asPlainTable().getRemoteTableName(), comment));
+ }
+
+ private String buildTableCommentSql(RemoteTableName remoteTableName, Optional comment)
+ {
+ return format(
+ "COMMENT ON TABLE %s IS %s",
+ quoted(remoteTableName),
+ comment.map(BaseJdbcClient::varcharLiteral).orElse("NULL"));
+ }
+
+ @Override
+ public Optional toColumnMapping(ConnectorSession session, Connection connection, JdbcTypeHandle typeHandle)
+ {
+ Optional mapping = getForcedMappingToVarchar(typeHandle);
+ if (mapping.isPresent()) {
+ return mapping;
+ }
+
+ switch (typeHandle.jdbcType()) {
+ 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.DOUBLE:
+ return Optional.of(doubleColumnMapping());
+ case Types.NUMERIC:
+ case Types.DECIMAL:
+ int decimalDigits = typeHandle.requiredDecimalDigits();
+ int decimalPrecision = typeHandle.requiredColumnSize();
+ if (getDecimalRounding(session) == ALLOW_OVERFLOW && decimalPrecision > Decimals.MAX_PRECISION) {
+ int scale = min(decimalDigits, getDecimalDefaultScale(session));
+ return Optional.of(decimalColumnMapping(createDecimalType(Decimals.MAX_PRECISION, scale), getDecimalRoundingMode(session)));
+ }
+ decimalPrecision = decimalPrecision + max(-decimalDigits, 0); // Map decimal(p, -s) (negative scale) to decimal(p+s, 0).
+ if (decimalPrecision > Decimals.MAX_PRECISION) {
+ break;
+ }
+ return Optional.of(decimalColumnMapping(createDecimalType(decimalPrecision, max(decimalDigits, 0))));
+
+ case Types.CHAR:
+ return Optional.of(charColumnMapping(typeHandle.requiredColumnSize(), typeHandle.caseSensitivity()));
+ case Types.VARCHAR:
+ case Types.LONGVARCHAR:
+ // varchar columns get created as varchar(default_length) in HsqlDB
+ if (typeHandle.requiredColumnSize() == DEFAULT_VARCHAR_LENGTH) {
+ return Optional.of(varcharColumnMapping(createUnboundedVarcharType(), typeHandle.caseSensitivity()));
+ }
+ return Optional.of(defaultVarcharColumnMapping(typeHandle.requiredColumnSize(), true));
+
+ case Types.BINARY:
+ case Types.VARBINARY:
+ case Types.LONGVARBINARY:
+ return Optional.of(varbinaryColumnMapping());
+
+ case Types.DATE:
+ return Optional.of(ColumnMapping.longMapping(
+ DATE,
+ dateReadFunctionUsingLocalDate(),
+ hsqlDbDateWriteFunction()));
+
+ case Types.TIME:
+ TimeType timeType = createTimeType(getTimePrecision(typeHandle.requiredColumnSize()));
+ return Optional.of(timeColumnMapping(timeType));
+ case Types.TIME_WITH_TIMEZONE:
+ int timePrecision = getTimeWithTimeZonePrecision(typeHandle.requiredColumnSize());
+ return Optional.of(timeWithTimeZoneColumnMapping(timePrecision));
+ }
+
+ if (getUnsupportedTypeHandling(session) == CONVERT_TO_VARCHAR) {
+ return mapToUnboundedVarchar(typeHandle);
+ }
+ return Optional.empty();
+ }
+
+ @Override
+ public WriteMapping toWriteMapping(ConnectorSession session, Type type)
+ {
+ 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("float", realWriteFunction());
+ }
+ if (type == DOUBLE) {
+ return WriteMapping.doubleMapping("double precision", doubleWriteFunction());
+ }
+ if (type instanceof DecimalType decimalType) {
+ 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 instanceof CharType charType) {
+ String dataType = format("char(%s)", charType.getLength());
+ return WriteMapping.sliceMapping(dataType, charWriteFunction());
+ }
+ if (type instanceof VarcharType varcharType) {
+ String dataType = varcharType.isUnbounded() ? "varchar(32768)" : format("varchar(%s)", varcharType.getBoundedLength());
+ return WriteMapping.sliceMapping(dataType, varcharWriteFunction());
+ }
+ if (type == VARBINARY) {
+ return WriteMapping.sliceMapping("varbinary", varbinaryWriteFunction());
+ }
+ if (type == DATE) {
+ return WriteMapping.longMapping("date", hsqlDbDateWriteFunction());
+ }
+
+ if (type instanceof TimeType timeType) {
+ if (timeType.getPrecision() <= MAX_SUPPORTED_DATE_TIME_PRECISION) {
+ return WriteMapping.longMapping(format("time(%s)", timeType.getPrecision()), timeWriteFunction(timeType.getPrecision()));
+ }
+ return WriteMapping.longMapping(format("time(%s)", MAX_SUPPORTED_DATE_TIME_PRECISION), timeWriteFunction(MAX_SUPPORTED_DATE_TIME_PRECISION));
+ }
+ if (type instanceof TimeWithTimeZoneType timeWithZoneType) {
+ if (timeWithZoneType.getPrecision() <= MAX_SUPPORTED_DATE_TIME_PRECISION) {
+ return WriteMapping.longMapping(format("time(%s) with time zone", timeWithZoneType.getPrecision()), timeWithTimeZoneWriteFunction());
+ }
+ return WriteMapping.longMapping(format("time(%s) with time zone", MAX_SUPPORTED_DATE_TIME_PRECISION), timeWithTimeZoneWriteFunction());
+ }
+
+ throw new TrinoException(NOT_SUPPORTED, "Unsupported column type: " + type.getDisplayName());
+ }
+
+ private static ColumnMapping charColumnMapping(int columnSize, Optional caseSensitivity)
+ {
+ if (columnSize > CharType.MAX_LENGTH) {
+ return varcharColumnMapping(columnSize, caseSensitivity);
+ }
+ return charColumnMapping(createCharType(columnSize), caseSensitivity);
+ }
+
+ private static LongWriteFunction hsqlDbDateWriteFunction()
+ {
+ return (statement, index, day) -> statement.setString(index, DATE_FORMATTER.format(LocalDate.ofEpochDay(day)));
+ }
+
+ private static ColumnMapping charColumnMapping(CharType charType, Optional caseSensitivity)
+ {
+ requireNonNull(charType, "charType is null");
+ PredicatePushdownController pushdownController = caseSensitivity.orElse(CASE_INSENSITIVE) == CASE_SENSITIVE
+ ? HSQLDB_CHARACTER_PUSHDOWN
+ : CASE_INSENSITIVE_CHARACTER_PUSHDOWN;
+ return ColumnMapping.sliceMapping(charType, charReadFunction(charType), charWriteFunction(), pushdownController);
+ }
+
+ private static ColumnMapping varcharColumnMapping(int columnSize, Optional caseSensitivity)
+ {
+ if (columnSize > VarcharType.MAX_LENGTH) {
+ return varcharColumnMapping(createUnboundedVarcharType(), caseSensitivity);
+ }
+ return varcharColumnMapping(createVarcharType(columnSize), caseSensitivity);
+ }
+
+ private static ColumnMapping varcharColumnMapping(VarcharType varcharType, Optional caseSensitivity)
+ {
+ PredicatePushdownController pushdownController = caseSensitivity.orElse(CASE_INSENSITIVE) == CASE_SENSITIVE
+ ? HSQLDB_CHARACTER_PUSHDOWN
+ : CASE_INSENSITIVE_CHARACTER_PUSHDOWN;
+ return ColumnMapping.sliceMapping(varcharType, varcharReadFunction(varcharType), varcharWriteFunction(), pushdownController);
+ }
+
+ @Override
+ protected void renameColumn(ConnectorSession session, Connection connection, RemoteTableName remoteTableName, String remoteColumnName, String newRemoteColumnName)
+ throws SQLException
+ {
+ try {
+ String sql = format(
+ "ALTER TABLE %s ALTER COLUMN %s RENAME TO %s",
+ quoted(remoteTableName.getCatalogName().orElse(null), remoteTableName.getSchemaName().orElse(null), remoteTableName.getTableName()),
+ quoted(remoteColumnName),
+ quoted(newRemoteColumnName));
+ execute(session, connection, sql);
+ }
+ catch (SQLSyntaxErrorException syntaxError) {
+ throw syntaxError;
+ }
+ }
+
+ @Override
+ public void setColumnType(ConnectorSession session, JdbcTableHandle handle, JdbcColumnHandle column, Type type)
+ {
+ throw new TrinoException(NOT_SUPPORTED, "This connector does not support setting column types");
+ }
+
+ @Override
+ protected void copyTableSchema(ConnectorSession session, Connection connection, String catalogName, String schemaName, String tableName, String newTableName, List columnNames)
+ {
+ // Copy all columns for enforcing NOT NULL option in the temp table
+ String tableCopyFormat = "CREATE TABLE %s AS (SELECT %s FROM %s) WITH NO DATA";
+ String sql = format(
+ tableCopyFormat,
+ 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);
+ }
+ }
+
+ @Override
+ protected List createTableSqls(RemoteTableName remoteTableName, List columns, ConnectorTableMetadata tableMetadata)
+ {
+ checkArgument(tableMetadata.getProperties().isEmpty(), "Unsupported table properties: %s", tableMetadata.getProperties());
+ ImmutableList.Builder createTableSqlsBuilder = ImmutableList.builder();
+ createTableSqlsBuilder.add(format("CREATE TABLE %s (%s)", quoted(remoteTableName), join(", ", columns)));
+ Optional tableComment = tableMetadata.getComment();
+ if (tableComment.isPresent()) {
+ createTableSqlsBuilder.add(buildTableCommentSql(remoteTableName, tableComment));
+ }
+ return createTableSqlsBuilder.build();
+ }
+
+ @Override
+ protected void renameTable(ConnectorSession session, String catalogName, String schemaName, String tableName, SchemaTableName newTable)
+ {
+ if (!schemaName.equalsIgnoreCase(newTable.getSchemaName())) {
+ throw new TrinoException(NOT_SUPPORTED, "This connector does not support renaming tables across schemas");
+ }
+ super.renameTable(session, catalogName, schemaName, tableName, newTable);
+ }
+
+ @Override
+ protected Optional> limitFunction()
+ {
+ return Optional.of((sql, limit) -> sql + " LIMIT " + limit);
+ }
+
+ @Override
+ public boolean isLimitGuaranteed(ConnectorSession session)
+ {
+ return true;
+ }
+
+ @Override
+ public boolean supportsTopN(ConnectorSession session, JdbcTableHandle handle, List sortOrder)
+ {
+ for (JdbcSortItem sortItem : sortOrder) {
+ Type sortItemType = sortItem.column().getColumnType();
+ if (sortItemType instanceof CharType || sortItemType instanceof VarcharType) {
+ // Remote database can be case insensitive.
+ return false;
+ }
+ }
+ return true;
+ }
+
+ @Override
+ protected Optional topNFunction()
+ {
+ return Optional.of((query, sortItems, limit) -> {
+ String orderBy = sortItems.stream()
+ .map(sortItem -> {
+ String ordering = sortItem.sortOrder().isAscending() ? "ASC" : "DESC";
+ String nullsHandling = sortItem.sortOrder().isNullsFirst() ? "NULLS FIRST" : "NULLS LAST";
+ return format("%s %s %s", quoted(sortItem.column().getColumnName()), ordering, nullsHandling);
+ })
+ .collect(joining(", "));
+ return format("%s ORDER BY %s LIMIT %d", query, orderBy, limit);
+ });
+ }
+
+ @Override
+ public boolean isTopNGuaranteed(ConnectorSession session)
+ {
+ return true;
+ }
+
+ @Override
+ protected boolean isSupportedJoinCondition(ConnectorSession session, JdbcJoinCondition joinCondition)
+ {
+ // Remote database can be case insensitive.
+ return Stream.of(joinCondition.getLeftColumn(), joinCondition.getRightColumn())
+ .map(JdbcColumnHandle::getColumnType)
+ .noneMatch(type -> type instanceof CharType || type instanceof VarcharType);
+ }
+
+ private static int getTimePrecision(int timeColumnSize)
+ {
+ return getTimePrecision(timeColumnSize, ZERO_PRECISION_TIME_COLUMN_SIZE);
+ }
+
+ private static int getTimeWithTimeZonePrecision(int timeColumnSize)
+ {
+ return getTimePrecision(timeColumnSize, ZERO_PRECISION_TIME_WITH_TIME_ZONE_COLUMN_SIZE);
+ }
+
+ private static int getTimePrecision(int timeColumnSize, int zeroPrecisionColumnSize)
+ {
+ if (timeColumnSize == zeroPrecisionColumnSize) {
+ return 0;
+ }
+ int timePrecision = timeColumnSize - zeroPrecisionColumnSize - 1;
+ verify(1 <= timePrecision && timePrecision <= MAX_SUPPORTED_DATE_TIME_PRECISION, "Unexpected time precision %s calculated from time column size %s", timePrecision, timeColumnSize);
+ return timePrecision;
+ }
+
+ private static ColumnMapping timeWithTimeZoneColumnMapping(int precision)
+ {
+ // HsqlDB supports timestamp with time zone precision up to nanoseconds
+ checkArgument(precision <= MAX_SUPPORTED_DATE_TIME_PRECISION, "unsupported precision value %s", precision);
+ return ColumnMapping.longMapping(
+ createTimeWithTimeZoneType(precision),
+ timeWithTimeZoneReadFunction(),
+ timeWithTimeZoneWriteFunction());
+ }
+
+ public static LongReadFunction timeWithTimeZoneReadFunction()
+ {
+ return (resultSet, columnIndex) -> {
+ OffsetTime time = resultSet.getObject(columnIndex, OffsetTime.class);
+ long nanosOfDay = time.toLocalTime().toNanoOfDay();
+ verify(nanosOfDay < NANOSECONDS_PER_DAY, "Invalid value of nanosOfDay: %s", nanosOfDay);
+ int offset = time.getOffset().getTotalSeconds() / 60;
+ return packTimeWithTimeZone(nanosOfDay, offset);
+ };
+ }
+
+ public static LongWriteFunction timeWithTimeZoneWriteFunction()
+ {
+ return LongWriteFunction.of(Types.TIME_WITH_TIMEZONE, (statement, index, packedTime) -> {
+ long nanosOfDay = unpackTimeNanos(packedTime);
+ verify(nanosOfDay < NANOSECONDS_PER_DAY, "Invalid value of nanosOfDay: %s", nanosOfDay);
+ ZoneOffset offset = ZoneOffset.ofTotalSeconds(unpackOffsetMinutes(packedTime) * 60);
+ statement.setObject(index, OffsetTime.of(LocalTime.ofNanoOfDay(nanosOfDay), offset));
+ });
+ }
+
+ @Override
+ public TableStatistics getTableStatistics(ConnectorSession session, JdbcTableHandle handle)
+ {
+ return TableStatistics.empty();
+ }
+}
diff --git a/plugin/trino-hsqldb/src/main/java/io/trino/plugin/hsqldb/HsqlDbClientModule.java b/plugin/trino-hsqldb/src/main/java/io/trino/plugin/hsqldb/HsqlDbClientModule.java
new file mode 100644
index 000000000000..fa39b67720b9
--- /dev/null
+++ b/plugin/trino-hsqldb/src/main/java/io/trino/plugin/hsqldb/HsqlDbClientModule.java
@@ -0,0 +1,73 @@
+/*
+ * 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.hsqldb;
+
+import com.google.inject.Binder;
+import com.google.inject.Module;
+import com.google.inject.Provides;
+import com.google.inject.Scopes;
+import com.google.inject.Singleton;
+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.JdbcMetadataConfig;
+import io.trino.plugin.jdbc.JdbcStatisticsConfig;
+import io.trino.plugin.jdbc.credential.CredentialProvider;
+import io.trino.plugin.jdbc.ptf.Query;
+import io.trino.spi.function.table.ConnectorTableFunction;
+import org.hsqldb.jdbcDriver;
+
+import java.util.Properties;
+
+import static com.google.inject.multibindings.Multibinder.newSetBinder;
+import static io.airlift.configuration.ConfigBinder.configBinder;
+
+public class HsqlDbClientModule
+ implements Module
+{
+ @Override
+ public void configure(Binder binder)
+ {
+ binder.bind(JdbcClient.class).annotatedWith(ForBaseJdbc.class).to(HsqlDbClient.class).in(Scopes.SINGLETON);
+ configBinder(binder).bindConfigDefaults(JdbcMetadataConfig.class, config -> config.setBulkListColumns(true));
+ configBinder(binder).bindConfig(HsqlDbJdbcConfig.class);
+ configBinder(binder).bindConfig(JdbcStatisticsConfig.class);
+ binder.install(new DecimalModule());
+ newSetBinder(binder, ConnectorTableFunction.class).addBinding().toProvider(Query.class).in(Scopes.SINGLETON);
+ }
+
+ @Provides
+ @Singleton
+ @ForBaseJdbc
+ public static ConnectionFactory createConnectionFactory(BaseJdbcConfig config, CredentialProvider credentialProvider, OpenTelemetry openTelemetry)
+ {
+ return DriverConnectionFactory.builder(new jdbcDriver(), config.getConnectionUrl(), credentialProvider)
+ .setConnectionProperties(getConnectionProperties())
+ .setOpenTelemetry(openTelemetry)
+ .build();
+ }
+
+ private static Properties getConnectionProperties()
+ {
+ Properties connectionProperties = new Properties();
+ // XXX: Sets the type of table created when the CREATE TABLE statement is executed.
+ // XXX: The default type is MEMORY.
+ connectionProperties.setProperty("hsqldb.default_table_type", "cached");
+ return connectionProperties;
+ }
+}
diff --git a/plugin/trino-hsqldb/src/main/java/io/trino/plugin/hsqldb/HsqlDbJdbcConfig.java b/plugin/trino-hsqldb/src/main/java/io/trino/plugin/hsqldb/HsqlDbJdbcConfig.java
new file mode 100644
index 000000000000..0210eae6ef13
--- /dev/null
+++ b/plugin/trino-hsqldb/src/main/java/io/trino/plugin/hsqldb/HsqlDbJdbcConfig.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.plugin.hsqldb;
+
+import io.trino.plugin.jdbc.BaseJdbcConfig;
+import jakarta.validation.constraints.AssertTrue;
+import org.hsqldb.jdbcDriver;
+
+public class HsqlDbJdbcConfig
+ extends BaseJdbcConfig
+{
+ @AssertTrue(message = "Invalid JDBC URL for HsqlDB connector")
+ public boolean isUrlValid()
+ {
+ jdbcDriver driver = new jdbcDriver();
+ return driver.acceptsURL(getConnectionUrl());
+ }
+
+ @AssertTrue(message = "Database must not be specified in JDBC URL for HsqlDB connector")
+ public boolean isUrlWithoutDatabase()
+ {
+ jdbcDriver driver = new jdbcDriver();
+ return driver.acceptsURL(getConnectionUrl());
+ }
+}
diff --git a/plugin/trino-hsqldb/src/main/java/io/trino/plugin/hsqldb/HsqlDbPlugin.java b/plugin/trino-hsqldb/src/main/java/io/trino/plugin/hsqldb/HsqlDbPlugin.java
new file mode 100644
index 000000000000..07fac6a2c368
--- /dev/null
+++ b/plugin/trino-hsqldb/src/main/java/io/trino/plugin/hsqldb/HsqlDbPlugin.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.hsqldb;
+
+import io.trino.plugin.jdbc.JdbcPlugin;
+
+public class HsqlDbPlugin
+ extends JdbcPlugin
+{
+ public HsqlDbPlugin()
+ {
+ super("hsqldb", HsqlDbClientModule::new);
+ }
+}
diff --git a/plugin/trino-hsqldb/src/main/java/io/trino/plugin/hsqldb/ImplementAvgBigint.java b/plugin/trino-hsqldb/src/main/java/io/trino/plugin/hsqldb/ImplementAvgBigint.java
new file mode 100644
index 000000000000..238aba49116e
--- /dev/null
+++ b/plugin/trino-hsqldb/src/main/java/io/trino/plugin/hsqldb/ImplementAvgBigint.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.plugin.hsqldb;
+
+import io.trino.plugin.jdbc.aggregation.BaseImplementAvgBigint;
+
+public class ImplementAvgBigint
+ extends BaseImplementAvgBigint
+{
+ @Override
+ protected String getRewriteFormatExpression()
+ {
+ return "avg(CAST(%s AS double))";
+ }
+}
diff --git a/plugin/trino-hsqldb/src/test/java/io/trino/plugin/hsqldb/HsqlDbCreateAndInsertDataSetup.java b/plugin/trino-hsqldb/src/test/java/io/trino/plugin/hsqldb/HsqlDbCreateAndInsertDataSetup.java
new file mode 100644
index 000000000000..39efb413cf59
--- /dev/null
+++ b/plugin/trino-hsqldb/src/test/java/io/trino/plugin/hsqldb/HsqlDbCreateAndInsertDataSetup.java
@@ -0,0 +1,55 @@
+/*
+ * 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.hsqldb;
+
+import io.trino.testing.datatype.ColumnSetup;
+import io.trino.testing.datatype.CreateAndInsertDataSetup;
+import io.trino.testing.sql.SqlExecutor;
+
+import java.util.List;
+import java.util.stream.IntStream;
+
+import static java.lang.String.format;
+import static java.util.stream.Collectors.joining;
+
+class HsqlDbCreateAndInsertDataSetup
+ extends CreateAndInsertDataSetup
+{
+ public HsqlDbCreateAndInsertDataSetup(SqlExecutor sqlExecutor, String tableNamePrefix)
+ {
+ super(sqlExecutor, tableNamePrefix);
+ }
+
+ @Override
+ protected String tableDefinition(List inputs)
+ {
+ if (inputs.stream().allMatch(input -> input.getDeclaredType().isPresent())) {
+ // When all types are explicitly specified, use ordinary CREATE TABLE
+ return IntStream.range(0, inputs.size())
+ .mapToObj(column -> format("col_%d %s", column, inputs.get(column).getDeclaredType().orElseThrow()))
+ .collect(joining(",\n", "(\n", ")"));
+ }
+
+ return IntStream.range(0, inputs.size())
+ .mapToObj(column -> {
+ ColumnSetup input = inputs.get(column);
+ if (input.getDeclaredType().isEmpty()) {
+ return format("%s AS col_%d", input.getInputLiteral(), column);
+ }
+
+ return format("CAST(%s AS %s) AS col_%d", input.getInputLiteral(), input.getDeclaredType().get(), column);
+ })
+ .collect(joining(",\n", "AS\n(SELECT\n", "\nWHERE 'with no' = 'data') WITH NO DATA"));
+ }
+}
diff --git a/plugin/trino-hsqldb/src/test/java/io/trino/plugin/hsqldb/HsqlDbQueryRunner.java b/plugin/trino-hsqldb/src/test/java/io/trino/plugin/hsqldb/HsqlDbQueryRunner.java
new file mode 100644
index 000000000000..7394f957208f
--- /dev/null
+++ b/plugin/trino-hsqldb/src/test/java/io/trino/plugin/hsqldb/HsqlDbQueryRunner.java
@@ -0,0 +1,178 @@
+/*
+ * 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.hsqldb;
+
+import com.google.common.collect.ImmutableList;
+import com.google.errorprone.annotations.CanIgnoreReturnValue;
+import io.airlift.log.Level;
+import io.airlift.log.Logger;
+import io.airlift.log.Logging;
+import io.trino.Session;
+import io.trino.metadata.QualifiedObjectName;
+import io.trino.plugin.tpch.TpchPlugin;
+import io.trino.testing.DistributedQueryRunner;
+import io.trino.testing.QueryRunner;
+import io.trino.tpch.TpchTable;
+import org.intellij.lang.annotations.Language;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static io.airlift.testing.Closeables.closeAllSuppress;
+import static io.airlift.units.Duration.nanosSince;
+import static io.trino.plugin.tpch.TpchMetadata.TINY_SCHEMA_NAME;
+import static io.trino.testing.TestingSession.testSessionBuilder;
+import static java.lang.String.format;
+import static java.util.Locale.ENGLISH;
+import static java.util.Objects.requireNonNull;
+import static org.assertj.core.api.Assertions.assertThat;
+
+public final class HsqlDbQueryRunner
+{
+ private HsqlDbQueryRunner() {}
+
+ static {
+ Logging logging = Logging.initialize();
+ logging.setLevel("org.hsqldb.jdbc", Level.OFF);
+ }
+
+ private static final Logger log = Logger.get(HsqlDbQueryRunner.class);
+
+ public static final String CATALOG = "hsqldb";
+ private static final String TEST_SCHEMA = "public";
+
+ public static Builder builder(TestingHsqlDbServer hsqlDbServer)
+ {
+ return new Builder()
+ .addConnectorProperty("connection-url", hsqlDbServer.getJdbcUrl())
+ .addConnectorProperty("connection-user", hsqlDbServer.getUsername())
+ .addConnectorProperty("connection-password", hsqlDbServer.getPassword());
+ }
+
+ public static final class Builder
+ extends DistributedQueryRunner.Builder
+ {
+ private final Map connectorProperties = new HashMap<>();
+ private List> initialTables = ImmutableList.of();
+
+ private Builder()
+ {
+ super(testSessionBuilder()
+ .setCatalog(CATALOG)
+ .setSchema(TEST_SCHEMA)
+ .build());
+ }
+
+ @CanIgnoreReturnValue
+ public Builder addConnectorProperties(Map connectorProperties)
+ {
+ this.connectorProperties.putAll(requireNonNull(connectorProperties, "connectorProperties is null"));
+ return this;
+ }
+
+ @CanIgnoreReturnValue
+ public Builder addConnectorProperty(String key, String value)
+ {
+ this.connectorProperties.put(key, value);
+ return this;
+ }
+
+ @CanIgnoreReturnValue
+ public Builder setInitialTables(Iterable> initialTables)
+ {
+ this.initialTables = ImmutableList.copyOf(requireNonNull(initialTables, "initialTables is null"));
+ return this;
+ }
+
+ @Override
+ public DistributedQueryRunner build()
+ throws Exception
+ {
+ DistributedQueryRunner queryRunner = super.build();
+ try {
+ queryRunner.installPlugin(new TpchPlugin());
+ queryRunner.createCatalog("tpch", "tpch");
+
+ queryRunner.installPlugin(new HsqlDbPlugin());
+ queryRunner.createCatalog(CATALOG, "hsqldb", connectorProperties);
+ log.info("%s catalog properties: %s", CATALOG, connectorProperties);
+
+ copyTpchTables(queryRunner, "tpch", TINY_SCHEMA_NAME, initialTables);
+
+ return queryRunner;
+ }
+ catch (Throwable e) {
+ closeAllSuppress(e, queryRunner);
+ throw e;
+ }
+ }
+ }
+
+ private static void copyTpchTables(
+ QueryRunner queryRunner,
+ String sourceCatalog,
+ String sourceSchema,
+ Iterable> tables)
+ {
+ copyTpchTables(queryRunner, sourceCatalog, sourceSchema, queryRunner.getDefaultSession(), tables);
+ }
+
+ private static void copyTpchTables(
+ QueryRunner queryRunner,
+ String sourceCatalog,
+ String sourceSchema,
+ Session session,
+ Iterable> tables)
+ {
+ for (TpchTable> table : tables) {
+ copyTable(queryRunner, sourceCatalog, sourceSchema, table.getTableName().toLowerCase(ENGLISH), session);
+ }
+ }
+
+ private static void copyTable(QueryRunner queryRunner, String sourceCatalog, String sourceSchema, String sourceTable, Session session)
+ {
+ QualifiedObjectName table = new QualifiedObjectName(sourceCatalog, sourceSchema, sourceTable);
+ copyTable(queryRunner, table, session);
+ }
+
+ private static void copyTable(QueryRunner queryRunner, QualifiedObjectName table, Session session)
+ {
+ long start = System.nanoTime();
+ @Language("SQL") String sql = format("CREATE TABLE IF NOT EXISTS %s AS (SELECT * FROM %s) WITH DATA", table.objectName(), table);
+ long rows = (Long) queryRunner.execute(session, sql).getMaterializedRows().get(0).getField(0);
+ log.debug("Imported %s rows from %s in %s", rows, table, nanosSince(start));
+
+ assertThat(queryRunner.execute(session, "SELECT count(*) FROM " + table.objectName()).getOnlyValue())
+ .as("Table is not loaded properly: %s", table.objectName())
+ .isEqualTo(queryRunner.execute(session, "SELECT count(*) FROM " + table).getOnlyValue());
+ }
+
+ public static void main(String[] args)
+ throws Exception
+ {
+ Logging logger = Logging.initialize();
+ logger.setLevel("io.trino.plugin.hsqldb", Level.DEBUG);
+ logger.setLevel("io.trino", Level.INFO);
+
+ QueryRunner queryRunner = builder(new TestingHsqlDbServer())
+ .addCoordinatorProperty("http-server.http.port", "8080")
+ .setInitialTables(TpchTable.getTables())
+ .build();
+
+ Logger log = Logger.get(HsqlDbQueryRunner.class);
+ log.info("======== SERVER STARTED ========");
+ log.info("\n====\n%s\n====", queryRunner.getCoordinator().getBaseUrl());
+ }
+}
diff --git a/plugin/trino-hsqldb/src/test/java/io/trino/plugin/hsqldb/TestHsqlDbCaseInsensitiveMapping.java b/plugin/trino-hsqldb/src/test/java/io/trino/plugin/hsqldb/TestHsqlDbCaseInsensitiveMapping.java
new file mode 100644
index 000000000000..52dbf2ad8158
--- /dev/null
+++ b/plugin/trino-hsqldb/src/test/java/io/trino/plugin/hsqldb/TestHsqlDbCaseInsensitiveMapping.java
@@ -0,0 +1,135 @@
+/*
+ * 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.hsqldb;
+
+import com.google.common.collect.ImmutableMap;
+import io.trino.plugin.jdbc.BaseCaseInsensitiveMappingTest;
+import io.trino.testing.QueryRunner;
+import io.trino.testing.sql.SqlExecutor;
+import org.intellij.lang.annotations.Language;
+import org.junit.jupiter.api.Test;
+
+import java.nio.file.Path;
+
+import static com.google.common.collect.ImmutableSet.toImmutableSet;
+import static io.trino.plugin.base.mapping.RuleBasedIdentifierMappingUtils.REFRESH_PERIOD_DURATION;
+import static io.trino.plugin.base.mapping.RuleBasedIdentifierMappingUtils.createRuleBasedIdentifierMappingFile;
+import static java.lang.String.format;
+import static java.util.Objects.requireNonNull;
+import static org.assertj.core.api.Assertions.assertThat;
+
+// With case-insensitive-name-matching enabled colliding schema/table names are considered as errors.
+// Some tests here create colliding names which can cause any other concurrent test to fail.
+final class TestHsqlDbCaseInsensitiveMapping
+ extends BaseCaseInsensitiveMappingTest
+{
+ private Path mappingFile;
+ private TestingHsqlDbServer server;
+
+ @Override
+ protected QueryRunner createQueryRunner()
+ throws Exception
+ {
+ mappingFile = createRuleBasedIdentifierMappingFile();
+ server = closeAfterClass(new TestingHsqlDbServer());
+ return HsqlDbQueryRunner.builder(server)
+ .addConnectorProperties(ImmutableMap.builder()
+ .put("case-insensitive-name-matching", "true")
+ .put("case-insensitive-name-matching.config-file", mappingFile.toFile().getAbsolutePath())
+ .put("case-insensitive-name-matching.config-file.refresh-period", REFRESH_PERIOD_DURATION.toString())
+ .buildOrThrow())
+ .build();
+ }
+
+ @Override
+ protected Path getMappingFile()
+ {
+ return requireNonNull(mappingFile, "mappingFile is null");
+ }
+
+ @Override
+ protected SqlExecutor onRemoteDatabase()
+ {
+ return server::execute;
+ }
+
+ @Override
+ protected AutoCloseable withSchema(String schemaName)
+ {
+ onRemoteDatabase().execute("CREATE SCHEMA " + quoted(schemaName));
+ return () -> onRemoteDatabase().execute("DROP SCHEMA " + quoted(schemaName) + " RESTRICT");
+ }
+
+ @Override
+ protected AutoCloseable withTable(String remoteSchemaName, String remoteTableName, String tableDefinition)
+ {
+ // FIXME: testTableNameClash() creates a table with a “public” schema which in this specific case should not be quoted
+ String quotedSchema = remoteSchemaName.equalsIgnoreCase("public") ? remoteSchemaName : quoted(remoteSchemaName);
+ String quotedName = quoted(remoteTableName);
+ onRemoteDatabase().execute(format("CREATE TABLE %s.%s %s", quotedSchema, quotedName, tableDefinition));
+ return () -> onRemoteDatabase().execute("DROP TABLE " + quotedSchema + "." + quotedName + " RESTRICT");
+ }
+
+ @Test
+ @Override
+ public void testNonLowerCaseTableName()
+ throws Exception
+ {
+ try (AutoCloseable ignore1 = withSchema("SomeSchema");
+ AutoCloseable ignore2 = withTable(
+ "SomeSchema",
+ "NonLowerCaseTable",
+ "(" +
+ quoted("lower_case_name") + " varchar(1), " +
+ quoted("Mixed_Case_Name") + " varchar(1), " +
+ quoted("UPPER_CASE_NAME") + " varchar(1))")) {
+ @Language("SQL") String sql;
+ if (optionalFromDual().isPresent()) {
+ sql = format("INSERT INTO %s.%s SELECT 'a', 'b', 'c' %s", quoted("SomeSchema"), quoted("NonLowerCaseTable"), optionalFromDual().get());
+ }
+ else {
+ sql = format("INSERT INTO %s.%s VALUES('a', 'b', 'c')", quoted("SomeSchema"), quoted("NonLowerCaseTable"));
+ }
+ onRemoteDatabase().execute(sql);
+ assertQuery(
+ "SELECT column_name FROM information_schema.columns WHERE table_schema = 'someschema' AND table_name = 'nonlowercasetable'",
+ "VALUES 'lower_case_name', 'mixed_case_name', 'upper_case_name'");
+ assertQuery(
+ "SELECT column_name FROM information_schema.columns WHERE table_name = 'nonlowercasetable'",
+ "VALUES 'lower_case_name', 'mixed_case_name', 'upper_case_name'");
+ assertThat(computeActual("SHOW COLUMNS FROM someschema.nonlowercasetable").getMaterializedRows().stream()
+ .map(row -> row.getField(0))
+ .collect(toImmutableSet()))
+ .containsOnly("lower_case_name", "mixed_case_name", "upper_case_name");
+
+ // Note: until https://github.com/prestodb/presto/issues/2863 is resolved, this is *the* way to access the tables.
+
+ assertQuery("SELECT lower_case_name FROM someschema.nonlowercasetable", "VALUES 'a'");
+ assertQuery("SELECT mixed_case_name FROM someschema.nonlowercasetable", "VALUES 'b'");
+ assertQuery("SELECT upper_case_name FROM someschema.nonlowercasetable", "VALUES 'c'");
+ assertQuery("SELECT upper_case_name FROM SomeSchema.NonLowerCaseTable", "VALUES 'c'");
+ assertQuery("SELECT upper_case_name FROM \"SomeSchema\".\"NonLowerCaseTable\"", "VALUES 'c'");
+
+ assertUpdate("INSERT INTO someschema.nonlowercasetable (lower_case_name) VALUES ('l')", 1);
+ assertUpdate("INSERT INTO someschema.nonlowercasetable (mixed_case_name) VALUES ('m')", 1);
+ assertUpdate("INSERT INTO someschema.nonlowercasetable (upper_case_name) VALUES ('u')", 1);
+ assertQuery(
+ "SELECT * FROM someschema.nonlowercasetable",
+ "VALUES ('a', 'b', 'c')," +
+ "('l', NULL, NULL)," +
+ "(NULL, 'm', NULL)," +
+ "(NULL, NULL, 'u')");
+ }
+ }
+}
diff --git a/plugin/trino-hsqldb/src/test/java/io/trino/plugin/hsqldb/TestHsqlDbConnectorTest.java b/plugin/trino-hsqldb/src/test/java/io/trino/plugin/hsqldb/TestHsqlDbConnectorTest.java
new file mode 100644
index 000000000000..06299cc1df4b
--- /dev/null
+++ b/plugin/trino-hsqldb/src/test/java/io/trino/plugin/hsqldb/TestHsqlDbConnectorTest.java
@@ -0,0 +1,511 @@
+/*
+ * 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.hsqldb;
+
+import com.google.common.collect.ImmutableList;
+import io.trino.Session;
+import io.trino.plugin.jdbc.BaseJdbcConnectorTest;
+import io.trino.testing.QueryRunner;
+import io.trino.testing.TestingConnectorBehavior;
+import io.trino.testing.sql.SqlExecutor;
+import io.trino.testing.sql.TestTable;
+import io.trino.testing.sql.TestView;
+import org.intellij.lang.annotations.Language;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.regex.Pattern;
+
+import static com.google.common.base.Strings.nullToEmpty;
+import static io.trino.spi.connector.ConnectorMetadata.MODIFYING_ROWS_MESSAGE;
+import static io.trino.testing.QueryAssertions.getTrinoExceptionCause;
+import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE;
+import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_VIEW;
+import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_NATIVE_QUERY;
+import static io.trino.testing.TestingNames.randomNameSuffix;
+import static java.lang.String.format;
+import static java.util.stream.Collectors.toList;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+final class TestHsqlDbConnectorTest
+ extends BaseJdbcConnectorTest
+{
+ private TestingHsqlDbServer server;
+
+ @Override
+ protected QueryRunner createQueryRunner()
+ throws Exception
+ {
+ server = closeAfterClass(new TestingHsqlDbServer());
+ return HsqlDbQueryRunner.builder(server)
+ .setInitialTables(REQUIRED_TPCH_TABLES)
+ .build();
+ }
+
+ @Override
+ protected SqlExecutor onRemoteDatabase()
+ {
+ return server::execute;
+ }
+
+ @Override
+ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
+ {
+ return switch (connectorBehavior) {
+ case SUPPORTS_ADD_COLUMN,
+ SUPPORTS_COMMENT_ON_COLUMN,
+ SUPPORTS_COMMENT_ON_TABLE,
+ SUPPORTS_CREATE_SCHEMA,
+ SUPPORTS_CREATE_TABLE,
+ SUPPORTS_CREATE_TABLE_WITH_DATA,
+ SUPPORTS_DELETE,
+ SUPPORTS_DROP_NOT_NULL_CONSTRAINT,
+ SUPPORTS_DROP_SCHEMA_CASCADE,
+ SUPPORTS_INSERT,
+ SUPPORTS_LIMIT_PUSHDOWN,
+ SUPPORTS_RENAME_COLUMN,
+ SUPPORTS_RENAME_SCHEMA,
+ SUPPORTS_RENAME_TABLE,
+ SUPPORTS_TOPN_PUSHDOWN -> true;
+ case SUPPORTS_ADD_COLUMN_WITH_COMMENT,
+ SUPPORTS_AGGREGATION_PUSHDOWN,
+ SUPPORTS_AGGREGATION_PUSHDOWN_CORRELATION,
+ SUPPORTS_AGGREGATION_PUSHDOWN_COUNT_DISTINCT,
+ SUPPORTS_AGGREGATION_PUSHDOWN_COVARIANCE,
+ SUPPORTS_AGGREGATION_PUSHDOWN_REGRESSION,
+ SUPPORTS_ARRAY,
+ SUPPORTS_COMMENT_ON_VIEW,
+ SUPPORTS_COMMENT_ON_VIEW_COLUMN,
+ SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT,
+ SUPPORTS_JOIN_PUSHDOWN,
+ SUPPORTS_JOIN_PUSHDOWN_WITH_DISTINCT_FROM,
+ SUPPORTS_MAP_TYPE,
+ SUPPORTS_NATIVE_QUERY,
+ SUPPORTS_NEGATIVE_DATE,
+ SUPPORTS_PREDICATE_ARITHMETIC_EXPRESSION_PUSHDOWN,
+ SUPPORTS_RENAME_TABLE_ACROSS_SCHEMAS,
+ SUPPORTS_ROW_TYPE,
+ SUPPORTS_SET_COLUMN_TYPE,
+ SUPPORTS_TOPN_PUSHDOWN_WITH_VARCHAR -> false;
+ default -> super.hasBehavior(connectorBehavior);
+ };
+ }
+
+ @Override
+ protected TestTable createTableWithDefaultColumns()
+ {
+ // FIXME: HsqlDB requires declaring the DEFAULT value before the NOT NULL constraint
+ return new TestTable(
+ onRemoteDatabase(),
+ "table",
+ "(col_required BIGINT NOT NULL," +
+ "col_nullable BIGINT," +
+ "col_default BIGINT DEFAULT 43," +
+ "col_nonnull_default BIGINT DEFAULT 42 NOT NULL," +
+ "col_required2 BIGINT NOT NULL)");
+ }
+
+ @Test
+ @Override
+ public void testCreateTableAsSelect()
+ {
+ // XXX: HsqlDB requires SELECT query to be enclosed in parentheses
+ String tableName = "test_ctas" + randomNameSuffix();
+ assertUpdate("CREATE TABLE IF NOT EXISTS " + tableName + " AS (SELECT name, regionkey FROM nation) WITH DATA", "SELECT count(*) FROM nation");
+ assertTableColumnNames(tableName, "name", "regionkey");
+ assertThat(getTableComment(getSession().getCatalog().orElseThrow(), getSession().getSchema().orElseThrow(), tableName)).isNull();
+ assertUpdate("DROP TABLE " + tableName);
+
+ // Some connectors support CREATE TABLE AS but not the ordinary CREATE TABLE. Let's test CTAS IF NOT EXISTS with a table that is guaranteed to exist.
+ assertUpdate("CREATE TABLE IF NOT EXISTS nation AS (SELECT nationkey, regionkey FROM nation) WITH DATA", 0);
+ assertTableColumnNames("nation", "nationkey", "name", "regionkey", "comment");
+
+ assertCreateTableAsSelect(
+ "SELECT nationkey, name, regionkey FROM nation",
+ "SELECT count(*) FROM nation");
+
+ assertCreateTableAsSelect(
+ "SELECT mktsegment, sum(acctbal) x FROM customer GROUP BY mktsegment",
+ "SELECT count(DISTINCT mktsegment) FROM customer");
+
+ assertCreateTableAsSelect(
+ "SELECT count(*) x FROM nation JOIN region ON nation.regionkey = region.regionkey",
+ "SELECT 1");
+
+ assertCreateTableAsSelect(
+ "SELECT * FROM nation",
+ "SELECT * FROM nation",
+ "SELECT count(*) FROM nation");
+
+ // Tests for CREATE TABLE with UNION ALL: exercises PushTableWriteThroughUnion optimizer
+
+ assertCreateTableAsSelect(
+ "SELECT name, nationkey, regionkey FROM nation WHERE nationkey % 2 = 0 UNION ALL " +
+ "SELECT name, nationkey, regionkey FROM nation WHERE nationkey % 2 = 1",
+ "SELECT name, nationkey, regionkey FROM nation",
+ "SELECT count(*) FROM nation");
+
+ assertCreateTableAsSelect(
+ Session.builder(getSession()).setSystemProperty("redistribute_writes", "true").build(),
+ "SELECT CAST(nationkey AS BIGINT) nationkey, regionkey FROM nation UNION ALL " +
+ "SELECT 1234567890, 123",
+ "SELECT nationkey, regionkey FROM nation UNION ALL " +
+ "SELECT 1234567890, 123",
+ "SELECT count(*) + 1 FROM nation");
+
+ assertCreateTableAsSelect(
+ Session.builder(getSession()).setSystemProperty("redistribute_writes", "false").build(),
+ "SELECT CAST(nationkey AS BIGINT) nationkey, regionkey FROM nation UNION ALL " +
+ "SELECT 1234567890, 123",
+ "SELECT nationkey, regionkey FROM nation UNION ALL " +
+ "SELECT 1234567890, 123",
+ "SELECT count(*) + 1 FROM nation");
+
+ tableName = "test_ctas" + randomNameSuffix();
+ assertExplainAnalyze("EXPLAIN ANALYZE CREATE TABLE " + tableName + " AS (SELECT name FROM nation) WITH DATA");
+ assertQuery("SELECT * from " + tableName, "SELECT name FROM nation");
+ assertUpdate("DROP TABLE " + tableName);
+ }
+
+ @Override
+ protected void assertCreateTableAsSelect(Session session, @Language("SQL") String query, @Language("SQL") String expectedQuery, @Language("SQL") String rowCountQuery)
+ {
+ // XXX: HsqlDB requires SELECT query to be enclosed in parentheses
+ String table = "test_ctas_" + randomNameSuffix();
+ assertUpdate(session, "CREATE TABLE " + table + " AS (" + query + ") WITH DATA", rowCountQuery);
+ assertQuery(session, "SELECT * FROM " + table, expectedQuery);
+ assertUpdate(session, "DROP TABLE " + table);
+
+ assertThat(getQueryRunner().tableExists(session, table)).isFalse();
+ }
+
+ @Override
+ protected boolean isColumnNameRejected(Exception exception, String columnName, boolean delimited)
+ {
+ return nullToEmpty(exception.getMessage()).contains("type not found or user lacks privilege:");
+ }
+
+ @Test
+ void testReadFromView()
+ {
+ try (TestView view = new TestView(onRemoteDatabase(), "test_view", "SELECT * FROM orders")) {
+ assertThat(getQueryRunner().tableExists(getSession(), view.getName())).isTrue();
+ assertQuery("SELECT orderkey FROM " + view.getName(), "SELECT orderkey FROM orders");
+ }
+ }
+
+ @Override
+ protected void verifyAddNotNullColumnToNonEmptyTableFailurePermissible(Throwable e)
+ {
+ assertThat(e).hasMessageStartingWith("default expression needed in statement ");
+ }
+
+ @Test
+ @Override
+ public void testCommentTable()
+ {
+ String catalogName = getSession().getCatalog().orElseThrow();
+ String schemaName = getSession().getSchema().orElseThrow();
+ try (TestTable table = new TestTable(getQueryRunner()::execute, "test_comment_", "(a integer)")) {
+ // comment initially not set
+ assertThat(getTableComment(catalogName, schemaName, table.getName())).isEqualTo(null);
+
+ // comment set
+ assertUpdate("COMMENT ON TABLE " + table.getName() + " IS 'new comment'");
+ assertThat((String) computeScalar("SHOW CREATE TABLE " + table.getName())).contains("COMMENT 'new comment'");
+ assertThat(getTableComment(catalogName, schemaName, table.getName())).isEqualTo("new comment");
+ assertThat(query(
+ "SELECT table_name, comment FROM system.metadata.table_comments " +
+ "WHERE catalog_name = '" + catalogName + "' AND schema_name = '" + schemaName + "'")) // without table_name filter
+ .skippingTypesCheck()
+ .containsAll("VALUES ('" + table.getName() + "', 'new comment')");
+
+ // comment deleted
+ assertUpdate("COMMENT ON TABLE " + table.getName() + " IS ''");
+ assertThat(getTableComment(catalogName, schemaName, table.getName())).isEqualTo("");
+
+ // comment set to non-empty value before verifying setting empty comment
+ assertUpdate("COMMENT ON TABLE " + table.getName() + " IS 'updated comment'");
+ assertThat(getTableComment(catalogName, schemaName, table.getName())).isEqualTo("updated comment");
+ }
+
+ String tableName = "test_comment_" + randomNameSuffix();
+ try {
+ // comment set when creating a table
+ assertUpdate("CREATE TABLE " + tableName + "(key integer) COMMENT 'new table comment'");
+ assertThat(getTableComment(catalogName, schemaName, tableName)).isEqualTo("new table comment");
+ }
+ finally {
+ assertUpdate("DROP TABLE IF EXISTS " + tableName);
+ }
+ }
+
+ @Test
+ @Override
+ public void testCommentColumn()
+ {
+ try (TestTable table = new TestTable(getQueryRunner()::execute, "test_comment_column_", "(a integer)")) {
+ // comment set
+ assertUpdate("COMMENT ON COLUMN public." + table.getName() + ".a IS 'new comment'");
+ assertThat((String) computeScalar("SHOW CREATE TABLE " + table.getName())).contains("COMMENT 'new comment'");
+ assertThat(getColumnComment(table.getName(), "a")).isEqualTo("new comment");
+
+ // comment deleted
+ assertUpdate("COMMENT ON COLUMN " + table.getName() + ".a IS ''");
+ assertThat(getColumnComment(table.getName(), "a")).isEqualTo(null);
+
+ // comment set to non-empty value before verifying setting empty comment
+ assertUpdate("COMMENT ON COLUMN " + table.getName() + ".a IS 'updated comment'");
+ assertThat(getColumnComment(table.getName(), "a")).isEqualTo("updated comment");
+ }
+ }
+
+ @Override
+ @Language("RegExp")
+ protected String errorMessageForInsertNegativeDate(String date)
+ {
+ return ".*";
+ }
+
+ @Override
+ protected void verifyConcurrentAddColumnFailurePermissible(Exception e)
+ {
+ assertThat(e).hasMessageContaining("was deadlocked on lock resources");
+ }
+
+ @Override
+ @Language("RegExp")
+ protected String errorMessageForCreateTableAsSelectNegativeDate(String date)
+ {
+ return ".*";
+ }
+
+ @Override
+ @Language("RegExp")
+ protected String errorMessageForInsertIntoNotNullColumn(String columnName)
+ {
+ return ".*";
+ }
+
+ @Override
+ protected OptionalInt maxSchemaNameLength()
+ {
+ return OptionalInt.of(128);
+ }
+
+ @Override
+ protected void verifySchemaNameLengthFailurePermissible(Throwable e)
+ {
+ assertThat(e).hasMessageContaining("user lacks privilege or object not found:");
+ }
+
+ @Override
+ protected OptionalInt maxTableNameLength()
+ {
+ return OptionalInt.of(128);
+ }
+
+ @Override
+ protected void verifyTableNameLengthFailurePermissible(Throwable e)
+ {
+ assertThat(e).hasMessageContaining("user lacks privilege or object not found:");
+ }
+
+ @Override
+ protected OptionalInt maxColumnNameLength()
+ {
+ return OptionalInt.of(128);
+ }
+
+ @Override
+ protected void verifyColumnNameLengthFailurePermissible(Throwable e)
+ {
+ assertThat(e).hasMessageContaining("user lacks privilege or object not found:");
+ }
+
+ @Test
+ @Override
+ public void testCreateViewSchemaNotFound()
+ {
+ skipTestUnless(hasBehavior(SUPPORTS_CREATE_VIEW));
+
+ String schemaName = "test_schema_" + randomNameSuffix();
+ String viewName = "test_view_create_no_schema_" + randomNameSuffix();
+ try {
+ assertQueryFails(
+ format("CREATE VIEW %s.%s AS SELECT 1 AS c1", schemaName, viewName),
+ format("invalid schema name: %s", schemaName));
+ assertQueryFails(
+ format("CREATE OR REPLACE VIEW %s.%s AS SELECT 1 AS c1", schemaName, viewName),
+ format("invalid schema name: %s", schemaName));
+ }
+ finally {
+ assertUpdate(format("DROP VIEW IF EXISTS %s.%s", schemaName, viewName));
+ }
+ }
+
+ @Test
+ @Override
+ public void testDeleteWithLike()
+ {
+ assertThatThrownBy(super::testDeleteWithLike)
+ .hasStackTraceContaining("TrinoException: " + MODIFYING_ROWS_MESSAGE);
+ }
+
+ @Test
+ @Override
+ public void testDateYearOfEraPredicate()
+ {
+ // HsqlDB throws an exception instead of an empty result when the value is out of range
+ assertQuery("SELECT orderdate FROM orders WHERE orderdate = DATE '1997-09-14'", "VALUES DATE '1997-09-14'");
+ assertQueryFails(
+ "SELECT * FROM orders WHERE orderdate = DATE '-1996-09-14'",
+ "data exception: invalid datetime format");
+ }
+
+ @Test
+ @Override
+ public void testDataMappingSmokeTest()
+ {
+ skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE));
+
+ for (DataMappingTestSetup dataMappingTestSetup : testDataMappingSmokeTestDataHsqlDB()) {
+ testDataMapping(dataMappingTestSetup);
+ }
+ }
+
+ private final List testDataMappingSmokeTestDataHsqlDB()
+ {
+ return testDataMappingSmokeTestData().stream()
+ .map(this::filterDataMappingSmokeTestData)
+ .flatMap(Optional::stream)
+ .collect(toList());
+ }
+
+ private void testDataMapping(DataMappingTestSetup dataMappingTestSetup)
+ {
+ String trinoTypeName = dataMappingTestSetup.getTrinoTypeName();
+ String sampleValueLiteral = dataMappingTestSetup.getSampleValueLiteral();
+ String highValueLiteral = dataMappingTestSetup.getHighValueLiteral();
+
+ String tableName = dataMappingTableName(trinoTypeName);
+
+ Runnable setup = () -> {
+ // TODO test with both CTAS *and* CREATE TABLE + INSERT, since they use different connector API methods.
+ String createTable = "" +
+ "CREATE TABLE " + tableName + " AS " +
+ "SELECT CAST(row_id AS varchar(50)) row_id, CAST(value AS " + trinoTypeName + ") value, CAST(value AS " + trinoTypeName + ") another_column " +
+ "FROM (VALUES " +
+ " ('null value', NULL), " +
+ " ('sample value', " + sampleValueLiteral + "), " +
+ " ('high value', " + highValueLiteral + ")) " +
+ " t(row_id, value)";
+ assertUpdate(createTable, 3);
+ };
+ if (dataMappingTestSetup.isUnsupportedType()) {
+ assertThatThrownBy(setup::run)
+ .satisfies(exception -> verifyUnsupportedTypeException(exception, trinoTypeName));
+ return;
+ }
+ setup.run();
+
+ // without pushdown, i.e. test read data mapping
+ assertQuery("SELECT row_id FROM " + tableName + " WHERE rand() = 42 OR value IS NULL", "VALUES 'null value'");
+ assertQuery("SELECT row_id FROM " + tableName + " WHERE rand() = 42 OR value IS NOT NULL", "VALUES 'sample value', 'high value'");
+ assertQuery("SELECT row_id FROM " + tableName + " WHERE rand() = 42 OR value = " + sampleValueLiteral, "VALUES 'sample value'");
+ assertQuery("SELECT row_id FROM " + tableName + " WHERE rand() = 42 OR value = " + highValueLiteral, "VALUES 'high value'");
+
+ assertQuery("SELECT row_id FROM " + tableName + " WHERE value IS NULL", "VALUES 'null value'");
+ assertQuery("SELECT row_id FROM " + tableName + " WHERE value IS NOT NULL", "VALUES 'sample value', 'high value'");
+ assertQuery("SELECT row_id FROM " + tableName + " WHERE value = " + sampleValueLiteral, "VALUES 'sample value'");
+ assertQuery("SELECT row_id FROM " + tableName + " WHERE value != " + sampleValueLiteral, "VALUES 'high value'");
+ assertQuery("SELECT row_id FROM " + tableName + " WHERE value <= " + sampleValueLiteral, "VALUES 'sample value'");
+ assertQuery("SELECT row_id FROM " + tableName + " WHERE value > " + sampleValueLiteral, "VALUES 'high value'");
+ assertQuery("SELECT row_id FROM " + tableName + " WHERE value <= " + highValueLiteral, "VALUES 'sample value', 'high value'");
+
+ assertQuery("SELECT row_id FROM " + tableName + " WHERE value IS NULL OR value = " + sampleValueLiteral, "VALUES 'null value', 'sample value'");
+ assertQuery("SELECT row_id FROM " + tableName + " WHERE value IS NULL OR value != " + sampleValueLiteral, "VALUES 'null value', 'high value'");
+ assertQuery("SELECT row_id FROM " + tableName + " WHERE value IS NULL OR value <= " + sampleValueLiteral, "VALUES 'null value', 'sample value'");
+ assertQuery("SELECT row_id FROM " + tableName + " WHERE value IS NULL OR value > " + sampleValueLiteral, "VALUES 'null value', 'high value'");
+ assertQuery("SELECT row_id FROM " + tableName + " WHERE value IS NULL OR value <= " + highValueLiteral, "VALUES 'null value', 'sample value', 'high value'");
+
+ // complex condition, one that cannot be represented with a TupleDomain
+ assertQuery("SELECT row_id FROM " + tableName + " WHERE value = " + sampleValueLiteral + " OR another_column = " + sampleValueLiteral, "VALUES 'sample value'");
+
+ assertUpdate("DROP TABLE " + tableName);
+ }
+
+ private void verifyUnsupportedTypeException(Throwable exception, String trinoTypeName)
+ {
+ String typeNameBase = trinoTypeName.replaceFirst("\\(.*", "");
+ String expectedMessagePart = format("(%1$s.*not (yet )?supported)|((?i)unsupported.*%1$s)|((?i)not supported.*%1$s)", Pattern.quote(typeNameBase));
+ assertThat(exception)
+ .hasMessageFindingMatch(expectedMessagePart)
+ .satisfies(e -> assertThat(getTrinoExceptionCause(e)).hasMessageFindingMatch(expectedMessagePart));
+ }
+
+ private List testDataMappingSmokeTestData()
+ {
+ return ImmutableList.builder()
+ .add(new DataMappingTestSetup("boolean", "false", "true"))
+ .add(new DataMappingTestSetup("tinyint", "37", "127"))
+ .add(new DataMappingTestSetup("smallint", "32123", "32767"))
+ .add(new DataMappingTestSetup("integer", "1274942432", "2147483647"))
+ .add(new DataMappingTestSetup("bigint", "312739231274942432", "9223372036854775807"))
+ .add(new DataMappingTestSetup("real", "REAL '567.123'", "REAL '999999.999'"))
+ .add(new DataMappingTestSetup("double", "DOUBLE '1234567890123.123'", "DOUBLE '9999999999999.999'"))
+ .add(new DataMappingTestSetup("decimal(5,3)", "12.345", "99.999"))
+ .add(new DataMappingTestSetup("decimal(15,3)", "123456789012.345", "999999999999.99"))
+ .add(new DataMappingTestSetup("date", "DATE '0001-01-01'", "DATE '1582-10-04'")) // before julian->gregorian switch
+ // FIXME: Should we support dates during the transition from the Julian to the Gregorian calendar?
+ //.add(new DataMappingTestSetup("date", "DATE '1582-10-05'", "DATE '1582-10-14'")) // during julian->gregorian switch
+ .add(new DataMappingTestSetup("date", "DATE '2020-02-12'", "DATE '9999-12-31'"))
+ .add(new DataMappingTestSetup("time", "TIME '15:03:00'", "TIME '23:59:59.999'"))
+ .add(new DataMappingTestSetup("time(6)", "TIME '15:03:00'", "TIME '23:59:59.999999'"))
+ .add(new DataMappingTestSetup("char(3)", "'ab'", "'zzz'"))
+ .add(new DataMappingTestSetup("varchar(3)", "'de'", "'zzz'"))
+ .add(new DataMappingTestSetup("varchar", "'łąka for the win'", "'ŻŻŻŻŻŻŻŻŻŻ'"))
+ .add(new DataMappingTestSetup("varchar", "'a \\backslash'", "'a a'")) // `a` sorts after `\`; \b may be interpreted as an escape sequence
+ .add(new DataMappingTestSetup("varchar", "'end backslash \\'", "'end backslash a'")) // `a` sorts after `\`; final \ before end quote may confuse a parser
+ .add(new DataMappingTestSetup("varchar", "U&'a \\000a newline'", "'a a'")) // `a` sorts after `\n`; newlines can require special handling in a remote system's language
+ .add(new DataMappingTestSetup("varbinary", "X'12ab3f'", "X'ffffffffffffffffffff'"))
+ .build();
+ }
+
+ @Test
+ @Override
+ public void verifySupportsNativeQueryDeclaration()
+ {
+ if (hasBehavior(SUPPORTS_NATIVE_QUERY)) {
+ // Covered by testNativeQuerySelectFromNation
+ return;
+ }
+ // FIXME: cannot support native queries and I can't run this test
+ // assertQueryFails(
+ // format("SELECT * FROM TABLE(system.query(query => 'SELECT name FROM %s.nation WHERE nationkey = 0'))", getSession().getSchema().orElseThrow()),
+ // "line 1:21: Table function 'system.query' not registered");
+ }
+
+ @Test
+ @Disabled
+ @Override
+ public void testAddColumnConcurrently()
+ {
+ // TODO: Enable this test after finding the failure cause
+ }
+}
diff --git a/plugin/trino-hsqldb/src/test/java/io/trino/plugin/hsqldb/TestHsqlDbJdbcConfig.java b/plugin/trino-hsqldb/src/test/java/io/trino/plugin/hsqldb/TestHsqlDbJdbcConfig.java
new file mode 100644
index 000000000000..300c278f34ea
--- /dev/null
+++ b/plugin/trino-hsqldb/src/test/java/io/trino/plugin/hsqldb/TestHsqlDbJdbcConfig.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.plugin.hsqldb;
+
+import org.junit.jupiter.api.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+final class TestHsqlDbJdbcConfig
+{
+ @Test
+ void testIsUrlValid()
+ {
+ assertThat(isUrlValid("jdbc:hsqldb:mem")).isTrue();
+ assertThat(isUrlValid("jdbc:hsqldb:hsql://localhost/")).isTrue();
+ assertThat(isUrlValid("jdbc:nothsqldb:mem")).isFalse();
+ assertThat(isUrlValid("jdbc:nothsqldb:hsql://localhost/")).isFalse();
+ }
+
+ private static boolean isUrlValid(String url)
+ {
+ HsqlDbJdbcConfig config = new HsqlDbJdbcConfig();
+ config.setConnectionUrl(url);
+ return config.isUrlValid();
+ }
+}
diff --git a/plugin/trino-hsqldb/src/test/java/io/trino/plugin/hsqldb/TestHsqlDbPlugin.java b/plugin/trino-hsqldb/src/test/java/io/trino/plugin/hsqldb/TestHsqlDbPlugin.java
new file mode 100644
index 000000000000..7dd945a65c12
--- /dev/null
+++ b/plugin/trino-hsqldb/src/test/java/io/trino/plugin/hsqldb/TestHsqlDbPlugin.java
@@ -0,0 +1,47 @@
+/*
+ * 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.hsqldb;
+
+import com.google.common.collect.ImmutableMap;
+import io.trino.spi.Plugin;
+import io.trino.spi.connector.ConnectorFactory;
+import io.trino.testing.TestingConnectorContext;
+import org.junit.jupiter.api.Test;
+
+import static com.google.common.collect.Iterables.getOnlyElement;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+final class TestHsqlDbPlugin
+{
+ @Test
+ void testCreateConnector()
+ {
+ Plugin plugin = new HsqlDbPlugin();
+ ConnectorFactory factory = getOnlyElement(plugin.getConnectorFactories());
+ factory.create(
+ "test",
+ ImmutableMap.of(
+ "connection-url", "jdbc:hsqldb:hsql://localhost/",
+ "bootstrap.quiet", "true"),
+ new TestingConnectorContext()).shutdown();
+
+ assertThatThrownBy(() -> factory.create(
+ "test",
+ ImmutableMap.of(
+ "connection-url", "test",
+ "bootstrap.quiet", "true"),
+ new TestingConnectorContext()))
+ .hasMessageContaining("Invalid JDBC URL for HsqlDB connector");
+ }
+}
diff --git a/plugin/trino-hsqldb/src/test/java/io/trino/plugin/hsqldb/TestHsqlDbTypeMapping.java b/plugin/trino-hsqldb/src/test/java/io/trino/plugin/hsqldb/TestHsqlDbTypeMapping.java
new file mode 100644
index 000000000000..4a241f0bb907
--- /dev/null
+++ b/plugin/trino-hsqldb/src/test/java/io/trino/plugin/hsqldb/TestHsqlDbTypeMapping.java
@@ -0,0 +1,794 @@
+/*
+ * 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.hsqldb;
+
+import io.trino.Session;
+import io.trino.spi.type.TimeZoneKey;
+import io.trino.testing.AbstractTestQueryFramework;
+import io.trino.testing.QueryRunner;
+import io.trino.testing.TestingSession;
+import io.trino.testing.datatype.CreateAndInsertDataSetup;
+import io.trino.testing.datatype.CreateAsSelectDataSetup;
+import io.trino.testing.datatype.DataSetup;
+import io.trino.testing.datatype.SqlDataTypeTest;
+import io.trino.testing.sql.TestTable;
+import io.trino.testing.sql.TrinoSqlExecutor;
+import org.intellij.lang.annotations.Language;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInstance;
+import org.junit.jupiter.api.parallel.Execution;
+
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.util.function.Function;
+
+import static com.google.common.base.Preconditions.checkState;
+import static com.google.common.base.Verify.verify;
+import static io.trino.spi.type.BigintType.BIGINT;
+import static io.trino.spi.type.BooleanType.BOOLEAN;
+import static io.trino.spi.type.CharType.createCharType;
+import static io.trino.spi.type.DateType.DATE;
+import static io.trino.spi.type.DecimalType.createDecimalType;
+import static io.trino.spi.type.DoubleType.DOUBLE;
+import static io.trino.spi.type.IntegerType.INTEGER;
+import static io.trino.spi.type.SmallintType.SMALLINT;
+import static io.trino.spi.type.TimeType.createTimeType;
+import static io.trino.spi.type.TimeWithTimeZoneType.createTimeWithTimeZoneType;
+import static io.trino.spi.type.TinyintType.TINYINT;
+import static io.trino.spi.type.VarbinaryType.VARBINARY;
+import static io.trino.spi.type.VarcharType.createVarcharType;
+import static io.trino.testing.TestingNames.randomNameSuffix;
+import static java.lang.String.format;
+import static java.time.ZoneOffset.UTC;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS;
+import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT;
+
+/**
+ * @see HsqlDB Short Guide to Data Types
+ * Some tests expect the America/Bahia_Banderas timezone for proper timestamp resolution.
+ * This requires passing the -Duser.timezone=America/Bahia_Banderas flag to your JVM.
+ */
+@TestInstance(PER_CLASS)
+@Execution(CONCURRENT)
+final class TestHsqlDbTypeMapping
+ extends AbstractTestQueryFramework
+{
+ protected TestingHsqlDbServer server;
+
+ private final ZoneId jvmZone = ZoneId.systemDefault();
+ // no DST in 1970, but has DST in later years (e.g. 2018)
+ private final ZoneId vilnius = ZoneId.of("Europe/Vilnius");
+ // minutes offset change since 1970-01-01, no DST
+ private final ZoneId kathmandu = ZoneId.of("Asia/Kathmandu");
+
+ @Language("RegExp")
+ private static final String NUMERIC_VALUE_OUT_OF_RANGE = "data exception: numeric value out of range";
+ @Language("RegExp")
+ private static final String INVALID_DATETIME_FORMAT = "data exception: invalid datetime format";
+
+ @BeforeAll
+ void setUp()
+ {
+ checkState(jvmZone.getId().equals("America/Bahia_Banderas"), "This test assumes certain JVM time zone");
+ checkIsGap(jvmZone, LocalDate.of(1932, 4, 1));
+ checkIsGap(vilnius, LocalDate.of(1983, 4, 1));
+ verify(vilnius.getRules().getValidOffsets(LocalDate.of(1983, 10, 1).atStartOfDay().minusMinutes(1)).size() == 2);
+ }
+
+ @Override
+ protected QueryRunner createQueryRunner()
+ throws Exception
+ {
+ server = closeAfterClass(new TestingHsqlDbServer());
+ return HsqlDbQueryRunner.builder(server).build();
+ }
+
+ @Test
+ void testBoolean()
+ {
+ SqlDataTypeTest.create()
+ .addRoundTrip("BOOLEAN", "TRUE", BOOLEAN, "TRUE")
+ .addRoundTrip("BOOLEAN", "FALSE", BOOLEAN, "FALSE")
+ .addRoundTrip("BOOLEAN", "NULL", BOOLEAN, "CAST(NULL AS BOOLEAN)")
+ .execute(getQueryRunner(), hsqlDbCreateAndInsert("test_boolean"))
+ .execute(getQueryRunner(), trinoCreateAsSelect("test_boolean"))
+ .execute(getQueryRunner(), trinoCreateAndInsert("test_boolean"));
+ }
+
+ @Test
+ void testTinyInt()
+ {
+ SqlDataTypeTest.create()
+ .addRoundTrip("TINYINT", "-128", TINYINT, "CAST(-128 AS TINYINT)")
+ .addRoundTrip("TINYINT", "127", TINYINT, "CAST(127 AS TINYINT)")
+ .addRoundTrip("TINYINT", "NULL", TINYINT, "CAST(NULL AS TINYINT)")
+ .execute(getQueryRunner(), hsqlDbCreateAndInsert("test_tinyint"))
+ .execute(getQueryRunner(), trinoCreateAsSelect("test_tinyint"))
+ .execute(getQueryRunner(), trinoCreateAndInsert("test_tinyint"));
+ }
+
+ @Test
+ void testUnsupportedTinyInt()
+ {
+ try (TestTable table = new TestTable(server::execute, "test_unsupported_tinyint", "(data tinyint)")) {
+ assertHsqlDbQueryFails(
+ "INSERT INTO " + table.getName() + " VALUES (-129)", // min - 1
+ NUMERIC_VALUE_OUT_OF_RANGE);
+ assertHsqlDbQueryFails(
+ "INSERT INTO " + table.getName() + " VALUES (128)", // max + 1
+ NUMERIC_VALUE_OUT_OF_RANGE);
+ }
+ }
+
+ @Test
+ void testSmallInt()
+ {
+ SqlDataTypeTest.create()
+ .addRoundTrip("SMALLINT", "-32768", SMALLINT, "SMALLINT '-32768'")
+ .addRoundTrip("SMALLINT", "32767", SMALLINT, "SMALLINT '32767'")
+ .addRoundTrip("SMALLINT", "NULL", SMALLINT, "CAST(NULL AS SMALLINT)")
+ .execute(getQueryRunner(), hsqlDbCreateAndInsert("test_smallint"))
+ .execute(getQueryRunner(), trinoCreateAsSelect("test_smallint"))
+ .execute(getQueryRunner(), trinoCreateAndInsert("test_smallint"));
+ }
+
+ @Test
+ void testUnsupportedSmallint()
+ {
+ try (TestTable table = new TestTable(server::execute, "test_unsupported_smallint", "(data smallint)")) {
+ assertHsqlDbQueryFails(
+ "INSERT INTO " + table.getName() + " VALUES (-32769)", // min - 1
+ NUMERIC_VALUE_OUT_OF_RANGE);
+ assertHsqlDbQueryFails(
+ "INSERT INTO " + table.getName() + " VALUES (32768)", // max + 1
+ NUMERIC_VALUE_OUT_OF_RANGE);
+ }
+ }
+
+ @Test
+ void testInteger()
+ {
+ SqlDataTypeTest.create()
+ .addRoundTrip("INTEGER", "-2147483648", INTEGER, "-2147483648")
+ .addRoundTrip("INTEGER", "2147483647", INTEGER, "2147483647")
+ .addRoundTrip("INTEGER", "NULL", INTEGER, "CAST(NULL AS INTEGER)")
+ .execute(getQueryRunner(), hsqlDbCreateAndInsert("test_integer"))
+ .execute(getQueryRunner(), trinoCreateAsSelect("test_integer"))
+ .execute(getQueryRunner(), trinoCreateAndInsert("test_integer"));
+ }
+
+ @Test
+ void testUnsupportedInteger()
+ {
+ try (TestTable table = new TestTable(server::execute, "test_unsupported_integer", "(data integer)")) {
+ assertHsqlDbQueryFails(
+ "INSERT INTO " + table.getName() + " VALUES (-2147483649)", // min - 1
+ NUMERIC_VALUE_OUT_OF_RANGE);
+ assertHsqlDbQueryFails(
+ "INSERT INTO " + table.getName() + " VALUES (2147483648)", // max + 1
+ NUMERIC_VALUE_OUT_OF_RANGE);
+ }
+ }
+
+ @Test
+ void testInt()
+ {
+ SqlDataTypeTest.create()
+ .addRoundTrip("INT", "-2147483648", INTEGER, "-2147483648")
+ .addRoundTrip("INT", "2147483647", INTEGER, "2147483647")
+ .addRoundTrip("INT", "NULL", INTEGER, "CAST(NULL AS INT)")
+ .execute(getQueryRunner(), hsqlDbCreateAndInsert("test_int"))
+ .execute(getQueryRunner(), trinoCreateAsSelect("test_int"))
+ .execute(getQueryRunner(), trinoCreateAndInsert("test_int"));
+ }
+
+ @Test
+ void testUnsupportedInt()
+ {
+ try (TestTable table = new TestTable(server::execute, "test_unsupported_int", "(data int)")) {
+ assertHsqlDbQueryFails(
+ "INSERT INTO " + table.getName() + " VALUES (-2147483649)", // min - 1
+ NUMERIC_VALUE_OUT_OF_RANGE);
+ assertHsqlDbQueryFails(
+ "INSERT INTO " + table.getName() + " VALUES (2147483648)", // max + 1
+ NUMERIC_VALUE_OUT_OF_RANGE);
+ }
+ }
+
+ @Test
+ void testBigInt()
+ {
+ SqlDataTypeTest.create()
+ .addRoundTrip("BIGINT", "-9223372036854775808", BIGINT, "-9223372036854775808")
+ .addRoundTrip("BIGINT", "9223372036854775807", BIGINT, "9223372036854775807")
+ .addRoundTrip("BIGINT", "NULL", BIGINT, "CAST(NULL AS BIGINT)")
+ .execute(getQueryRunner(), hsqlDbCreateAndInsert("test_bigint"))
+ .execute(getQueryRunner(), trinoCreateAsSelect("test_bigint"))
+ .execute(getQueryRunner(), trinoCreateAndInsert("test_bigint"));
+ }
+
+ @Test
+ void testUnsupportedBigInt()
+ {
+ try (TestTable table = new TestTable(server::execute, "test_unsupported_bigint", "(data bigint)")) {
+ assertHsqlDbQueryFails(
+ "INSERT INTO " + table.getName() + " VALUES (-9223372036854775809)", // min - 1
+ NUMERIC_VALUE_OUT_OF_RANGE);
+ assertHsqlDbQueryFails(
+ "INSERT INTO " + table.getName() + " VALUES (9223372036854775808)", // max + 1
+ NUMERIC_VALUE_OUT_OF_RANGE);
+ }
+ }
+
+ @Test
+ void testDecimal()
+ {
+ SqlDataTypeTest.create()
+ .addRoundTrip("DECIMAL(3, 0)", "CAST(NULL AS DECIMAL(3, 0))", createDecimalType(3, 0), "CAST(NULL AS DECIMAL(3, 0))")
+ .addRoundTrip("DECIMAL(3, 0)", "CAST('193' AS DECIMAL(3, 0))", createDecimalType(3, 0), "CAST('193' AS DECIMAL(3, 0))")
+ .addRoundTrip("DECIMAL(3, 0)", "CAST('19' AS DECIMAL(3, 0))", createDecimalType(3, 0), "CAST('19' AS DECIMAL(3, 0))")
+ .addRoundTrip("DECIMAL(3, 0)", "CAST('-193' AS DECIMAL(3, 0))", createDecimalType(3, 0), "CAST('-193' AS DECIMAL(3, 0))")
+ .addRoundTrip("DECIMAL(3, 1)", "CAST('10.0' AS DECIMAL(3, 1))", createDecimalType(3, 1), "CAST('10.0' AS DECIMAL(3, 1))")
+ .addRoundTrip("DECIMAL(3, 1)", "CAST('10.1' AS DECIMAL(3, 1))", createDecimalType(3, 1), "CAST('10.1' AS DECIMAL(3, 1))")
+ .addRoundTrip("DECIMAL(3, 1)", "CAST('-10.1' AS DECIMAL(3, 1))", createDecimalType(3, 1), "CAST('-10.1' AS DECIMAL(3, 1))")
+ .addRoundTrip("DECIMAL(4, 2)", "CAST('2' AS DECIMAL(4, 2))", createDecimalType(4, 2), "CAST('2' AS DECIMAL(4, 2))")
+ .addRoundTrip("DECIMAL(4, 2)", "CAST('2.3' AS DECIMAL(4, 2))", createDecimalType(4, 2), "CAST('2.3' AS DECIMAL(4, 2))")
+ .addRoundTrip("DECIMAL(24, 2)", "CAST('2' AS DECIMAL(24, 2))", createDecimalType(24, 2), "CAST('2' AS DECIMAL(24, 2))")
+ .addRoundTrip("DECIMAL(24, 2)", "CAST('2.3' AS DECIMAL(24, 2))", createDecimalType(24, 2), "CAST('2.3' AS DECIMAL(24, 2))")
+ .addRoundTrip("DECIMAL(24, 2)", "CAST('123456789.3' AS DECIMAL(24, 2))", createDecimalType(24, 2), "CAST('123456789.3' AS DECIMAL(24, 2))")
+ .addRoundTrip("DECIMAL(24, 4)", "CAST('12345678901234567890.31' AS DECIMAL(24, 4))", createDecimalType(24, 4), "CAST('12345678901234567890.31' AS DECIMAL(24, 4))")
+ .addRoundTrip("DECIMAL(30, 5)", "CAST('3141592653589793238462643.38327' AS DECIMAL(30, 5))", createDecimalType(30, 5), "CAST('3141592653589793238462643.38327' AS DECIMAL(30, 5))")
+ .addRoundTrip("DECIMAL(30, 5)", "CAST('-3141592653589793238462643.38327' AS DECIMAL(30, 5))", createDecimalType(30, 5), "CAST('-3141592653589793238462643.38327' AS DECIMAL(30, 5))")
+ .addRoundTrip("DECIMAL(38, 0)", "CAST(NULL AS DECIMAL(38, 0))", createDecimalType(38, 0), "CAST(NULL AS DECIMAL(38, 0))")
+ .addRoundTrip("DECIMAL(38, 0)", "CAST('27182818284590452353602874713526624977' AS DECIMAL(38, 0))", createDecimalType(38, 0), "CAST('27182818284590452353602874713526624977' AS DECIMAL(38, 0))")
+ .addRoundTrip("DECIMAL(38, 0)", "CAST('-27182818284590452353602874713526624977' AS DECIMAL(38, 0))", createDecimalType(38, 0), "CAST('-27182818284590452353602874713526624977' AS DECIMAL(38, 0))")
+ .addRoundTrip("DECIMAL(38, 38)", "CAST('0.27182818284590452353602874713526624977' AS DECIMAL(38, 38))", createDecimalType(38, 38), "CAST('0.27182818284590452353602874713526624977' AS DECIMAL(38, 38))")
+ .addRoundTrip("DECIMAL(38, 38)", "CAST('-0.27182818284590452353602874713526624977' AS DECIMAL(38, 38))", createDecimalType(38, 38), "CAST('-0.27182818284590452353602874713526624977' AS DECIMAL(38, 38))")
+ .execute(getQueryRunner(), hsqlDbCreateAndInsert("test_decimal"))
+ .execute(getQueryRunner(), trinoCreateAsSelect("test_decimal"))
+ .execute(getQueryRunner(), trinoCreateAndInsert("test_decimal"));
+ }
+
+ @Test
+ void testFloat()
+ {
+ // we are not testing Nan/-Infinity/+Infinity as those are not supported by MariaDB
+ SqlDataTypeTest.create()
+ .addRoundTrip("FLOAT", "3.14", DOUBLE, "DOUBLE '3.14'")
+ .addRoundTrip("FLOAT", "10.3e0", DOUBLE, "DOUBLE '10.3e0'")
+ .addRoundTrip("FLOAT", "NULL", DOUBLE, "CAST(NULL AS DOUBLE)")
+ .execute(getQueryRunner(), hsqlDbCreateAndInsert("test_float"));
+ //.execute(getQueryRunner(), trinoCreateAsSelect("test_float"))
+ //.execute(getQueryRunner(), trinoCreateAndInsert("test_float"));
+ }
+
+ @Test
+ void testDouble()
+ {
+ // we are not testing Nan/-Infinity/+Infinity as those are not supported by MariaDB
+ SqlDataTypeTest.create()
+ .addRoundTrip("DOUBLE", "3.14", DOUBLE, "CAST(3.14 AS DOUBLE)")
+ .addRoundTrip("DOUBLE", "1.0E100", DOUBLE, "1.0E100")
+ .addRoundTrip("DOUBLE", "1.23456E12", DOUBLE, "1.23456E12")
+ .addRoundTrip("DOUBLE", "NULL", DOUBLE, "CAST(NULL AS DOUBLE)")
+ .execute(getQueryRunner(), hsqlDbCreateAndInsert("test_double"))
+ .execute(getQueryRunner(), trinoCreateAsSelect("test_double"))
+ .execute(getQueryRunner(), trinoCreateAndInsert("test_double"));
+ }
+
+ @Test
+ void testVarchar()
+ {
+ SqlDataTypeTest.create()
+ .addRoundTrip("VARCHAR(10)", "'text_a'", createVarcharType(10), "CAST('text_a' AS VARCHAR(10))")
+ .addRoundTrip("VARCHAR(255)", "'text_b'", createVarcharType(255), "CAST('text_b' AS VARCHAR(255))")
+ .addRoundTrip("VARCHAR(4001)", "'text_c'", createVarcharType(4001), "CAST('text_c' AS VARCHAR(4001))")
+ .addRoundTrip("VARCHAR(5)", "CAST('攻殻機動隊' AS VARCHAR(5))", createVarcharType(5), "CAST('攻殻機動隊' AS VARCHAR(5))")
+ .addRoundTrip("VARCHAR(32)", "CAST('攻殻機動隊' AS VARCHAR(32))", createVarcharType(32), "CAST('攻殻機動隊' AS VARCHAR(32))")
+ .addRoundTrip("VARCHAR(20)", "CAST('😂' AS VARCHAR(20))", createVarcharType(20), "CAST('😂' AS VARCHAR(20))")
+ .addRoundTrip("VARCHAR(77)", "CAST('Ну, погоди!' AS VARCHAR(77))", createVarcharType(77), "CAST('Ну, погоди!' AS VARCHAR(77))")
+ .execute(getQueryRunner(), hsqlDbCreateAndInsert("test_varchar"))
+ .execute(getQueryRunner(), trinoCreateAsSelect("test_varchar"))
+ .execute(getQueryRunner(), trinoCreateAndInsert("test_varchar"));
+ }
+
+ @Test
+ void testUnboundedVarchar()
+ {
+ SqlDataTypeTest.create()
+ .addRoundTrip("VARCHAR", "'text_a'", createVarcharType(32768), "CAST('text_a' AS VARCHAR(32768))")
+ .addRoundTrip("VARCHAR", "'text_b'", createVarcharType(32768), "CAST('text_b' AS VARCHAR(32768))")
+ .addRoundTrip("VARCHAR", "'text_d'", createVarcharType(32768), "CAST('text_d' AS VARCHAR(32768))")
+ .addRoundTrip("VARCHAR", "'攻殻機動隊'", createVarcharType(32768), "CAST('攻殻機動隊' AS VARCHAR(32768))")
+ .addRoundTrip("VARCHAR", "'攻殻機動隊'", createVarcharType(32768), "CAST('攻殻機動隊' AS VARCHAR(32768))")
+ .addRoundTrip("VARCHAR", "'攻殻機動隊'", createVarcharType(32768), "CAST('攻殻機動隊' AS VARCHAR(32768))")
+ .addRoundTrip("VARCHAR", "'😂'", createVarcharType(32768), "CAST('😂' AS VARCHAR(32768))")
+ .addRoundTrip("VARCHAR", "'Ну, погоди!'", createVarcharType(32768), "CAST('Ну, погоди!' AS VARCHAR(32768))")
+ .addRoundTrip("VARCHAR", "'text_f'", createVarcharType(32768), "CAST('text_f' AS VARCHAR(32768))")
+ .execute(getQueryRunner(), hsqlDbCreateAndInsert("test_unbounded_varchar"))
+ .execute(getQueryRunner(), trinoCreateAndInsert("test_unbounded_varchar"))
+ .execute(getQueryRunner(), trinoCreateAsSelect("test_unbounded_varchar"));
+ }
+
+ @Test
+ void testCreatedParameterizedVarchar()
+ {
+ SqlDataTypeTest.create()
+ .addRoundTrip("varchar(32)", "'e'", createVarcharType(32), "CAST('e' AS VARCHAR(32))")
+ .addRoundTrip("varchar(15000)", "'f'", createVarcharType(15000), "CAST('f' AS VARCHAR(15000))")
+ .execute(getQueryRunner(), hsqlDbCreateAndInsert("test_parameterized_varchar"))
+ .execute(getQueryRunner(), trinoCreateAndInsert("test_parameterized_varchar"))
+ .execute(getQueryRunner(), trinoCreateAsSelect("test_parameterized_varchar"));
+ }
+
+ @Test
+ void testCreatedParameterizedVarcharUnicode()
+ {
+ SqlDataTypeTest.create()
+ .addRoundTrip("varchar(5)", "'攻殻機動隊'", createVarcharType(5), "CAST('攻殻機動隊' AS VARCHAR(5))")
+ .addRoundTrip("varchar(32)", "'攻殻機動隊'", createVarcharType(32), "CAST('攻殻機動隊' AS VARCHAR(32))")
+ .addRoundTrip("varchar(20000)", "'攻殻機動隊'", createVarcharType(20000), "CAST('攻殻機動隊' AS VARCHAR(20000))")
+ // FIXME: Why we need to put 2 as maximum length for passing this test (it fails with 1)?
+ .addRoundTrip("varchar(2)", "'😂'", createVarcharType(2), "CAST('😂' AS VARCHAR(2))")
+ .addRoundTrip("varchar(77)", "'Ну, погоди!'", createVarcharType(77), "CAST('Ну, погоди!' AS VARCHAR(77))")
+ .execute(getQueryRunner(), hsqlDbCreateAndInsert("test_parameterized_varchar_unicode"))
+ .execute(getQueryRunner(), trinoCreateAndInsert("test_parameterized_varchar_unicode"))
+ .execute(getQueryRunner(), trinoCreateAsSelect("test_parameterized_varchar_unicode"));
+ }
+
+ @Test
+ void testParameterizedChar()
+ {
+ SqlDataTypeTest.create()
+ .addRoundTrip("char", "''", createCharType(1), "CAST('' AS CHAR(1))")
+ .addRoundTrip("char", "'a'", createCharType(1), "CAST('a' AS CHAR(1))")
+ .addRoundTrip("char(1)", "''", createCharType(1), "CAST('' AS CHAR(1))")
+ .addRoundTrip("char(1)", "'a'", createCharType(1), "CAST('a' AS CHAR(1))")
+ .addRoundTrip("char(8)", "'abc'", createCharType(8), "CAST('abc' AS CHAR(8))")
+ .addRoundTrip("char(8)", "'12345678'", createCharType(8), "CAST('12345678' AS CHAR(8))")
+ .execute(getQueryRunner(), trinoCreateAsSelect("hsqldb_test_parameterized_char"))
+ .execute(getQueryRunner(), hsqlDbCreateAndInsert("hsqldb_test_parameterized_char"));
+ }
+
+ @Test
+ void testHsqlDbParameterizedCharUnicode()
+ {
+ SqlDataTypeTest.create()
+ .addRoundTrip("char(1)", "'攻'", createCharType(1), "CAST('攻' AS CHAR(1))")
+ .addRoundTrip("char(5)", "'攻殻'", createCharType(5), "CAST('攻殻' AS CHAR(5))")
+ .addRoundTrip("char(5)", "'攻殻機動隊'", createCharType(5), "CAST('攻殻機動隊' AS CHAR(5))")
+ // FIXME: Why we need to put 2 as maximum length for passing this test (it fails with 1)?
+ .addRoundTrip("char(2)", "'😂'", createCharType(2), "CAST('😂' AS char(2))")
+ .addRoundTrip("char(77)", "'Ну, погоди!'", createCharType(77), "CAST('Ну, погоди!' AS char(77))")
+ .execute(getQueryRunner(), hsqlDbCreateAndInsert("hsqldb_test_parameterized_char"));
+ }
+
+ @Test
+ void testCharTrailingSpace()
+ {
+ SqlDataTypeTest.create()
+ .addRoundTrip("char(10)", "'test'", createCharType(10), "CAST('test' AS CHAR(10))")
+ .addRoundTrip("char(10)", "'test '", createCharType(10), "CAST('test' AS CHAR(10))")
+ .addRoundTrip("char(10)", "'test '", createCharType(10), "CAST('test' AS CHAR(10))")
+ .execute(getQueryRunner(), hsqlDbCreateAndInsert("hsqldb_char_trailing_space"));
+ }
+
+ @Test
+ void testVarbinary()
+ {
+ varbinaryTestCases("varbinary(50)")
+ .execute(getQueryRunner(), hsqlDbCreateAndInsert("test_varbinary"));
+
+ //varbinaryTestCases("blob")
+ // .execute(getQueryRunner(), hsqlDbCreateAndInsert("test_varbinary"));
+
+ //varbinaryTestCases("varbinary")
+ // .execute(getQueryRunner(), trinoCreateAsSelect("test_varbinary"));
+ }
+
+ private SqlDataTypeTest varbinaryTestCases(String insertType)
+ {
+ return SqlDataTypeTest.create()
+ .addRoundTrip(insertType, "NULL", VARBINARY, "CAST(NULL AS varbinary)")
+ .addRoundTrip(insertType, "X''", VARBINARY, "X''")
+ .addRoundTrip(insertType, "X'68656C6C6F'", VARBINARY, "to_utf8('hello')")
+ .addRoundTrip(insertType, "X'5069C4996B6E6120C582C4856B61207720E69DB1E4BAACE983BD'", VARBINARY, "to_utf8('Piękna łąka w 東京都')")
+ .addRoundTrip(insertType, "X'4261672066756C6C206F6620F09F92B0'", VARBINARY, "to_utf8('Bag full of 💰')")
+ .addRoundTrip(insertType, "X'0001020304050607080DF9367AA7000000'", VARBINARY, "X'0001020304050607080DF9367AA7000000'") // non-text
+ .addRoundTrip(insertType, "X'000000000000'", VARBINARY, "X'000000000000'");
+ }
+
+ @Test
+ void testBinary()
+ {
+ SqlDataTypeTest.create()
+ .addRoundTrip("binary(18)", "NULL", VARBINARY, "CAST(NULL AS varbinary)")
+ .addRoundTrip("binary(18)", "X''", VARBINARY, "X'000000000000000000000000000000000000'")
+ .addRoundTrip("binary(18)", "X'68656C6C6F'", VARBINARY, "to_utf8('hello') || X'00000000000000000000000000'")
+ .addRoundTrip("binary(18)", "X'C582C4856B61207720E69DB1E4BAACE983BD'", VARBINARY, "to_utf8('łąka w 東京都')") // no trailing zeros
+ .addRoundTrip("binary(18)", "X'4261672066756C6C206F6620F09F92B0'", VARBINARY, "to_utf8('Bag full of 💰') || X'0000'")
+ .addRoundTrip("binary(18)", "X'0001020304050607080DF9367AA7000000'", VARBINARY, "X'0001020304050607080DF9367AA700000000'") // non-text prefix
+ .addRoundTrip("binary(18)", "X'000000000000'", VARBINARY, "X'000000000000000000000000000000000000'")
+ .execute(getQueryRunner(), hsqlDbCreateAndInsert("test_binary"));
+ }
+
+ /*@Test
+ void testUuid()
+ {
+ SqlDataTypeTest.create()
+ .addRoundTrip("uuid", "CAST ('00000000-0000-0000-0000-000000000000' AS UUID)", UUID)
+ .addRoundTrip("uuid", "CAST ('123e4567-e89b-12d3-a456-426655440000' AS UUID)", UUID)
+ //.execute(getQueryRunner(), hsqlDbCreateAndInsert("test_uuid"))
+ .execute(getQueryRunner(), trinoCreateAsSelect("test_uuid"))
+ .execute(getQueryRunner(), trinoCreateAndInsert("test_uuid"));
+ }*/
+
+ @Test
+ void testDate()
+ {
+ testDate(UTC);
+ testDate(jvmZone);
+ testDate(vilnius);
+ testDate(kathmandu);
+ testDate(TestingSession.DEFAULT_TIME_ZONE_KEY.getZoneId());
+ }
+
+ private void testDate(ZoneId sessionZone)
+ {
+ Session session = Session.builder(getSession())
+ .setTimeZoneKey(TimeZoneKey.getTimeZoneKey(sessionZone.getId()))
+ .build();
+
+ dateTest(Function.identity())
+ .execute(getQueryRunner(), session, hsqlDbCreateAndInsert("test_date"));
+
+ dateTest(inputLiteral -> format("DATE %s", inputLiteral))
+ .execute(getQueryRunner(), session, trinoCreateAsSelect(session, "test_date"))
+ .execute(getQueryRunner(), session, trinoCreateAsSelect("test_date"))
+ .execute(getQueryRunner(), session, trinoCreateAndInsert(session, "test_date"))
+ .execute(getQueryRunner(), session, trinoCreateAndInsert("test_date"));
+ }
+
+ private SqlDataTypeTest dateTest(Function inputLiteralFactory)
+ {
+ // BC dates not supported by HsqlDB
+ return SqlDataTypeTest.create()
+ .addRoundTrip("date", "NULL", DATE, "CAST(NULL AS DATE)")
+ // min date supported by HsqlDB
+ .addRoundTrip("date", inputLiteralFactory.apply("'0001-01-01'"), DATE, "DATE '0001-01-01'")
+ .addRoundTrip("date", inputLiteralFactory.apply("'0012-12-12'"), DATE, "DATE '0012-12-12'")
+ // before julian->gregorian switch
+ .addRoundTrip("date", inputLiteralFactory.apply("'1582-10-04'"), DATE, "DATE '1582-10-04'")
+ // after julian->gregorian switch
+ .addRoundTrip("date", inputLiteralFactory.apply("'1582-10-15'"), DATE, "DATE '1582-10-15'")
+ // before epoch
+ .addRoundTrip("date", inputLiteralFactory.apply("'1952-04-03'"), DATE, "DATE '1952-04-03'")
+ .addRoundTrip("date", inputLiteralFactory.apply("'1970-01-01'"), DATE, "DATE '1970-01-01'")
+ .addRoundTrip("date", inputLiteralFactory.apply("'1970-02-03'"), DATE, "DATE '1970-02-03'")
+ // summer on northern hemisphere (possible DST)
+ .addRoundTrip("date", inputLiteralFactory.apply("'2017-07-01'"), DATE, "DATE '2017-07-01'")
+ // winter on northern hemisphere (possible DST on southern hemisphere)
+ .addRoundTrip("date", inputLiteralFactory.apply("'2017-01-01'"), DATE, "DATE '2017-01-01'")
+ .addRoundTrip("date", inputLiteralFactory.apply("'1983-04-01'"), DATE, "DATE '1983-04-01'")
+ .addRoundTrip("date", inputLiteralFactory.apply("'1983-10-01'"), DATE, "DATE '1983-10-01'")
+ // max date supported by HsqlDB
+ .addRoundTrip("date", inputLiteralFactory.apply("'9999-12-31'"), DATE, "DATE '9999-12-31'");
+ }
+
+ @Test
+ void testUnsupportedDate()
+ {
+ // HsqlDB does not support negative dates
+ String unsupportedMin = "'-0001-01-01'";
+ // HsqlDB does not support > 4 digit years
+ String unsupportedMax = "'11111-01-01'";
+ // HsqlDB does not support dates during julian->gregorian switch
+ String startSwitch = "'1582-10-05'";
+ String middleSwitch = "'1582-10-10'";
+ String endSwitch = "'1582-10-14'";
+ String tableName = "test_date_unsupported" + randomNameSuffix();
+ assertUpdate(format("CREATE TABLE %s (test_date date)", tableName));
+ try {
+ assertQueryFails(format("INSERT INTO %s VALUES (date %s)", tableName, unsupportedMin),
+ INVALID_DATETIME_FORMAT);
+ assertQueryFails(format("INSERT INTO %s VALUES (date %s)", tableName, unsupportedMax),
+ INVALID_DATETIME_FORMAT);
+ assertQueryFails(format("INSERT INTO %s VALUES (date %s)", tableName, startSwitch),
+ INVALID_DATETIME_FORMAT);
+ assertQueryFails(format("INSERT INTO %s VALUES (date %s)", tableName, middleSwitch),
+ INVALID_DATETIME_FORMAT);
+ assertQueryFails(format("INSERT INTO %s VALUES (date %s)", tableName, endSwitch),
+ INVALID_DATETIME_FORMAT);
+ }
+ finally {
+ assertUpdate("DROP TABLE " + tableName);
+ }
+ }
+
+ @Test
+ void testTime()
+ {
+ testTime(UTC);
+ testTime(jvmZone);
+ testTime(vilnius);
+ testTime(kathmandu);
+ testDate(TestingSession.DEFAULT_TIME_ZONE_KEY.getZoneId());
+ }
+
+ private void testTime(ZoneId sessionZone)
+ {
+ Session session = Session.builder(getSession())
+ .setTimeZoneKey(TimeZoneKey.getTimeZoneKey(sessionZone.getId()))
+ .build();
+
+ commonTimeTest(Function.identity())
+ .execute(getQueryRunner(), session, hsqlDbCreateAndInsert("test_time"));
+
+ trinoTimeTest(inputLiteral -> format("TIME %s", inputLiteral))
+ .execute(getQueryRunner(), session, trinoCreateAsSelect(session, "test_time"))
+ .execute(getQueryRunner(), session, trinoCreateAsSelect("test_time"))
+ .execute(getQueryRunner(), session, trinoCreateAndInsert(session, "test_time"))
+ .execute(getQueryRunner(), session, trinoCreateAndInsert("test_time"));
+ }
+
+ private SqlDataTypeTest commonTimeTest(Function inputLiteralFactory)
+ {
+ return SqlDataTypeTest.create()
+ .addRoundTrip("time(0)", inputLiteralFactory.apply("'00:00:00'"), createTimeType(0), "TIME '00:00:00'")
+ .addRoundTrip("time(9)", inputLiteralFactory.apply("'00:00:00.000000000'"), createTimeType(9), "TIME '00:00:00.000000000'")
+ .addRoundTrip("time(9)", inputLiteralFactory.apply("'00:00:00.123456789'"), createTimeType(9), "TIME '00:00:00.123456789'")
+ .addRoundTrip("time(0)", inputLiteralFactory.apply("'12:34:56'"), createTimeType(0), "TIME '12:34:56'")
+ .addRoundTrip("time(9)", inputLiteralFactory.apply("'12:34:56.123456789'"), createTimeType(9), "TIME '12:34:56.123456789'")
+
+ // maximal value for a precision
+ .addRoundTrip("time(0)", inputLiteralFactory.apply("'23:59:59'"), createTimeType(0), "TIME '23:59:59'")
+ .addRoundTrip("time(1)", inputLiteralFactory.apply("'23:59:59.9'"), createTimeType(1), "TIME '23:59:59.9'")
+ .addRoundTrip("time(2)", inputLiteralFactory.apply("'23:59:59.99'"), createTimeType(2), "TIME '23:59:59.99'")
+ .addRoundTrip("time(3)", inputLiteralFactory.apply("'23:59:59.999'"), createTimeType(3), "TIME '23:59:59.999'")
+ .addRoundTrip("time(4)", inputLiteralFactory.apply("'23:59:59.9999'"), createTimeType(4), "TIME '23:59:59.9999'")
+ .addRoundTrip("time(5)", inputLiteralFactory.apply("'23:59:59.99999'"), createTimeType(5), "TIME '23:59:59.99999'")
+ .addRoundTrip("time(6)", inputLiteralFactory.apply("'23:59:59.999999'"), createTimeType(6), "TIME '23:59:59.999999'")
+ .addRoundTrip("time(7)", inputLiteralFactory.apply("'23:59:59.9999999'"), createTimeType(7), "TIME '23:59:59.9999999'")
+ .addRoundTrip("time(8)", inputLiteralFactory.apply("'23:59:59.99999999'"), createTimeType(8), "TIME '23:59:59.99999999'")
+ .addRoundTrip("time(9)", inputLiteralFactory.apply("'23:59:59.999999999'"), createTimeType(9), "TIME '23:59:59.999999999'")
+
+ .addRoundTrip("time(0)", inputLiteralFactory.apply("'00:00:00'"), createTimeType(0), "TIME '00:00:00'")
+ .addRoundTrip("time(9)", inputLiteralFactory.apply("'00:00:00.000000000'"), createTimeType(9), "TIME '00:00:00.000000000'")
+ .addRoundTrip("time(9)", inputLiteralFactory.apply("'00:00:00.123456789'"), createTimeType(9), "TIME '00:00:00.123456789'")
+ .addRoundTrip("time(0)", inputLiteralFactory.apply("'12:34:56'"), createTimeType(0), "TIME '12:34:56'")
+ .addRoundTrip("time(9)", inputLiteralFactory.apply("'12:34:56.123456789'"), createTimeType(9), "TIME '12:34:56.123456789'")
+
+ // maximal value for a precision
+ .addRoundTrip("time(0)", inputLiteralFactory.apply("'23:59:59'"), createTimeType(0), "TIME '23:59:59'")
+ .addRoundTrip("time(1)", inputLiteralFactory.apply("'23:59:59.9'"), createTimeType(1), "TIME '23:59:59.9'")
+ .addRoundTrip("time(2)", inputLiteralFactory.apply("'23:59:59.99'"), createTimeType(2), "TIME '23:59:59.99'")
+ .addRoundTrip("time(3)", inputLiteralFactory.apply("'23:59:59.999'"), createTimeType(3), "TIME '23:59:59.999'")
+ .addRoundTrip("time(4)", inputLiteralFactory.apply("'23:59:59.9999'"), createTimeType(4), "TIME '23:59:59.9999'")
+ .addRoundTrip("time(5)", inputLiteralFactory.apply("'23:59:59.99999'"), createTimeType(5), "TIME '23:59:59.99999'")
+ .addRoundTrip("time(6)", inputLiteralFactory.apply("'23:59:59.999999'"), createTimeType(6), "TIME '23:59:59.999999'")
+ .addRoundTrip("time(7)", inputLiteralFactory.apply("'23:59:59.9999999'"), createTimeType(7), "TIME '23:59:59.9999999'")
+ .addRoundTrip("time(8)", inputLiteralFactory.apply("'23:59:59.99999999'"), createTimeType(8), "TIME '23:59:59.99999999'")
+ .addRoundTrip("time(9)", inputLiteralFactory.apply("'23:59:59.999999999'"), createTimeType(9), "TIME '23:59:59.999999999'");
+ }
+
+ private SqlDataTypeTest trinoTimeTest(Function inputLiteralFactory)
+ {
+ return commonTimeTest(inputLiteralFactory)
+ // round down
+ .addRoundTrip(inputLiteralFactory.apply("'00:00:00.0000000001'"), "TIME '00:00:00.000000000'")
+ .addRoundTrip(inputLiteralFactory.apply("'00:00:00.000000000001'"), "TIME '00:00:00.000000000'")
+
+ // round down, maximal value
+ .addRoundTrip(inputLiteralFactory.apply("'00:00:00.0000000004'"), "TIME '00:00:00.000000000'")
+ .addRoundTrip(inputLiteralFactory.apply("'00:00:00.00000000049'"), "TIME '00:00:00.000000000'")
+ .addRoundTrip(inputLiteralFactory.apply("'00:00:00.000000000449'"), "TIME '00:00:00.000000000'")
+
+ // round up to next day, minimal value
+ .addRoundTrip(inputLiteralFactory.apply("'23:59:59.9999999995'"), "TIME '00:00:00.000000000'")
+ .addRoundTrip(inputLiteralFactory.apply("'23:59:59.99999999950'"), "TIME '00:00:00.000000000'")
+ .addRoundTrip(inputLiteralFactory.apply("'23:59:59.999999999500'"), "TIME '00:00:00.000000000'")
+
+ // round up to next day, maximal value
+ .addRoundTrip(inputLiteralFactory.apply("'23:59:59.9999999999'"), "TIME '00:00:00.000000000'")
+ .addRoundTrip(inputLiteralFactory.apply("'23:59:59.99999999999'"), "TIME '00:00:00.000000000'")
+ .addRoundTrip(inputLiteralFactory.apply("'23:59:59.999999999999'"), "TIME '00:00:00.000000000'")
+
+ // round down
+ .addRoundTrip(inputLiteralFactory.apply("'23:59:59.999999999499'"), "TIME '23:59:59.999999999'");
+ }
+
+ @Test
+ void testUnsupportedTime()
+ {
+ // HsqlDB does not support negative hours
+ String unsupportedNegativeHour = "'-01:00:00'";
+ // HsqlDB does not support negative minutes
+ String unsupportedNegativeMinute = "'00:-01:00'";
+ // HsqlDB does not support negative second
+ String unsupportedNegativeSecond = "'00:00:-01'";
+ // HsqlDB does not support > 23 digit hours
+ String unsupportedHour = "'24:00:00'";
+ // HsqlDB does not support > 59 digit minutes
+ String unsupportedMinute = "'00:60:00'";
+ // HsqlDB does not support > 59 digit seconds
+ String unsupportedSecond = "'00:00:60'";
+ String tableName = "test_time_unsupported" + randomNameSuffix();
+ String expectedMessage = "line 1:53: %s is not a valid TIME literal";
+ assertUpdate(format("CREATE TABLE %s (test_time time)", tableName));
+ try {
+ assertQueryFails(format("INSERT INTO %s VALUES (time %s)", tableName, unsupportedNegativeHour),
+ format(expectedMessage, unsupportedNegativeHour));
+ assertQueryFails(format("INSERT INTO %s VALUES (time %s)", tableName, unsupportedNegativeMinute),
+ format(expectedMessage, unsupportedNegativeMinute));
+ assertQueryFails(format("INSERT INTO %s VALUES (time %s)", tableName, unsupportedNegativeSecond),
+ format(expectedMessage, unsupportedNegativeSecond));
+ assertQueryFails(format("INSERT INTO %s VALUES (time %s)", tableName, unsupportedHour),
+ format(expectedMessage, unsupportedHour));
+ assertQueryFails(format("INSERT INTO %s VALUES (time %s)", tableName, unsupportedMinute),
+ format(expectedMessage, unsupportedMinute));
+ assertQueryFails(format("INSERT INTO %s VALUES (time %s)", tableName, unsupportedSecond),
+ format(expectedMessage, unsupportedSecond));
+ }
+ finally {
+ assertUpdate("DROP TABLE " + tableName);
+ }
+ }
+
+ @Test
+ void testTimeWithTimeZone()
+ {
+ testTimeWithTimeZone(UTC);
+ testTimeWithTimeZone(jvmZone);
+ testTimeWithTimeZone(vilnius);
+ testTimeWithTimeZone(kathmandu);
+ testTimeWithTimeZone(TestingSession.DEFAULT_TIME_ZONE_KEY.getZoneId());
+ }
+
+ private void testTimeWithTimeZone(ZoneId sessionZone)
+ {
+ Session session = Session.builder(getSession())
+ .setTimeZoneKey(TimeZoneKey.getTimeZoneKey(sessionZone.getId()))
+ .build();
+
+ commonTimeWithTimeZoneTest(inputLiteral -> format("TIME %s", inputLiteral))
+ .execute(getQueryRunner(), session, hsqlDbCreateAndInsert("test_time_with_zone"));
+
+ trinoTimeWithTimeZoneTest(inputLiteral -> format("TIME %s", inputLiteral))
+ .execute(getQueryRunner(), session, trinoCreateAsSelect(session, "test_time_with_zone"))
+ .execute(getQueryRunner(), session, trinoCreateAsSelect("test_time_with_zone"))
+ .execute(getQueryRunner(), session, trinoCreateAndInsert(session, "test_time_with_zone"))
+ .execute(getQueryRunner(), session, trinoCreateAndInsert("test_time_with_zone"));
+ }
+
+ private SqlDataTypeTest commonTimeWithTimeZoneTest(Function inputLiteralFactory)
+ {
+ return SqlDataTypeTest.create()
+ .addRoundTrip("TIME(0) WITH TIME ZONE", inputLiteralFactory.apply("'20:08:08-08:00'"), createTimeWithTimeZoneType(0))
+ .addRoundTrip("TIME(1) WITH TIME ZONE", inputLiteralFactory.apply("'20:08:08.1-08:00'"), createTimeWithTimeZoneType(1))
+ .addRoundTrip("TIME(2) WITH TIME ZONE", inputLiteralFactory.apply("'20:08:08.12-08:00'"), createTimeWithTimeZoneType(2))
+ .addRoundTrip("TIME(3) WITH TIME ZONE", inputLiteralFactory.apply("'13:29:38.123-01:00'"), createTimeWithTimeZoneType(3))
+ .addRoundTrip("TIME(4) WITH TIME ZONE", inputLiteralFactory.apply("'13:29:38.1234-01:00'"), createTimeWithTimeZoneType(4))
+ .addRoundTrip("TIME(5) WITH TIME ZONE", inputLiteralFactory.apply("'13:29:38.12345+02:00'"), createTimeWithTimeZoneType(5))
+ .addRoundTrip("TIME(6) WITH TIME ZONE", inputLiteralFactory.apply("'13:29:38.123456+02:00'"), createTimeWithTimeZoneType(6))
+ .addRoundTrip("TIME(7) WITH TIME ZONE", inputLiteralFactory.apply("'13:29:38.1234567+02:00'"), createTimeWithTimeZoneType(7))
+ .addRoundTrip("TIME(8) WITH TIME ZONE", inputLiteralFactory.apply("'13:29:38.12345678+02:00'"), createTimeWithTimeZoneType(8))
+ .addRoundTrip("TIME(9) WITH TIME ZONE", inputLiteralFactory.apply("'13:29:38.123456789+02:00'"), createTimeWithTimeZoneType(9));
+ }
+
+ private SqlDataTypeTest trinoTimeWithTimeZoneTest(Function inputLiteralFactory)
+ {
+ return commonTimeWithTimeZoneTest(inputLiteralFactory)
+ // round down
+ .addRoundTrip(inputLiteralFactory.apply("'00:00:00.0000000001'"), "TIME '00:00:00.000000000'")
+ .addRoundTrip(inputLiteralFactory.apply("'00:00:00.000000000001'"), "TIME '00:00:00.000000000'")
+
+ // round down, maximal value
+ .addRoundTrip(inputLiteralFactory.apply("'00:00:00.0000000004'"), "TIME '00:00:00.000000000'")
+ .addRoundTrip(inputLiteralFactory.apply("'00:00:00.00000000049'"), "TIME '00:00:00.000000000'")
+ .addRoundTrip(inputLiteralFactory.apply("'00:00:00.000000000449'"), "TIME '00:00:00.000000000'")
+
+ // round up to next day, minimal value
+ .addRoundTrip(inputLiteralFactory.apply("'23:59:59.9999999995'"), "TIME '00:00:00.000000000'")
+ .addRoundTrip(inputLiteralFactory.apply("'23:59:59.99999999950'"), "TIME '00:00:00.000000000'")
+ .addRoundTrip(inputLiteralFactory.apply("'23:59:59.999999999500'"), "TIME '00:00:00.000000000'")
+
+ // round up to next day, maximal value
+ .addRoundTrip(inputLiteralFactory.apply("'23:59:59.9999999999'"), "TIME '00:00:00.000000000'")
+ .addRoundTrip(inputLiteralFactory.apply("'23:59:59.99999999999'"), "TIME '00:00:00.000000000'")
+ .addRoundTrip(inputLiteralFactory.apply("'23:59:59.999999999999'"), "TIME '00:00:00.000000000'")
+
+ // round down
+ .addRoundTrip(inputLiteralFactory.apply("'23:59:59.999999999499'"), "TIME '23:59:59.999999999'");
+ }
+
+ @Test
+ void testUnsupportedTimeWithTimeZone()
+ {
+ // HsqlDB does not support negative hours
+ String unsupportedNegativeHour = "'-01:00:00-8:00'";
+ // HsqlDB does not support > 23 digit hours
+ String unsupportedHour = "'24:00:00-4:00'";
+ // HsqlDB does not support > 59 digit seconds
+ String unsupportedSecond = "'00:00:60-2:00'";
+ String tableName = "test_time_with_zone_unsupported" + randomNameSuffix();
+ String expectedMessage = "line 1:63: %s is not a valid TIME literal";
+ assertUpdate(format("CREATE TABLE %s (test_time time with time zone)", tableName));
+ try {
+ assertQueryFails(format("INSERT INTO %s VALUES (time %s)", tableName, unsupportedNegativeHour),
+ format(expectedMessage, unsupportedNegativeHour));
+ assertQueryFails(format("INSERT INTO %s VALUES (time %s)", tableName, unsupportedHour),
+ format(expectedMessage, unsupportedHour));
+ assertQueryFails(format("INSERT INTO %s VALUES (time %s)", tableName, unsupportedSecond),
+ format(expectedMessage, unsupportedSecond));
+ }
+ finally {
+ assertUpdate("DROP TABLE " + tableName);
+ }
+ }
+
+ private DataSetup trinoCreateAsSelect(String tableNamePrefix)
+ {
+ return trinoCreateAsSelect(getSession(), tableNamePrefix);
+ }
+
+ private DataSetup trinoCreateAsSelect(Session session, String tableNamePrefix)
+ {
+ return new CreateAsSelectDataSetup(new TrinoSqlExecutor(getQueryRunner(), session), tableNamePrefix);
+ }
+
+ private DataSetup trinoCreateAndInsert(String tableNamePrefix)
+ {
+ return trinoCreateAndInsert(getSession(), tableNamePrefix);
+ }
+
+ private DataSetup trinoCreateAndInsert(Session session, String tableNamePrefix)
+ {
+ return new CreateAndInsertDataSetup(new TrinoSqlExecutor(getQueryRunner(), session), tableNamePrefix);
+ }
+
+ private DataSetup hsqlDbCreateAndInsert(String tableNamePrefix)
+ {
+ return new HsqlDbCreateAndInsertDataSetup(server::execute, tableNamePrefix);
+ }
+
+ private static boolean isGap(ZoneId zone, LocalDateTime dateTime)
+ {
+ return zone.getRules().getValidOffsets(dateTime).isEmpty();
+ }
+
+ private static void checkIsGap(ZoneId zone, LocalDateTime dateTime)
+ {
+ verify(isGap(zone, dateTime), "Expected %s to be a gap in %s", dateTime, zone);
+ }
+
+ private static void checkIsGap(ZoneId zone, LocalDate date)
+ {
+ verify(isGap(zone, date), "Expected %s to be a gap in %s", date, zone);
+ }
+
+ private static boolean isGap(ZoneId zone, LocalDate date)
+ {
+ return zone.getRules().getValidOffsets(date.atStartOfDay()).isEmpty();
+ }
+
+ private void assertHsqlDbQueryFails(@Language("SQL") String sql, String expectedMessage)
+ {
+ assertThatThrownBy(() -> server.execute(sql))
+ .cause()
+ .hasMessageContaining(expectedMessage);
+ }
+}
diff --git a/plugin/trino-hsqldb/src/test/java/io/trino/plugin/hsqldb/TestingHsqlDbServer.java b/plugin/trino-hsqldb/src/test/java/io/trino/plugin/hsqldb/TestingHsqlDbServer.java
new file mode 100644
index 000000000000..e4abd134d02f
--- /dev/null
+++ b/plugin/trino-hsqldb/src/test/java/io/trino/plugin/hsqldb/TestingHsqlDbServer.java
@@ -0,0 +1,114 @@
+/*
+ * 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.hsqldb;
+
+import io.airlift.log.Logger;
+import io.trino.testing.ResourcePresence;
+import io.trino.testing.containers.PrintingLogConsumer;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.images.builder.ImageFromDockerfile;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+final class TestingHsqlDbServer
+ implements AutoCloseable
+{
+ private static final Logger log = Logger.get(TestingHsqlDbServer.class);
+
+ public static final String DEFAULT_VERSION = "2.7.4";
+ public static final String LATEST_VERSION = "2.7.4";
+ private static final String DEFAULT_JMV = "adoptopenjdk/openjdk11";
+ private static final String HSQLDB_ARCHIVE = "hsqldb-%s.jar";
+ private static final String HSQLDB_URL = "jdbc:hsqldb:hsql://%s:%s/";
+ private static final String DOWNLOAD_LOCATION = "https://repo1.maven.org/maven2/org/hsqldb/hsqldb/%s/%s";
+ private static final int HSQLDB_PORT = 9001;
+
+ private static class HsqldbContainer
+ extends GenericContainer
+ {
+ private HsqldbContainer(ImageFromDockerfile image)
+ {
+ super(image);
+ }
+ }
+
+ private final HsqldbContainer container;
+
+ public TestingHsqlDbServer()
+ {
+ this(DEFAULT_VERSION);
+ }
+
+ public TestingHsqlDbServer(String tag)
+ {
+ String archive = String.format(HSQLDB_ARCHIVE, tag);
+ String location = String.format(DOWNLOAD_LOCATION, tag, archive);
+ ImageFromDockerfile image = new ImageFromDockerfile()
+ .withDockerfileFromBuilder(builder ->
+ builder
+ .from(DEFAULT_JMV)
+ .add(location, archive)
+ .expose(HSQLDB_PORT)
+ .cmd("java", "-cp",
+ archive, "org.hsqldb.server.Server",
+ "--port", Integer.toString(HSQLDB_PORT),
+ "--silent", "false")
+ .build());
+ container = new HsqldbContainer(image).withExposedPorts(HSQLDB_PORT);
+ container.start();
+ container.followOutput(new PrintingLogConsumer("HsqlDB"));
+ log.info("%s version %s listening on port: %s", TestingHsqlDbServer.class.getName(), tag, getJdbcUrl());
+ }
+
+ public void execute(String sql)
+ {
+ try (Connection connection = DriverManager.getConnection(getJdbcUrl(), getUsername(), getPassword());
+ Statement statement = connection.createStatement()) {
+ statement.execute(sql);
+ }
+ catch (SQLException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ public String getUsername()
+ {
+ return "SA";
+ }
+
+ public String getPassword()
+ {
+ return "";
+ }
+
+ public String getJdbcUrl()
+ {
+ return String.format(HSQLDB_URL, container.getHost(), container.getMappedPort(HSQLDB_PORT));
+ }
+
+ @Override
+ public void close()
+ {
+ container.close();
+ }
+
+ @ResourcePresence
+ public boolean isRunning()
+ {
+ return container.isRunning();
+ }
+}
diff --git a/pom.xml b/pom.xml
index aa91ebf22d83..375adda5dbaf 100644
--- a/pom.xml
+++ b/pom.xml
@@ -79,6 +79,7 @@
plugin/trino-geospatial
plugin/trino-google-sheets
plugin/trino-hive
+ plugin/trino-hsqldb
plugin/trino-http-event-listener
plugin/trino-http-server-event-listener
plugin/trino-hudi
diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-all/hsqldb.properties b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-all/hsqldb.properties
new file mode 100644
index 000000000000..23a38e892de8
--- /dev/null
+++ b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-all/hsqldb.properties
@@ -0,0 +1,4 @@
+connector.name=hsqldb
+connection-url=jdbc:hsqldb:hsql://localhost:9001/
+connection-user=SA
+connection-password=
diff --git a/testing/trino-server-dev/etc/catalog/hsqldb.properties b/testing/trino-server-dev/etc/catalog/hsqldb.properties
new file mode 100644
index 000000000000..23a38e892de8
--- /dev/null
+++ b/testing/trino-server-dev/etc/catalog/hsqldb.properties
@@ -0,0 +1,4 @@
+connector.name=hsqldb
+connection-url=jdbc:hsqldb:hsql://localhost:9001/
+connection-user=SA
+connection-password=
diff --git a/testing/trino-server-dev/etc/config.properties b/testing/trino-server-dev/etc/config.properties
index 35fd47cf7a54..61b63d9e636f 100644
--- a/testing/trino-server-dev/etc/config.properties
+++ b/testing/trino-server-dev/etc/config.properties
@@ -35,6 +35,7 @@ plugin.bundles=\
../../plugin/trino-memory/pom.xml,\
../../plugin/trino-jmx/pom.xml,\
../../plugin/trino-hive/pom.xml,\
+ ../../plugin/trino-hsqldb/pom.xml,\
../../plugin/trino-hudi/pom.xml,\
../../plugin/trino-example-http/pom.xml,\
../../plugin/trino-faker/pom.xml,\