diff --git a/core/trino-server/src/main/provisio/trino.xml b/core/trino-server/src/main/provisio/trino.xml index 8fed2ae3937d..8fe719a3d48a 100644 --- a/core/trino-server/src/main/provisio/trino.xml +++ b/core/trino-server/src/main/provisio/trino.xml @@ -380,4 +380,10 @@ + + + + + + diff --git a/docs/src/main/sphinx/connector.md b/docs/src/main/sphinx/connector.md index 9c6259eb632f..289b18655df0 100644 --- a/docs/src/main/sphinx/connector.md +++ b/docs/src/main/sphinx/connector.md @@ -23,6 +23,7 @@ Hive Hudi Iceberg Ignite +InfluxDB JMX Kafka Kudu diff --git a/docs/src/main/sphinx/connector/influxdb.rst b/docs/src/main/sphinx/connector/influxdb.rst new file mode 100644 index 000000000000..c7ad8e59fe37 --- /dev/null +++ b/docs/src/main/sphinx/connector/influxdb.rst @@ -0,0 +1,167 @@ +======================= +InfluxDB connector +======================= + +The InfluxDB Connector allows access to `InfluxDB `_ data from Trino. +This document describes how to setup the InfluxDB Connector to run SQL queries against InfluxDB. + +Requirements +------------ + +To connect to InfluxDB, you need: + +* InfluxDB v1.8.0 or higher.(v2.x not currently supported) +* Network access from the Trino coordinator and workers to InfluxDB. + Port 8086 is the default port. + +Configuration +------------- + +To configure the InfluxDB connector, create a catalog properties file +``etc/catalog/example.properties`` with the following contents, +replacing the properties as appropriate: + +.. code-block:: properties + + connector.name=influxdb + influx.endpoint=http://localhost:8086 + influx.username=username + influx.password=password + +Configuration properties +^^^^^^^^^^^^^^^^^^^^^^^^ + +.. list-table:: The following configuration properties are available + :widths: 35, 55, 10 + :header-rows: 1 + + * - Property name + - Description + - Default + * - ``influx.endpoint`` + - Endpoint of the InfluxDB server to connect to. This property is required. + - + * - ``influx.username`` + - User name to use to connect to InfluxDB. + - + * - ``influx.password`` + - Password to use to connect to InfluxDB. + - + * - ``influx.connect-timeout`` + - The socket connect timeout to InfluxDB server. + - ``10s`` + * - ``influx.write-timeout`` + - The socket write timeout to InfluxDB server. + - ``10s`` + * - ``influx.read-timeout`` + - The socket read timeout to InfluxDB server. + - ``60s`` + +.. _influxdb-type-mapping: + +Type mapping +------------ + +Because Trino and InfluxDB each support types that the other does not, this +connector :ref:`maps some types ` when reading data. + + +InfluxDB type to Trino type mapping +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +The connector maps InfluxDB types to the corresponding Trino types +according to the following table: + +.. list-table:: InfluxDB type to Trino type mapping + :widths: 30, 30, 50 + :header-rows: 1 + + * - InfluxDB type + - Trino type + - Notes + * - ``TIMESTAMP`` + - ``TIMESTAMP`` + - ``Timestamp`` key ``time`` is unix nanosecond timestamp in influxdb. see `data types `_. + * - ``BOOLEAN`` + - ``BOOLEAN`` + - + * - ``FLOAT`` + - ``DOUBLE`` + - ``FLOAT`` is 64-bit floating-point numbers in influxdb. see `data types `_. + * - ``INTEGER`` + - ``BIGINT`` + - ``INTEGER`` is 64-bit integers in influxdb. see `data types `_. + * - ``STRING`` + - ``VARCHAR`` + - + +No other types are supported. + + +SQL support +----------- + +The connector provides :ref:`globally available ` and +:ref:`read operation ` statements to access data and +metadata in the InfluxDB catalog. + +Creation and deletion of schemas are supported. +As InfluxDB does not support creation of empty tables, creation of tables is not supported but deletion of tables is supported. + +.. _influxdb-pushdown: + +Pushdown support +^^^^^^^^^^^^^^^^ + +The connector supports pushdown for a number of operations: + +* :ref:`limit-pushdown` + +* :ref:`projection-pushdown` + +* :ref:`predicate-pushdown` + +But there are some special limitations for predicate-pushdown: + +To understand easily, think a measurement "student" in influxdb + +.. code-block:: text + + SHOW TAG KEYS: + tagKey + ------ + grade + class + + SHOW FIELD KEYS: + fieldKey fieldType + -------- --------- + name string + age integer + score float + + +Predicate pushdown of keys of ``STRING``, ``BOOLEAN``, ``INTEGER`` and ``FLOAT`` types are supported: + +For ``TIMESTAMP`` key, supports equality predicates ``=``, range predicates, such as ``>``, ``<``. + +For keys of ``STRING`` or ``BOOLEAN`` type (both tag set and field keys in InfluxDB), supports equality predicates ``=`` only. + +For keys of ``INTEGER`` or ``FLOAT`` type (field keys in InfluxDB), supports +equality predicates ``=``, inequality predicates ``!=``, and range predicates, such as ``>``, ``<``, or ``BETWEEN``. + +.. note:: + Decimal integer literals ``int_lit = ("1"…"9"){digit}`` pushdown works, hexadecimal and octal literals are not currently supported. + Floating-point literals ``float_lit = int_lit"."int_lit`` pushdown works, exponents are not currently supported. + +.. code-block:: sql + + -- Not pushed down + SELECT * FROM student WHERE name > 'CLOUD'; + SELECT * FROM student WHERE score <> 1.0E2; + -- Pushed down + SELECT * FROM student WHERE name = 'CLOUD'; + SELECT * FROM student WHERE name != 'CLOUD'; + SELECT * FROM student WHERE age > 12 and age < 14; + SELECT * FROM student WHERE score > 80.0 and score != 100; + SELECT * FROM student WHERE time >= timestamp '2020-01-01 00:00:00' and time <= timestamp '2020-01-01 23:59:59'; diff --git a/plugin/trino-influxdb/pom.xml b/plugin/trino-influxdb/pom.xml new file mode 100644 index 000000000000..fb85394bc53d --- /dev/null +++ b/plugin/trino-influxdb/pom.xml @@ -0,0 +1,215 @@ + + + 4.0.0 + + io.trino + trino-root + 471-SNAPSHOT + ../../pom.xml + + + trino-influxdb + trino-plugin + Trino - InfluxDB Connector + + + ${project.parent.basedir} + + + + + + com.google.guava + guava + + + com.google.inject + guice + + + + com.squareup.okhttp3 + logging-interceptor + + + com.squareup.okhttp3 + okhttp + + + io.airlift + bootstrap + + + io.airlift + configuration + + + io.airlift + json + + + io.airlift + log + + + io.airlift + units + + + io.trino + trino-plugin-toolkit + + + jakarta.annotation + jakarta.annotation-api + + + jakarta.validation + jakarta.validation-api + + + + org.influxdb + influxdb-java + 2.23 + + + com.squareup.okhttp3 + logging-interceptor + + + com.squareup.okhttp3 + okhttp + + + + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + io.airlift + slice + provided + + + + io.opentelemetry + opentelemetry-api + provided + + + io.opentelemetry + opentelemetry-context + provided + + + + io.trino + trino-spi + provided + + + org.openjdk.jol + jol-core + provided + + + com.fasterxml.jackson.core + jackson-databind + runtime + + + io.airlift + node + runtime + + + io.airlift + http-server + test + + + + io.airlift + junit-extensions + test + + + io.airlift + testing + test + + + io.trino + trino-client + test + + + io.trino + trino-main + test + + + io.trino + trino-main + test-jar + test + + + io.trino + trino-spi + test-jar + test + + + io.trino + trino-testing + test + + + io.trino + trino-testing-services + test + + + io.trino + trino-tpch + test + + + io.trino.tpch + tpch + test + + + org.assertj + assertj-core + test + + + org.junit.jupiter + junit-jupiter-api + 5.11.3 + test + + + org.testcontainers + influxdb + test + + + org.testcontainers + testcontainers + test + + + org.testng + testng + test + + + + diff --git a/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxClient.java b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxClient.java new file mode 100644 index 000000000000..f68e2054be74 --- /dev/null +++ b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxClient.java @@ -0,0 +1,39 @@ +/* + * 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.influxdb; + +import io.trino.spi.connector.SchemaTableName; +import org.influxdb.dto.Query; +import org.influxdb.dto.QueryResult; + +import java.util.List; +import java.util.Optional; + +public interface InfluxClient +{ + InfluxRecord query(Query query); + + List getSchemaNames(); + + List getSchemaTableNames(String schemaName); + + Optional getTableHandle(String schemaName, String tableName); + + void dropTable(String schemaName, String tableName); + + QueryResult createDatabase(String databaseName); + + void dropDatabase(String databaseName); +} diff --git a/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxColumnHandle.java b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxColumnHandle.java new file mode 100644 index 000000000000..988455a7c08b --- /dev/null +++ b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxColumnHandle.java @@ -0,0 +1,95 @@ +/* + * 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.influxdb; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.type.Type; + +import java.util.Objects; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static io.trino.plugin.influxdb.InfluxConstant.ColumnKind; +import static java.util.Objects.requireNonNull; + +public class InfluxColumnHandle + implements ColumnHandle +{ + private final String name; + private final Type type; + private final ColumnKind kind; + + @JsonCreator + public InfluxColumnHandle( + @JsonProperty("columnName") String name, + @JsonProperty("columnType") Type type, + @JsonProperty("columnKind") ColumnKind kind) + { + this.name = requireNonNull(name, "columnName is null"); + this.type = requireNonNull(type, "columnType is null"); + this.kind = requireNonNull(kind, "columnKind is null"); + } + + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty + public Type getType() + { + return type; + } + + @JsonProperty + public ColumnKind getKind() + { + return kind; + } + + @Override + public int hashCode() + { + return Objects.hash(name, type, kind); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if ((obj == null) || (getClass() != obj.getClass())) { + return false; + } + + InfluxColumnHandle other = (InfluxColumnHandle) obj; + return Objects.equals(this.name, other.name) && + Objects.equals(this.type, other.type) && + Objects.equals(this.kind, other.kind); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("name", name) + .add("type", type) + .add("kind", kind) + .toString(); + } +} diff --git a/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxConfig.java b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxConfig.java new file mode 100644 index 000000000000..8a688c889c13 --- /dev/null +++ b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxConfig.java @@ -0,0 +1,109 @@ +/* + * 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.influxdb; + +import io.airlift.configuration.Config; +import io.airlift.configuration.ConfigSecuritySensitive; +import io.airlift.units.Duration; +import io.airlift.units.MinDuration; +import jakarta.validation.constraints.AssertFalse; +import jakarta.validation.constraints.NotNull; +import jakarta.validation.constraints.Pattern; + +import java.util.Optional; + +import static java.util.concurrent.TimeUnit.SECONDS; + +public class InfluxConfig +{ + private String endpoint; + private String username; + private String password; + private Duration connectTimeOut = new Duration(10, SECONDS); + private Duration readTimeOut = new Duration(60, SECONDS); + + @NotNull + @Pattern(message = "Invalid endpoint. Expected http:// ", regexp = "^http://.*") + public String getEndpoint() + { + return endpoint; + } + + @Config("influx.endpoint") + public InfluxConfig setEndpoint(String endpoint) + { + this.endpoint = endpoint; + return this; + } + + @NotNull + public Optional getUsername() + { + return Optional.ofNullable(username); + } + + @Config("influx.username") + public InfluxConfig setUsername(String username) + { + this.username = username; + return this; + } + + @NotNull + public Optional getPassword() + { + return Optional.ofNullable(password); + } + + @ConfigSecuritySensitive + @Config("influx.password") + public InfluxConfig setPassword(String password) + { + this.password = password; + return this; + } + + @MinDuration("0s") + public Duration getConnectTimeOut() + { + return connectTimeOut; + } + + @Config("influx.connect-timeout") + public InfluxConfig setConnectTimeOut(Duration connectTimeOut) + { + this.connectTimeOut = connectTimeOut; + return this; + } + + @MinDuration("0s") + public Duration getReadTimeOut() + { + return readTimeOut; + } + + @Config("influx.read-timeout") + public InfluxConfig setReadTimeOut(Duration readTimeOut) + { + this.readTimeOut = readTimeOut; + return this; + } + + @AssertFalse(message = "'influx.username' and 'influx.password' must be both empty or both non-empty.") + public boolean isValidPasswordConfig() + { + return (username == null) ^ (password == null); + } +} diff --git a/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxConnector.java b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxConnector.java new file mode 100644 index 000000000000..8107dc2035da --- /dev/null +++ b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxConnector.java @@ -0,0 +1,80 @@ +/* + * 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.influxdb; + +import com.google.inject.Inject; +import io.airlift.bootstrap.LifeCycleManager; +import io.trino.spi.connector.Connector; +import io.trino.spi.connector.ConnectorMetadata; +import io.trino.spi.connector.ConnectorRecordSetProvider; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorSplitManager; +import io.trino.spi.connector.ConnectorTransactionHandle; +import io.trino.spi.transaction.IsolationLevel; + +import static io.trino.plugin.influxdb.InfluxTransactionHandle.INSTANCE; +import static java.util.Objects.requireNonNull; + +public class InfluxConnector + implements Connector +{ + private final LifeCycleManager lifeCycleManager; + private final InfluxMetadata metadata; + private final InfluxSplitManager splitManager; + private final InfluxRecordSetProvider recordSetProvider; + + @Inject + public InfluxConnector( + LifeCycleManager lifeCycleManager, + InfluxMetadata metadata, + InfluxSplitManager splitManager, + InfluxRecordSetProvider recordSetProvider) + { + this.lifeCycleManager = requireNonNull(lifeCycleManager, "lifeCycleManager is null"); + this.metadata = requireNonNull(metadata, "metadata is null"); + this.splitManager = requireNonNull(splitManager, "splitManager is null"); + this.recordSetProvider = requireNonNull(recordSetProvider, "recordSetProvider is null"); + } + + @Override + public ConnectorTransactionHandle beginTransaction(IsolationLevel isolationLevel, boolean readOnly, boolean autoCommit) + { + return INSTANCE; + } + + @Override + public ConnectorMetadata getMetadata(ConnectorSession session, ConnectorTransactionHandle transactionHandle) + { + return metadata; + } + + @Override + public ConnectorSplitManager getSplitManager() + { + return splitManager; + } + + @Override + public ConnectorRecordSetProvider getRecordSetProvider() + { + return recordSetProvider; + } + + @Override + public final void shutdown() + { + lifeCycleManager.stop(); + } +} diff --git a/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxConnectorFactory.java b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxConnectorFactory.java new file mode 100644 index 000000000000..72412de9c46d --- /dev/null +++ b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxConnectorFactory.java @@ -0,0 +1,58 @@ +/* + * 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.influxdb; + +import com.google.inject.Injector; +import io.airlift.bootstrap.Bootstrap; +import io.airlift.json.JsonModule; +import io.trino.plugin.base.TypeDeserializerModule; +import io.trino.spi.connector.Connector; +import io.trino.spi.connector.ConnectorContext; +import io.trino.spi.connector.ConnectorFactory; + +import java.util.Map; + +import static io.trino.plugin.base.Versions.checkStrictSpiVersionMatch; +import static java.util.Objects.requireNonNull; + +public class InfluxConnectorFactory + implements ConnectorFactory +{ + @Override + public String getName() + { + return "influxdb"; + } + + @Override + public Connector create(String catalogName, Map requiredConfig, ConnectorContext context) + { + requireNonNull(requiredConfig, "requiredConfig is null"); + checkStrictSpiVersionMatch(context, this); + + // A plugin is not required to use Guice; it is just very convenient + Bootstrap app = new Bootstrap( + new JsonModule(), + new TypeDeserializerModule(context.getTypeManager()), + new InfluxModule()); + + Injector injector = app + .doNotInitializeLogging() + .setRequiredConfigurationProperties(requiredConfig) + .initialize(); + + return injector.getInstance(InfluxConnector.class); + } +} diff --git a/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxConstant.java b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxConstant.java new file mode 100644 index 000000000000..43fe3a3629a7 --- /dev/null +++ b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxConstant.java @@ -0,0 +1,42 @@ +/* + * 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.influxdb; + +public class InfluxConstant +{ + public enum ColumnName + { + TIME("time"); + + private final String name; + + ColumnName(String name) + { + this.name = name; + } + + public String getName() + { + return name; + } + } + + public enum ColumnKind + { + TIME, + TAG, + FIELD, + } +} diff --git a/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxMetadata.java b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxMetadata.java new file mode 100644 index 000000000000..f02256ae78f7 --- /dev/null +++ b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxMetadata.java @@ -0,0 +1,348 @@ +/* + * 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.influxdb; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.inject.Inject; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.Assignment; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.connector.ColumnMetadata; +import io.trino.spi.connector.ConnectorMetadata; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorTableHandle; +import io.trino.spi.connector.ConnectorTableMetadata; +import io.trino.spi.connector.ConnectorTableVersion; +import io.trino.spi.connector.Constraint; +import io.trino.spi.connector.ConstraintApplicationResult; +import io.trino.spi.connector.LimitApplicationResult; +import io.trino.spi.connector.ProjectionApplicationResult; +import io.trino.spi.connector.SchemaNotFoundException; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.connector.SchemaTablePrefix; +import io.trino.spi.connector.TableColumnsMetadata; +import io.trino.spi.connector.TableNotFoundException; +import io.trino.spi.expression.ConnectorExpression; +import io.trino.spi.predicate.Domain; +import io.trino.spi.predicate.EquatableValueSet; +import io.trino.spi.predicate.Range; +import io.trino.spi.predicate.SortedRangeSet; +import io.trino.spi.predicate.TupleDomain; +import io.trino.spi.security.TrinoPrincipal; +import io.trino.spi.type.TimestampType; +import jakarta.annotation.Nullable; +import org.influxdb.dto.QueryResult; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.OptionalInt; +import java.util.Set; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.plugin.influxdb.TypeUtils.isPushdownSupportedType; +import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; +import static io.trino.spi.StandardErrorCode.SCHEMA_NOT_EMPTY; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.BooleanType.BOOLEAN; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static java.lang.Math.toIntExact; +import static java.util.Objects.requireNonNull; + +public class InfluxMetadata + implements ConnectorMetadata +{ + private final InfluxClient client; + + @Inject + public InfluxMetadata(InfluxClient client) + { + this.client = requireNonNull(client, "client is null"); + } + + @Override + public List listSchemaNames(ConnectorSession session) + { + return ImmutableList.copyOf(client.getSchemaNames()); + } + + @Override + public List listTables(ConnectorSession session, Optional optionalSchemaName) + { + Set schemaNames = optionalSchemaName.map(ImmutableSet::of) + .orElseGet(() -> ImmutableSet.copyOf(client.getSchemaNames())); + + return schemaNames.stream() + .flatMap(schemaName -> client.getSchemaTableNames(schemaName).stream()) + .collect(toImmutableList()); + } + + @Nullable + @Override + public ConnectorTableHandle getTableHandle( + ConnectorSession session, + SchemaTableName schemaTableName, + Optional startVersion, + Optional endVersion) + { + if (startVersion.isPresent() || endVersion.isPresent()) { + throw new TrinoException(NOT_SUPPORTED, "This connector does not support versioned tables"); + } + + return getTableHandle(session, schemaTableName); + } + + @Nullable + public ConnectorTableHandle getTableHandle(ConnectorSession session, SchemaTableName schemaTableName) + { + return client.getTableHandle(schemaTableName.getSchemaName(), schemaTableName.getTableName()) + .map(_ -> new InfluxTableHandle(schemaTableName.getSchemaName(), schemaTableName.getTableName())).orElse(null); + } + + @Override + public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle tableHandle) + { + return getTableMetadata(((InfluxTableHandle) tableHandle).toSchemaTableName()) + .orElseThrow(() -> new RuntimeException("The table handle is invalid " + tableHandle)); + } + + @Override + public Map getColumnHandles(ConnectorSession session, ConnectorTableHandle tableHandle) + { + InfluxTableHandle influxTableHandle = (InfluxTableHandle) tableHandle; + + InfluxTableHandle table = client.getTableHandle(influxTableHandle.getSchemaName(), influxTableHandle.getTableName()) + .orElseThrow(() -> new TableNotFoundException(influxTableHandle.toSchemaTableName())); + + ImmutableMap.Builder columnHandles = + ImmutableMap.builderWithExpectedSize(table.getColumns().size()); + for (InfluxColumnHandle column : table.getColumns()) { + columnHandles.put(column.getName(), column); + } + return columnHandles.buildOrThrow(); + } + + @Override + public ColumnMetadata getColumnMetadata(ConnectorSession session, ConnectorTableHandle tableHandle, ColumnHandle columnHandle) + { + InfluxColumnHandle influxColumnHandle = (InfluxColumnHandle) columnHandle; + return new ColumnMetadata(influxColumnHandle.getName(), influxColumnHandle.getType()); + } + + @Override + public Iterator streamTableColumns(ConnectorSession session, SchemaTablePrefix prefix) + { + requireNonNull(prefix, "prefix is null"); + ImmutableMap.Builder> columns = ImmutableMap.builder(); + for (SchemaTableName tableName : listTables(session, prefix.getSchema())) { + if (!prefix.matches(tableName)) { + continue; + } + getTableMetadata(tableName).ifPresent(tableMetadata -> columns.put(tableName, tableMetadata.getColumns())); + } + return columns.buildOrThrow().entrySet().stream() + .map(entry -> TableColumnsMetadata.forTable(entry.getKey(), entry.getValue())).iterator(); + } + + @Override + public Optional> applyLimit( + ConnectorSession session, + ConnectorTableHandle handle, + long limit) + { + InfluxTableHandle tableHandle = (InfluxTableHandle) handle; + // InfluxQL Limit 0 is equivalent to setting no limit + if (limit == 0) { + return Optional.empty(); + } + // InfluxQL doesn't support limit number greater than integer max + if (limit > Integer.MAX_VALUE) { + return Optional.empty(); + } + if (tableHandle.getLimit().isPresent() && tableHandle.getLimit().getAsInt() <= limit) { + return Optional.empty(); + } + + return Optional.of(new LimitApplicationResult<>( + tableHandle.withLimit(OptionalInt.of(toIntExact(limit))), + true, + false)); + } + + @Override + public void dropTable(ConnectorSession session, ConnectorTableHandle handle) + { + InfluxTableHandle tableHandle = (InfluxTableHandle) handle; + if (client.getTableHandle(tableHandle.getSchemaName(), tableHandle.getTableName()).isEmpty()) { + throw new TableNotFoundException(tableHandle.toSchemaTableName()); + } + + client.dropTable(tableHandle.getSchemaName(), tableHandle.getTableName()); + } + + @Override + public void dropSchema(ConnectorSession session, String schemaName, boolean cascade) + { + if (!client.getSchemaNames().contains(schemaName)) { + throw new SchemaNotFoundException(schemaName); + } + + if (!cascade) { + boolean isEmpty = listTables(session, Optional.of(schemaName)).isEmpty(); + if (!isEmpty) { + throw new TrinoException(SCHEMA_NOT_EMPTY, "Cannot drop non-empty schema '%s'".formatted(schemaName)); + } + } + client.dropDatabase(schemaName); + } + + @Override + public void createSchema(ConnectorSession session, String schemaName, Map properties, TrinoPrincipal owner) + { + QueryResult result = client.createDatabase(schemaName); + if (!result.getResults().getFirst().hasError()) { + return; + } + throw new TrinoException(NOT_SUPPORTED, "Failed to create schema: " + result.getResults().getFirst().getError()); + } + + @Override + public Optional> applyFilter( + ConnectorSession session, + ConnectorTableHandle handle, + Constraint constraint) + { + InfluxTableHandle tableHandle = (InfluxTableHandle) handle; + TupleDomain oldDomain = tableHandle.getConstraint(); + TupleDomain newDomain = oldDomain.intersect(constraint.getSummary()); + TupleDomain remainingFilter; + if (newDomain.isNone()) { + remainingFilter = TupleDomain.all(); + } + else { + Map domains = newDomain.getDomains().orElseThrow(); + Map supported = new HashMap<>(); + Map unsupported = new HashMap<>(); + domains.forEach((key, domain) -> { + if (isPushdownSupportedType(((InfluxColumnHandle) key).getType()) + && isPushdownSupportedDomain(domain)) { + supported.put(key, domain); + } + else { + unsupported.put(key, domain); + } + }); + newDomain = TupleDomain.withColumnDomains(supported); + remainingFilter = TupleDomain.withColumnDomains(unsupported); + } + + if (oldDomain.equals(newDomain)) { + return Optional.empty(); + } + + return Optional.of(new ConstraintApplicationResult<>( + tableHandle.withConstraint(newDomain), + remainingFilter, + constraint.getExpression(), + false)); + } + + @Override + public Optional> applyProjection( + ConnectorSession session, + ConnectorTableHandle handle, + List projections, + Map assignments) + { + InfluxTableHandle tableHandle = (InfluxTableHandle) handle; + List oldProjections = ((InfluxTableHandle) handle).getProjections(); + List newProjections = ImmutableList.copyOf(assignments.values()); + if (oldProjections.equals(newProjections)) { + return Optional.empty(); + } + + List assignmentsList = assignments.entrySet().stream() + .map(assignment -> new Assignment( + assignment.getKey(), + assignment.getValue(), + ((InfluxColumnHandle) assignment.getValue()).getType())) + .collect(toImmutableList()); + return Optional.of(new ProjectionApplicationResult<>( + tableHandle.withProjections(newProjections), + projections, + assignmentsList, + false)); + } + + private Optional getTableMetadata(SchemaTableName schemaTableName) + { + Optional tableHandle = client.getTableHandle(schemaTableName.getSchemaName(), schemaTableName.getTableName()); + + return tableHandle.map(table -> { + ImmutableList.Builder columnMetadataBuilder = + ImmutableList.builderWithExpectedSize(table.getColumns().size()); + List columns = table.getColumns(); + for (InfluxColumnHandle column : columns) { + columnMetadataBuilder.add(new ColumnMetadata(column.getName(), column.getType())); + } + return new ConnectorTableMetadata(schemaTableName, columnMetadataBuilder.build()); + }); + } + + private boolean isPushdownSupportedDomain(Domain domain) + { + if (domain.getValues() instanceof SortedRangeSet rangeSet) { + if (rangeSet.getOrderedRanges().isEmpty()) { + return false; + } + List ranges = rangeSet.getOrderedRanges(); + return isRangeSupportsTimestamp(ranges) || + isRangeSupportsBoolean(ranges) || + isRangeSupportsNumber(ranges) || + isRangeSupportsVarchar(ranges); + } + else if (domain.getValues() instanceof EquatableValueSet valueSet) { + return !valueSet.getDiscreteSet().isEmpty(); + } + return false; + } + + private static boolean isRangeSupportsVarchar(List ranges) + { + return ranges.stream().allMatch(range -> range.getType() == VARCHAR) && + ranges.stream().anyMatch(Range::isSingleValue); + } + + private static boolean isRangeSupportsBoolean(List ranges) + { + return ranges.stream().allMatch(range -> range.getType() == BOOLEAN) && + ranges.stream().anyMatch(Range::isSingleValue); + } + + private static boolean isRangeSupportsNumber(List ranges) + { + return ranges.stream().allMatch(range -> range.getType() == DOUBLE || range.getType() == BIGINT); + } + + private static boolean isRangeSupportsTimestamp(List ranges) + { + return ranges.stream().allMatch(range -> range.getType() instanceof TimestampType); + } +} diff --git a/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxModule.java b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxModule.java new file mode 100644 index 000000000000..d55bb1fd2ed3 --- /dev/null +++ b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxModule.java @@ -0,0 +1,38 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.influxdb; + +import com.google.inject.Binder; +import com.google.inject.Module; + +import static com.google.inject.Scopes.SINGLETON; +import static io.airlift.configuration.ConfigBinder.configBinder; + +public class InfluxModule + implements Module +{ + @Override + public void configure(Binder binder) + { + configBinder(binder).bindConfig(InfluxConfig.class); + + binder.bind(InfluxClient.class).to(NativeInfluxClient.class).in(SINGLETON); + binder.bind(NativeInfluxClient.class).in(SINGLETON); + binder.bind(InfluxMetadata.class).in(SINGLETON); + binder.bind(InfluxSplitManager.class).in(SINGLETON); + binder.bind(InfluxRecordSetProvider.class).in(SINGLETON); + binder.bind(InfluxConnector.class).in(SINGLETON); + } +} diff --git a/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxPlugin.java b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxPlugin.java new file mode 100644 index 000000000000..800c0754389b --- /dev/null +++ b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxPlugin.java @@ -0,0 +1,28 @@ +/* + * 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.influxdb; + +import com.google.common.collect.ImmutableList; +import io.trino.spi.Plugin; +import io.trino.spi.connector.ConnectorFactory; + +public class InfluxPlugin + implements Plugin +{ + @Override + public Iterable getConnectorFactories() + { + return ImmutableList.of(new InfluxConnectorFactory()); + } +} diff --git a/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxRecord.java b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxRecord.java new file mode 100644 index 000000000000..1bcacf6224ca --- /dev/null +++ b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxRecord.java @@ -0,0 +1,59 @@ +/* + * 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.influxdb; + +import java.util.List; +import java.util.Objects; + +public class InfluxRecord +{ + private final List columns; + private final List> values; + + public InfluxRecord(List columns, List> values) + { + this.columns = columns; + this.values = values; + } + + public List getColumns() + { + return columns; + } + + public List> getValues() + { + return values; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + InfluxRecord that = (InfluxRecord) o; + return Objects.equals(columns, that.columns) && Objects.equals(values, that.values); + } + + @Override + public int hashCode() + { + return Objects.hash(columns, values); + } +} diff --git a/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxRecordCursor.java b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxRecordCursor.java new file mode 100644 index 000000000000..9135bb102f16 --- /dev/null +++ b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxRecordCursor.java @@ -0,0 +1,198 @@ +/* + * 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.influxdb; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import io.airlift.slice.Slice; +import io.trino.spi.connector.RecordCursor; +import io.trino.spi.type.Type; + +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.IntStream; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.airlift.slice.Slices.utf8Slice; +import static io.trino.plugin.influxdb.InfluxConstant.ColumnName.TIME; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.BooleanType.BOOLEAN; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static io.trino.spi.type.TimestampType.TIMESTAMP_NANOS; +import static io.trino.spi.type.VarcharType.createUnboundedVarcharType; +import static java.time.format.DateTimeFormatter.ISO_DATE_TIME; +import static java.util.function.Function.identity; +import static java.util.stream.Collectors.toMap; + +public class InfluxRecordCursor + implements RecordCursor +{ + private long totalBytes; + private final List columnHandles; + private final Iterator> sourceDataIterator; + private List fieldValues; + + public InfluxRecordCursor(List columnHandles, InfluxRecord sourceData) + { + this.columnHandles = ImmutableList.copyOf(columnHandles); + List columns; + List> recordSet; + + //even column not contains 'time', it still returns as the first field in influx-client query result + boolean noTimeCol = columnHandles.stream().noneMatch(columnHandle -> columnHandle.getName().equals(TIME.getName())); + if (noTimeCol && sourceData.getColumns().contains(TIME.getName())) { + columns = sourceData.getColumns().subList(1, sourceData.getColumns().size()); + recordSet = sourceData.getValues().stream().map(record -> record.subList(1, record.size())).collect(toImmutableList()); + } + else { + columns = sourceData.getColumns(); + recordSet = sourceData.getValues(); + } + + //align source data columns index according to column handles + List> alignedRecordSet = Lists.newArrayListWithExpectedSize(recordSet.size()); + Map nameIdxMap = IntStream.range(0, columns.size()).boxed().collect(toMap(columns::get, identity())); + List names = columnHandles.stream().map(InfluxColumnHandle::getName).toList(); + for (List record : recordSet) { + List alignedRecord = Lists.newArrayListWithExpectedSize(names.size()); + for (String name : names) { + Object value = record.get(nameIdxMap.get(name)); + alignedRecord.add(value); + } + alignedRecordSet.add(alignedRecord); + } + + this.sourceDataIterator = alignedRecordSet.iterator(); + } + + @Override + public long getCompletedBytes() + { + return totalBytes; + } + + @Override + public long getReadTimeNanos() + { + return 0; + } + + @Override + public Type getType(int field) + { + checkArgument(field < columnHandles.size(), "Invalid field index"); + return columnHandles.get(field).getType(); + } + + @Override + public boolean advanceNextPosition() + { + if (!sourceDataIterator.hasNext()) { + return false; + } + fieldValues = sourceDataIterator.next(); + totalBytes += fieldValues.size(); + return true; + } + + private Object getFieldValue(int field) + { + checkState(fieldValues != null, "Cursor has not been advanced yet"); + return fieldValues.get(field); + } + + @Override + public boolean getBoolean(int field) + { + checkFieldType(field, BOOLEAN); + Object value = getFieldValue(field); + if (value instanceof Boolean) { + return (Boolean) value; + } + if (value instanceof Number number) { + return number.intValue() != 0; + } + return false; + } + + @Override + public long getLong(int field) + { + checkFieldType(field, BIGINT); + Object value = getFieldValue(field); + if (value instanceof Number number) { + return number.longValue(); + } + return 0; + } + + @Override + public double getDouble(int field) + { + checkFieldType(field, DOUBLE); + Object value = getFieldValue(field); + if (value instanceof Number) { + return ((Number) value).doubleValue(); + } + return 0; + } + + @Override + public Slice getSlice(int field) + { + checkFieldType(field, createUnboundedVarcharType()); + Object value = getFieldValue(field); + return utf8Slice((String) value); + } + + @Override + public Object getObject(int field) + { + Type type = getType(field); + Object value = getFieldValue(field); + if (type == TIMESTAMP_NANOS) { + Instant utc = LocalDateTime.parse((String) value, ISO_DATE_TIME).atZone(ZoneId.of("UTC")).toInstant(); + return TimestampUtils.longTimestamp(utc); + } + else { + throw new UnsupportedOperationException(); + } + } + + @Override + public boolean isNull(int field) + { + checkArgument(field < columnHandles.size(), "Invalid field index"); + return Objects.isNull(getFieldValue(field)); + } + + private void checkFieldType(int field, Type expected) + { + Type actual = getType(field); + checkArgument(expected.equals(actual), "Expected field %s to be type %s but is %s", field, expected, actual); + } + + @Override + public void close() + { + } +} diff --git a/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxRecordSet.java b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxRecordSet.java new file mode 100644 index 000000000000..485731576876 --- /dev/null +++ b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxRecordSet.java @@ -0,0 +1,72 @@ +/* + * 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.influxdb; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; +import io.trino.spi.connector.RecordCursor; +import io.trino.spi.connector.RecordSet; +import io.trino.spi.type.Type; +import org.influxdb.dto.Query; + +import java.util.List; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.plugin.influxdb.QueryUtils.buildQueryCommand; +import static java.util.Objects.requireNonNull; + +public class InfluxRecordSet + implements RecordSet +{ + private final InfluxRecord sourceData; + private final List columnHandles; + private final List columnTypes; + + public InfluxRecordSet(InfluxTableHandle tableHandle, List columnHandles, InfluxClient client) + { + requireNonNull(tableHandle, "tableHandle is null"); + requireNonNull(columnHandles, "columnHandles is null"); + + this.columnHandles = ImmutableList.copyOf(columnHandles); + this.columnTypes = columnHandles.stream() + .map(InfluxColumnHandle::getType) + .collect(toImmutableList()); + this.sourceData = querySourceData(client, tableHandle, columnHandles); + } + + private InfluxRecord querySourceData(InfluxClient client, InfluxTableHandle tableHandle, List columnHandles) + { + Query query = buildQueryCommand(tableHandle, columnHandles); + return client.query(query); + } + + @VisibleForTesting + public InfluxRecord getSourceData() + { + return sourceData; + } + + @Override + public List getColumnTypes() + { + return columnTypes; + } + + @Override + public RecordCursor cursor() + { + return new InfluxRecordCursor(columnHandles, sourceData); + } +} diff --git a/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxRecordSetProvider.java b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxRecordSetProvider.java new file mode 100644 index 000000000000..43a71b51b736 --- /dev/null +++ b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxRecordSetProvider.java @@ -0,0 +1,57 @@ +/* + * 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.influxdb; + +import com.google.inject.Inject; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.connector.ConnectorRecordSetProvider; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorSplit; +import io.trino.spi.connector.ConnectorTableHandle; +import io.trino.spi.connector.ConnectorTransactionHandle; +import io.trino.spi.connector.RecordSet; + +import java.util.List; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static java.util.Objects.requireNonNull; + +public class InfluxRecordSetProvider + implements ConnectorRecordSetProvider +{ + private final InfluxClient client; + + @Inject + public InfluxRecordSetProvider(InfluxClient client) + { + this.client = requireNonNull(client, "client is null"); + } + + @Override + public RecordSet getRecordSet( + ConnectorTransactionHandle transaction, + ConnectorSession session, + ConnectorSplit split, + ConnectorTableHandle table, + List columns) + { + InfluxTableHandle tableHandle = (InfluxTableHandle) table; + List columnHandles = columns.stream() + .map(InfluxColumnHandle.class::cast) + .collect(toImmutableList()); + + return new InfluxRecordSet(tableHandle, columnHandles, client); + } +} diff --git a/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxSplit.java b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxSplit.java new file mode 100644 index 000000000000..886428b7d26c --- /dev/null +++ b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxSplit.java @@ -0,0 +1,57 @@ +/* + * 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.influxdb; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.airlift.slice.SizeOf; +import io.trino.spi.HostAddress; +import io.trino.spi.connector.ConnectorSplit; + +import java.util.List; +import java.util.Map; + +import static io.airlift.slice.SizeOf.estimatedSizeOf; +import static java.util.stream.Collectors.joining; + +public record InfluxSplit(List addresses) + implements ConnectorSplit +{ + private static final long INSTANCE_SIZE = SizeOf.instanceSize(InfluxSplit.class); + + public InfluxSplit() + { + this(ImmutableList.of()); + } + + @Override + public List getAddresses() + { + return addresses; + } + + @Override + public Map getSplitInfo() + { + return ImmutableMap.of("addresses", addresses.stream().map(HostAddress::toString).collect(joining(","))); + } + + @Override + public long getRetainedSizeInBytes() + { + return INSTANCE_SIZE + + estimatedSizeOf(addresses, HostAddress::getRetainedSizeInBytes); + } +} diff --git a/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxSplitManager.java b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxSplitManager.java new file mode 100644 index 000000000000..91942dcc33be --- /dev/null +++ b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxSplitManager.java @@ -0,0 +1,40 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.influxdb; + +import com.google.common.collect.ImmutableList; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.ConnectorSplitManager; +import io.trino.spi.connector.ConnectorSplitSource; +import io.trino.spi.connector.ConnectorTableHandle; +import io.trino.spi.connector.ConnectorTransactionHandle; +import io.trino.spi.connector.Constraint; +import io.trino.spi.connector.DynamicFilter; +import io.trino.spi.connector.FixedSplitSource; + +public class InfluxSplitManager + implements ConnectorSplitManager +{ + @Override + public ConnectorSplitSource getSplits( + ConnectorTransactionHandle transaction, + ConnectorSession session, + ConnectorTableHandle connectorTableHandle, + DynamicFilter dynamicFilter, + Constraint constraint) + { + return new FixedSplitSource(ImmutableList.of(new InfluxSplit())); + } +} diff --git a/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxTableHandle.java b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxTableHandle.java new file mode 100644 index 000000000000..24d924d2d0fc --- /dev/null +++ b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxTableHandle.java @@ -0,0 +1,179 @@ +/* + * 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.influxdb; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.connector.ConnectorTableHandle; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.predicate.TupleDomain; + +import java.util.List; +import java.util.Objects; +import java.util.OptionalInt; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +public class InfluxTableHandle + implements ConnectorTableHandle +{ + private final String schemaName; + private final String tableName; + private final List columns; + private final TupleDomain constraint; + private final List projections; + private final OptionalInt limit; + + public InfluxTableHandle(String schemaName, String tableName) + { + this(schemaName, tableName, ImmutableList.of(), TupleDomain.all(), ImmutableList.of(), OptionalInt.empty()); + } + + public InfluxTableHandle(String schemaName, String tableName, List columns) + { + this(schemaName, tableName, columns, TupleDomain.all(), ImmutableList.of(), OptionalInt.empty()); + } + + @JsonCreator + public InfluxTableHandle( + @JsonProperty("schemaName") String schemaName, + @JsonProperty("tableName") String tableName, + @JsonProperty("columns") List columns, + @JsonProperty("constraint") TupleDomain constraint, + @JsonProperty("projections") List projections, + @JsonProperty("limit") OptionalInt limit) + { + this.schemaName = requireNonNull(schemaName, "schemaName is null"); + this.tableName = requireNonNull(tableName, "tableName is null"); + this.constraint = requireNonNull(constraint, "constraint is null"); + this.columns = ImmutableList.copyOf(requireNonNull(columns, "columns is null")); + this.projections = ImmutableList.copyOf(requireNonNull(projections, "projections is null")); + this.limit = requireNonNull(limit, "limit is null"); + } + + public InfluxTableHandle withProjections(List projections) + { + return new InfluxTableHandle( + schemaName, + tableName, + columns, + constraint, + projections, + limit); + } + + public InfluxTableHandle withConstraint(TupleDomain constraint) + { + return new InfluxTableHandle( + schemaName, + tableName, + columns, + constraint, + projections, + limit); + } + + public InfluxTableHandle withLimit(OptionalInt limit) + { + return new InfluxTableHandle( + schemaName, + tableName, + columns, + constraint, + projections, + limit); + } + + @JsonProperty + public String getSchemaName() + { + return schemaName; + } + + @JsonProperty + public String getTableName() + { + return tableName; + } + + @JsonProperty + public List getColumns() + { + return columns; + } + + @JsonProperty + public TupleDomain getConstraint() + { + return constraint; + } + + @JsonProperty + public List getProjections() + { + return projections; + } + + @JsonProperty + public OptionalInt getLimit() + { + return limit; + } + + public SchemaTableName toSchemaTableName() + { + return new SchemaTableName(schemaName, tableName); + } + + @Override + public int hashCode() + { + return Objects.hash(schemaName, tableName); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + InfluxTableHandle that = (InfluxTableHandle) o; + return Objects.equals(schemaName, that.schemaName) && + Objects.equals(tableName, that.tableName) && + Objects.equals(columns, that.columns) && + Objects.equals(constraint, that.constraint) && + Objects.equals(projections, that.projections) && + Objects.equals(limit, that.limit); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("schemaName", schemaName) + .add("tableName", tableName) + .add("columns", columns) + .add("constraint", constraint) + .add("projections", projections) + .add("limit", limit) + .toString(); + } +} diff --git a/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxTransactionHandle.java b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxTransactionHandle.java new file mode 100644 index 000000000000..11edc4613299 --- /dev/null +++ b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/InfluxTransactionHandle.java @@ -0,0 +1,23 @@ +/* + * 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.influxdb; + +import io.trino.spi.connector.ConnectorTransactionHandle; + +public enum InfluxTransactionHandle + implements ConnectorTransactionHandle +{ + INSTANCE +} diff --git a/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/NativeInfluxClient.java b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/NativeInfluxClient.java new file mode 100644 index 000000000000..7cc78fb924e9 --- /dev/null +++ b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/NativeInfluxClient.java @@ -0,0 +1,282 @@ +/* + * 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.influxdb; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Sets; +import com.google.inject.Inject; +import io.airlift.log.Logger; +import io.trino.spi.connector.SchemaTableName; +import okhttp3.ConnectionPool; +import okhttp3.OkHttpClient; +import okhttp3.logging.HttpLoggingInterceptor; +import org.influxdb.InfluxDB; +import org.influxdb.InfluxDBFactory; +import org.influxdb.dto.Query; +import org.influxdb.dto.QueryResult; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.Set; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static com.google.common.collect.Iterables.getOnlyElement; +import static io.trino.plugin.influxdb.InfluxConstant.ColumnKind.FIELD; +import static io.trino.plugin.influxdb.InfluxConstant.ColumnKind.TAG; +import static io.trino.plugin.influxdb.InfluxConstant.ColumnKind.TIME; +import static io.trino.plugin.influxdb.InfluxConstant.ColumnName; +import static io.trino.plugin.influxdb.TypeUtils.TIMESTAMP; +import static io.trino.plugin.influxdb.TypeUtils.toTrinoType; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static java.util.Locale.ENGLISH; +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.MINUTES; +import static okhttp3.logging.HttpLoggingInterceptor.Level.NONE; +import static org.influxdb.dto.QueryResult.Result; +import static org.influxdb.dto.QueryResult.Series; + +public class NativeInfluxClient + implements InfluxClient +{ + private static final Logger log = Logger.get(NativeInfluxClient.class); + + private static final List INTERNAL_SCHEMAS = ImmutableList.of("_internal"); + private static final String SHOW_DATABASE_CMD = "SHOW DATABASES"; + private static final String SHOW_MEASUREMENTS_CMD = "SHOW MEASUREMENTS"; + private static final String SHOW_TAG_KEYS_CMD = "SHOW TAG KEYS FROM \"$measurement\""; + private static final String SHOW_FIELD_KEYS_CMD = "SHOW FIELD KEYS FROM \"$measurement\""; + private final InfluxDB client; + + @Inject + public NativeInfluxClient(InfluxConfig config) + { + OkHttpClient.Builder httpBuilder = new OkHttpClient.Builder() + .connectionPool(new ConnectionPool(10, 10, MINUTES)) + .connectTimeout(config.getConnectTimeOut().toMillis(), MILLISECONDS) + .readTimeout(config.getReadTimeOut().toMillis(), MILLISECONDS) + .addInterceptor(new HttpLoggingInterceptor().setLevel(NONE)); + + if (config.getUsername().isEmpty() || config.getPassword().isEmpty()) { + this.client = InfluxDBFactory.connect(config.getEndpoint(), httpBuilder); + } + else { + this.client = InfluxDBFactory.connect(config.getEndpoint(), config.getUsername().get(), config.getPassword().get(), httpBuilder); + } + } + + @Override + public InfluxRecord query(Query query) + { + try { + QueryResult queryResult = client.query(query); + requireNonNull(queryResult); + + if (queryResult.getResults() == null || queryResult.getResults().isEmpty()) { + return new InfluxRecord(ImmutableList.of(), ImmutableList.of()); + } + Result result = getOnlyElement(queryResult.getResults()); + if (result.getSeries() == null || result.getSeries().isEmpty()) { + return new InfluxRecord(ImmutableList.of(), ImmutableList.of()); + } + + if (result.getSeries().size() == 1) { + Series series = getOnlyElement(result.getSeries()); + return new InfluxRecord(series.getColumns(), series.getValues()); + } + else { + ImmutableList.Builder columns = ImmutableList.builder(); + List> values = new ArrayList<>(); + Series firstSeries = result.getSeries().get(0); + columns.addAll(firstSeries.getColumns()); + columns.addAll(firstSeries.getTags().keySet()); + for (Series series : result.getSeries()) { + List value = new ArrayList<>(); + value.addAll(series.getValues().get(0)); + value.addAll(series.getTags().values()); + values.add(value); + } + return new InfluxRecord(columns.build(), values); + } + } + catch (Throwable e) { + log.error("InfluxDB query error: %s.", e); + throw new RuntimeException(e); + } + } + + @Override + public List getSchemaNames() + { + QueryResult result = client.query(new Query(SHOW_DATABASE_CMD)); + requireNonNull(result); + + List results = result.getResults(); + if (results == null || results.isEmpty()) { + return ImmutableList.of(); + } + List series = getOnlyElement(results).getSeries(); + if (series == null || series.isEmpty()) { + return ImmutableList.of(); + } + List> databases = getOnlyElement(series).getValues(); + if (databases == null || databases.isEmpty()) { + return ImmutableList.of(); + } + return databases.stream() + .map(database -> getOnlyElement(database).toString()) + .filter(name -> !INTERNAL_SCHEMAS.contains(name.toLowerCase(ENGLISH))) + .collect(toImmutableList()); + } + + @Override + public List getSchemaTableNames(String schemaName) + { + QueryResult result = client.query(new Query(SHOW_MEASUREMENTS_CMD, schemaName)); + requireNonNull(result); + + List results = result.getResults(); + if (results == null || results.isEmpty()) { + return ImmutableList.of(); + } + List series = getOnlyElement(results).getSeries(); + if (series == null || series.isEmpty()) { + return ImmutableList.of(); + } + List> measurements = getOnlyElement(series).getValues(); + if (measurements == null || measurements.isEmpty()) { + return ImmutableList.of(); + } + return measurements.stream() + .map(measurement -> getOnlyElement(measurement).toString()) + .distinct() + .map(measurement -> new SchemaTableName(schemaName, measurement)) + .collect(toImmutableList()); + } + + @Override + public Optional getTableHandle(String schemaName, String tableName) + { + requireNonNull(schemaName, "schemaName is null"); + requireNonNull(tableName, "tableName is null"); + + if (!this.getSchemaTableNames(schemaName).contains(new SchemaTableName(schemaName, tableName))) { + return Optional.empty(); + } + + String fieldKeysCommand = SHOW_FIELD_KEYS_CMD.replace("$measurement", tableName); + String tagKeysCommand = SHOW_TAG_KEYS_CMD.replace("$measurement", tableName); + QueryResult fieldResult = client.query(new Query(fieldKeysCommand, schemaName)); + QueryResult tagResult = client.query(new Query(tagKeysCommand, schemaName)); + + List columns = this.buildColumns(fieldResult, tagResult); + return Optional.of(new InfluxTableHandle(schemaName, tableName, columns)); + } + + @Override + public void dropTable(String schemaName, String tableName) + { + requireNonNull(schemaName, "schemaName is null"); + requireNonNull(tableName, "tableName is null"); + + client.query(new Query("DROP MEASUREMENT " + tableName, schemaName)); + } + + @Override + public QueryResult createDatabase(String databaseName) + { + requireNonNull(databaseName, "databaseName is null"); + + return client.query(new Query("CREATE DATABASE " + databaseName, databaseName, true)); + } + + @Override + public void dropDatabase(String databaseName) + { + requireNonNull(databaseName, "databaseName is null"); + + client.query(new Query("DROP DATABASE " + databaseName)); + } + + private List buildColumns(QueryResult fieldResult, QueryResult tagResult) + { + ImmutableList.Builder columnBuilder = ImmutableList.builder(); + columnBuilder.add(new InfluxColumnHandle(ColumnName.TIME.getName(), toTrinoType(TIMESTAMP), TIME)); + + if (fieldResult.getResults() != null && !fieldResult.getResults().isEmpty()) { + ImmutableList columnHandles = buildInfluxFieldColumns(fieldResult.getResults()); + columnBuilder.addAll(columnHandles); + } + if (tagResult.getResults() != null && !tagResult.getResults().isEmpty()) { + ImmutableList columnHandles = buildInfluxTagColumns(tagResult.getResults()); + columnBuilder.addAll(columnHandles); + } + return columnBuilder.build(); + } + + private ImmutableList buildInfluxFieldColumns(List result) + { + return buildInfluxColumns(getOnlyElement(result), false, true); + } + + private ImmutableList buildInfluxTagColumns(List result) + { + return buildInfluxColumns(getOnlyElement(result), true, false); + } + + private ImmutableList buildInfluxColumns(Result result, boolean isTag, boolean isField) + { + ImmutableList.Builder columnBuilder = ImmutableList.builder(); + + List series = result.getSeries(); + if (series == null || series.isEmpty()) { + return ImmutableList.of(); + } + List columns = getOnlyElement(series).getColumns(); + List> values = getOnlyElement(series).getValues(); + if (columns == null || columns.isEmpty() || values == null || values.isEmpty()) { + return ImmutableList.of(); + } + + if (isField) { + Set fieldSetKeys = Sets.newHashSetWithExpectedSize(values.size()); + for (List fields : values) { + String fieldKey = (String) fields.get(0); + String fieldType = (String) fields.get(1); + String fieldSetKey = fieldKey.toLowerCase(ENGLISH); + if (!fieldSetKeys.contains(fieldSetKey)) { + columnBuilder.add(new InfluxColumnHandle(fieldKey, toTrinoType(fieldType), FIELD)); + fieldSetKeys.add(fieldSetKey); + } + } + } + if (isTag) { + Set tagSetKeys = Sets.newHashSetWithExpectedSize(values.size()); + for (List tags : values) { + for (Object tag : tags) { + String tagKey = String.valueOf(tag); + String tagSetKey = tagKey.toLowerCase(ENGLISH); + if (!tagSetKeys.contains(tagSetKey)) { + columnBuilder.add(new InfluxColumnHandle(tagKey, VARCHAR, TAG)); + tagSetKeys.add(tagSetKey); + } + } + } + } + + return columnBuilder.build(); + } +} diff --git a/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/QueryUtils.java b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/QueryUtils.java new file mode 100644 index 000000000000..9ee8826c5acd --- /dev/null +++ b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/QueryUtils.java @@ -0,0 +1,239 @@ +/* + * 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.influxdb; + +import com.google.common.collect.ImmutableList; +import io.airlift.slice.Slice; +import io.trino.spi.connector.ColumnHandle; +import io.trino.spi.predicate.DiscreteValues; +import io.trino.spi.predicate.Domain; +import io.trino.spi.predicate.Range; +import io.trino.spi.predicate.Ranges; +import io.trino.spi.predicate.TupleDomain; +import io.trino.spi.type.LongTimestamp; +import io.trino.spi.type.TimestampType; +import org.influxdb.dto.Query; +import org.influxdb.querybuilder.SelectQueryImpl; +import org.influxdb.querybuilder.SelectionQueryImpl; +import org.influxdb.querybuilder.WhereNested; +import org.influxdb.querybuilder.WhereQueryImpl; +import org.influxdb.querybuilder.clauses.AndConjunction; +import org.influxdb.querybuilder.clauses.NestedClause; + +import java.math.BigDecimal; +import java.util.List; +import java.util.Optional; + +import static io.trino.plugin.influxdb.TimestampUtils.unixTimestamp; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.BooleanType.BOOLEAN; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder; +import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.eq; +import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.gt; +import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.gte; +import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.lt; +import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.lte; + +public class QueryUtils +{ + private QueryUtils() {} + + public static Query buildQueryCommand(InfluxTableHandle tableHandle, List columnHandles) + { + SelectionQueryImpl selectionQuery = QueryBuilder.select(); + appendProjection(selectionQuery, tableHandle, columnHandles); + SelectQueryImpl selectQuery = selectionQuery.from(tableHandle.getSchemaName(), wrapQuota(tableHandle.getTableName())); + appendWhereClause(selectQuery, tableHandle); + tableHandle.getLimit().ifPresent(selectQuery::limit); + return selectQuery; + } + + private static void appendProjection(SelectionQueryImpl query, InfluxTableHandle tableHandle, List columnHandles) + { + List columns = buildProjection(tableHandle.getProjections()); + if (!columns.isEmpty()) { + columns.stream() + .map(QueryUtils::wrapQuota) + .forEach(query::column); + } + else { + columnHandles.stream() + .map(col -> wrapQuota(col.getName())) + .forEach(query::column); + } + } + + private static List buildProjection(List assignments) + { + if (assignments.isEmpty()) { + return ImmutableList.of(); + } + ImmutableList.Builder columns = ImmutableList.builder(); + assignments.stream() + .map(InfluxColumnHandle.class::cast) + .map(InfluxColumnHandle::getName) + .forEach(columns::add); + return columns.build(); + } + + private static void appendWhereClause(SelectQueryImpl query, InfluxTableHandle tableHandle) + { + TupleDomain constraint = tableHandle.getConstraint(); + constraint.getDomains().ifPresent(columnHandleDomainMap -> { + WhereQueryImpl where = query.where(); + columnHandleDomainMap.forEach((columnHandle, domain) -> + appendDomainFilter(where, (InfluxColumnHandle) columnHandle, domain)); + }); + } + + private static void appendDomainFilter(WhereQueryImpl where, InfluxColumnHandle columnHandle, Domain domain) + { + String columnName = wrapQuota(columnHandle.getName()); + domain.getValues().getValuesProcessor().consume( + ranges -> appendDomainRangeFilter(where, columnName, ranges), + discreteValues -> appendDomainDiscreteFilter(where, columnName, discreteValues), + allOrNone -> {}); + } + + private static void appendDomainRangeFilter(WhereQueryImpl where, String columnName, Ranges ranges) + { + List orderedRanges = ranges.getOrderedRanges(); + WhereNested> nestedWhere = where.andNested(); + if (orderedRanges.stream().allMatch(x -> x.getType() instanceof TimestampType)) { + appendDomainTimeRangeFilter(nestedWhere, columnName, orderedRanges); + } + else if (orderedRanges.stream().allMatch(x -> x.getType() == DOUBLE || x.getType() == BIGINT)) { + appendDomainNumRangeFilter(nestedWhere, columnName, orderedRanges); + } + else if (orderedRanges.stream().allMatch(x -> x.getType() == BOOLEAN)) { + appendDomainBooleanRangeFilter(nestedWhere, columnName, orderedRanges); + } + else if (orderedRanges.stream().allMatch(x -> x.getType() == VARCHAR)) { + appendDomainVarcharRangeFilter(nestedWhere, columnName, orderedRanges); + } + nestedWhere.close(); + } + + private static void appendDomainTimeRangeFilter(WhereNested> where, String columnName, List ranges) + { + for (Range range : ranges) { + if (range.isSingleValue()) { + Object singleValue = range.getSingleValue(); + LongTimestamp timestamp = (LongTimestamp) singleValue; + where.or(eq(columnName, unixTimestamp(timestamp))); + } + else { + boolean lowInclusive = range.isLowInclusive(); + Optional lowValue = range.getLowValue(); + boolean highInclusive = range.isHighInclusive(); + Optional highValue = range.getHighValue(); + + if (lowValue.isPresent() && highValue.isPresent()) { + long lowTimestamp = unixTimestamp((LongTimestamp) lowValue.get()); + AndConjunction lowConjunction = new AndConjunction(lowInclusive ? gte(columnName, lowTimestamp) : gt(columnName, lowTimestamp)); + long highTimestamp = unixTimestamp((LongTimestamp) highValue.get()); + AndConjunction highConjunction = new AndConjunction(highInclusive ? lte(columnName, highTimestamp) : lt(columnName, highTimestamp)); + where.or(new NestedClause(ImmutableList.of(lowConjunction, highConjunction))); + } + else if (lowValue.isPresent()) { + long lowTimestamp = unixTimestamp((LongTimestamp) lowValue.get()); + where.or(lowInclusive ? gte(columnName, lowTimestamp) : gt(columnName, lowTimestamp)); + } + else if (highValue.isPresent()) { + long highTimestamp = unixTimestamp((LongTimestamp) highValue.get()); + where.or(highInclusive ? lte(columnName, highTimestamp) : lt(columnName, highTimestamp)); + } + } + } + } + + private static void appendDomainNumRangeFilter(WhereNested> where, String columnName, List ranges) + { + for (Range range : ranges) { + if (range.isSingleValue()) { + Object singleValue = range.getSingleValue(); + if (singleValue instanceof Double doubleValue) { + BigDecimal value = new BigDecimal(doubleValue.toString()); + if (value.toString().contains("E") || value.toString().contains("e")) { + where.and(eq(1, 1)); + } + else { + where.or(eq(columnName, value)); + } + } + else if (singleValue instanceof Long longValue) { + where.or(eq(columnName, longValue)); + } + } + else { + boolean lowInclusive = range.isLowInclusive(); + Optional lowValue = range.getLowValue(); + boolean highInclusive = range.isHighInclusive(); + Optional highValue = range.getHighValue(); + + if (lowValue.isPresent() && highValue.isPresent()) { + AndConjunction lowConjunction = new AndConjunction(lowInclusive ? gte(columnName, lowValue.get()) : gt(columnName, lowValue.get())); + AndConjunction highConjunction = new AndConjunction(highInclusive ? lte(columnName, highValue.get()) : lt(columnName, highValue.get())); + where.or(new NestedClause(ImmutableList.of(lowConjunction, highConjunction))); + } + else if (lowValue.isPresent()) { + Object value = lowValue.get(); + where.or(lowInclusive ? gte(columnName, value) : gt(columnName, value)); + } + else if (highValue.isPresent()) { + Object value = highValue.get(); + where.or(highInclusive ? lte(columnName, value) : lt(columnName, value)); + } + } + } + } + + private static void appendDomainBooleanRangeFilter(WhereNested> where, String columnName, List ranges) + { + for (Range range : ranges) { + if (range.isSingleValue()) { + Boolean singleValue = (Boolean) range.getSingleValue(); + where.and(eq(columnName, singleValue)); + } + } + } + + private static void appendDomainVarcharRangeFilter(WhereNested> where, String columnName, List ranges) + { + for (Range range : ranges) { + if (range.isSingleValue()) { + Object singleValue = range.getSingleValue(); + singleValue = ((Slice) singleValue).toStringUtf8(); + where.and(eq(columnName, singleValue)); + } + } + } + + private static void appendDomainDiscreteFilter(WhereQueryImpl where, String columnName, DiscreteValues discreteValues) + { + WhereNested> whereNested = where.andNested(); + for (Object value : discreteValues.getValues()) { + whereNested.or(eq(columnName, value)); + } + whereNested.close(); + } + + private static String wrapQuota(String column) + { + return "\"" + column + "\""; + } +} diff --git a/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/TimestampUtils.java b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/TimestampUtils.java new file mode 100644 index 000000000000..04cc730f05ae --- /dev/null +++ b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/TimestampUtils.java @@ -0,0 +1,41 @@ +/* + * 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.influxdb; + +import io.trino.spi.type.LongTimestamp; + +import java.time.Instant; + +import static io.trino.spi.type.Timestamps.MICROSECONDS_PER_SECOND; +import static io.trino.spi.type.Timestamps.NANOSECONDS_PER_MICROSECOND; +import static io.trino.spi.type.Timestamps.PICOSECONDS_PER_NANOSECOND; +import static java.time.temporal.ChronoField.MICRO_OF_SECOND; + +public final class TimestampUtils +{ + private TimestampUtils() {} + + public static LongTimestamp longTimestamp(Instant start) + { + return new LongTimestamp( + start.getEpochSecond() * MICROSECONDS_PER_SECOND + start.getLong(MICRO_OF_SECOND), + (start.getNano() % NANOSECONDS_PER_MICROSECOND) * PICOSECONDS_PER_NANOSECOND); + } + + public static long unixTimestamp(LongTimestamp timestamp) + { + return timestamp.getEpochMicros() * NANOSECONDS_PER_MICROSECOND + + timestamp.getPicosOfMicro() / PICOSECONDS_PER_NANOSECOND; + } +} diff --git a/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/TypeUtils.java b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/TypeUtils.java new file mode 100644 index 000000000000..1fb96dc5e908 --- /dev/null +++ b/plugin/trino-influxdb/src/main/java/io/trino/plugin/influxdb/TypeUtils.java @@ -0,0 +1,60 @@ +/* + * 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.influxdb; + +import com.google.common.collect.ImmutableSet; +import io.trino.spi.type.BigintType; +import io.trino.spi.type.BooleanType; +import io.trino.spi.type.DoubleType; +import io.trino.spi.type.TimestampType; +import io.trino.spi.type.Type; +import io.trino.spi.type.VarcharType; + +import java.util.Set; + +public final class TypeUtils +{ + public static final String BOOLEAN = "boolean"; + public static final String STRING = "string"; + public static final String INTEGER = "integer"; + public static final String FLOAT = "float"; + public static final String TIMESTAMP = "timestamp"; + + private static final Set supportedTypes = ImmutableSet.of( + BooleanType.BOOLEAN, + VarcharType.VARCHAR, + BigintType.BIGINT, + DoubleType.DOUBLE); + + private TypeUtils() {} + + public static Type toTrinoType(String influxType) + { + return switch (influxType) { + case TIMESTAMP -> TimestampType.TIMESTAMP_NANOS; + case BOOLEAN -> BooleanType.BOOLEAN; + case STRING -> VarcharType.VARCHAR; + case INTEGER -> BigintType.BIGINT; + case FLOAT -> DoubleType.DOUBLE; + default -> throw new RuntimeException("type is not supported : " + influxType); + }; + } + + public static boolean isPushdownSupportedType(Type type) + { + return type instanceof TimestampType + || supportedTypes.contains(type); + } +} diff --git a/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/InfluxDataSetup.java b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/InfluxDataSetup.java new file mode 100644 index 000000000000..31332cfa1c64 --- /dev/null +++ b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/InfluxDataSetup.java @@ -0,0 +1,108 @@ +/* + * 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.influxdb; + +import com.google.common.collect.Maps; +import io.trino.spi.type.Type; +import io.trino.testing.datatype.ColumnSetup; +import io.trino.testing.datatype.DataSetup; +import io.trino.testing.sql.TemporaryRelation; +import org.influxdb.dto.Point; + +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.List; +import java.util.Map; + +import static io.trino.plugin.influxdb.TypeUtils.toTrinoType; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.BooleanType.BOOLEAN; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static java.lang.Boolean.parseBoolean; +import static java.lang.Double.parseDouble; +import static java.lang.Long.parseLong; +import static java.lang.String.format; +import static java.time.format.DateTimeFormatter.ISO_DATE_TIME; +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.TimeUnit.NANOSECONDS; + +public class InfluxDataSetup + implements DataSetup +{ + private static final String TEST_DATABASE_TPCH = "tpch"; + private final InfluxSession session; + private final String tableNamePrefix; + + public InfluxDataSetup(InfluxSession session, String tableNamePrefix) + { + this.session = requireNonNull(session, "session is null"); + this.tableNamePrefix = requireNonNull(tableNamePrefix, "tableNamePrefix is null"); + } + + @Override + public TemporaryRelation setupTemporaryRelation(List inputs) + { + TestingInfluxTable testTable = new TestingInfluxTable(this.tableNamePrefix); + try { + insertRowsWithExpectedValues(inputs, testTable); + } + catch (Exception e) { + throw new RuntimeException(e); + } + return testTable; + } + + private void insertRowsWithExpectedValues(List inputs, TestingInfluxTable testTable) + { + Map nullValueFields = Maps.newHashMapWithExpectedSize(inputs.size()); + Point.Builder builder = Point.measurement(testTable.getName()); + int index = 0; + for (ColumnSetup input : inputs) { + String filedName = format("col_%s", index); + String inputType = input.getDeclaredType().orElse("string"); + String inputValue = input.getInputLiteral(); + if ("NULL".equalsIgnoreCase(inputValue)) { + nullValueFields.put(filedName, null); + continue; + } + if (index == 0) { + Instant utc = LocalDateTime.parse(inputValue, ISO_DATE_TIME).atZone(ZoneId.of("UTC")).toInstant(); + long time = utc.getEpochSecond() * 1_000_000_000L + utc.getNano(); + builder.time(time, NANOSECONDS); + } + else { + Type type = toTrinoType(inputType); + if (type == BOOLEAN) { + boolean fieldValue = parseBoolean(inputValue); + builder.addField(filedName, fieldValue); + } + else if (type == BIGINT) { + long fieldValue = parseLong(inputValue); + builder.addField(filedName, fieldValue); + } + else if (type == DOUBLE) { + double fieldValue = parseDouble(inputValue); + builder.addField(filedName, fieldValue); + } + else { + builder.addField(filedName, inputValue); + } + } + index++; + } + builder.fields(nullValueFields); + session.write(TEST_DATABASE_TPCH, builder.build()); + } +} diff --git a/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/InfluxDataTool.java b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/InfluxDataTool.java new file mode 100644 index 000000000000..49626864b9a1 --- /dev/null +++ b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/InfluxDataTool.java @@ -0,0 +1,80 @@ +/* + * 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.influxdb; + +import com.google.common.collect.ImmutableList; +import org.influxdb.dto.Point; + +import java.util.List; +import java.util.Locale; +import java.util.Random; +import java.util.concurrent.TimeUnit; + +public class InfluxDataTool +{ + public static final String TEST_DATABASE = "test_database"; + public static final String TEST_DATABASE_ANOTHER = "test_database_another"; + public static final String TEST_DATABASE_TEMPORARY = "test_database_temporary"; + public static final String TEST_MEASUREMENT_W = "test_measurement_w"; + public static final String TEST_MEASUREMENT_X = "test_measurement_x"; + public static final String TEST_MEASUREMENT_Y = "test_measurement_y"; + public static final String TEST_MEASUREMENT_Z = "test_measurement_z"; + + private static final String CREATE_DATABASE_CMD = "CREATE DATABASE %s"; + + private final InfluxSession session; + + public InfluxDataTool(InfluxSession session) + { + this.session = session; + } + + public void setUpDatabase() + { + this.setUpDatabase(ImmutableList.of(TEST_DATABASE, TEST_DATABASE_ANOTHER)); + } + + public void setUpDatabase(List databaseNames) + { + for (String databaseName : databaseNames) { + String command = String.format(CREATE_DATABASE_CMD, databaseName); + session.execute(command); + } + } + + public void setUpDataForTest() + { + this.setUpDataForTest(TEST_DATABASE, ImmutableList.of(TEST_MEASUREMENT_X, TEST_MEASUREMENT_Y)); + this.setUpDataForTest(TEST_DATABASE_ANOTHER, ImmutableList.of(TEST_MEASUREMENT_Z)); + } + + public void setUpDataForTest(String databaseName, List tableNames) + { + String[] countries = Locale.getISOCountries(); + long start = System.currentTimeMillis() - 100_1000L; + for (String tableName : tableNames) { + for (int i = 0; i < 100; i++) { + Point.Builder builder = Point.measurement(tableName) + .tag("country", countries[new Random().nextInt(countries.length)]) + .addField("f1", new Random().nextInt(100)) + .addField("f2", Math.round(new Random().nextDouble() * 100 * 100) / 100.0) + .addField("f3", "data_" + new Random(100)) + .addField("f4", new Random().nextBoolean()) + .time(start + i * 1000L, TimeUnit.MILLISECONDS); + session.write(databaseName, builder.build()); + } + } + } +} diff --git a/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/InfluxLoader.java b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/InfluxLoader.java new file mode 100644 index 000000000000..fffb99667643 --- /dev/null +++ b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/InfluxLoader.java @@ -0,0 +1,136 @@ +/* + * 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.influxdb; + +import com.google.common.collect.ImmutableMap; +import io.trino.Session; +import io.trino.client.Column; +import io.trino.client.QueryStatusInfo; +import io.trino.client.ResultRows; +import io.trino.server.testing.TestingTrinoServer; +import io.trino.spi.type.Type; +import io.trino.spi.type.VarcharType; +import io.trino.testing.AbstractTestingTrinoClient; +import io.trino.testing.ResultsSession; +import org.influxdb.dto.BatchPoints; +import org.influxdb.dto.Point; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; + +import static com.google.common.base.Preconditions.checkState; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.BooleanType.BOOLEAN; +import static io.trino.spi.type.DateType.DATE; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static io.trino.spi.type.IntegerType.INTEGER; +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.TimeUnit.MILLISECONDS; + +public class InfluxLoader + extends AbstractTestingTrinoClient +{ + private final String schemaName; + private final String tableName; + private final InfluxSession session; + + public InfluxLoader( + InfluxSession session, + String schemaName, + String tableName, + TestingTrinoServer trinoServer, + Session defaultSession) + { + super(trinoServer, defaultSession); + + this.schemaName = requireNonNull(schemaName, "schemaName is null"); + this.tableName = requireNonNull(tableName, "tableName is null"); + this.session = requireNonNull(session, "session is null"); + } + + @Override + public ResultsSession getResultSession(Session session) + { + requireNonNull(session, "session is null"); + return new InfluxLoadingSession(); + } + + long timestamp = System.currentTimeMillis(); + + private class InfluxLoadingSession + implements ResultsSession + { + private final AtomicReference> types = new AtomicReference<>(); + + private InfluxLoadingSession() {} + + @Override + public void addResults(QueryStatusInfo statusInfo, ResultRows data) + { + if (types.get() == null && statusInfo.getColumns() != null) { + types.set(getTypes(statusInfo.getColumns())); + } + + if (data.isNull()) { + return; + } + checkState(types.get() != null, "Type information is missing"); + List columns = statusInfo.getColumns(); + + BatchPoints.Builder points = BatchPoints.database(schemaName); + for (List fields : data) { + ImmutableMap.Builder fieldBuilder = ImmutableMap.builderWithExpectedSize(fields.size()); + Point.Builder pointBuilder = Point.measurement(tableName).time(timestamp++, MILLISECONDS); + for (int i = 0; i < fields.size(); i++) { + Type type = types.get().get(i); + String name = columns.get(i).getName(); + Object value = convertValue(fields.get(i), type); + fieldBuilder.put(name, value); + } + pointBuilder.fields(fieldBuilder.buildOrThrow()); + points.point(pointBuilder.build()); + } + + session.writeBatch(schemaName, points.build()); + } + + @Override + public Void build(Map setSessionProperties, Set resetSessionProperties) + { + return null; + } + + private Object convertValue(Object value, Type type) + { + if (value == null) { + return null; + } + if (type instanceof VarcharType || type == BOOLEAN || type == DATE) { + return value; + } + if (type == INTEGER) { + return ((Number) value).intValue(); + } + if (type == BIGINT) { + return ((Number) value).longValue(); + } + if (type == DOUBLE) { + return ((Number) value).doubleValue(); + } + throw new IllegalArgumentException("Unhandled type: " + type); + } + } +} diff --git a/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/InfluxQueryRunner.java b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/InfluxQueryRunner.java new file mode 100644 index 000000000000..5a8d0626817a --- /dev/null +++ b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/InfluxQueryRunner.java @@ -0,0 +1,118 @@ +/* + * 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.influxdb; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.airlift.log.Logger; +import io.trino.Session; +import io.trino.metadata.QualifiedObjectName; +import io.trino.plugin.tpch.TpchPlugin; +import io.trino.testing.DistributedQueryRunner; +import io.trino.testing.TestingTrinoClient; +import io.trino.tpch.TpchTable; + +import java.util.HashMap; +import java.util.Map; + +import static io.airlift.units.Duration.nanosSince; +import static io.trino.plugin.influxdb.TestingInfluxServer.PASSWORD; +import static io.trino.plugin.influxdb.TestingInfluxServer.USERNAME; +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.concurrent.TimeUnit.SECONDS; + +public class InfluxQueryRunner +{ + private static final String TPCH_SCHEMA = "tpch"; + private static final Logger LOG = Logger.get(InfluxQueryRunner.class); + + private InfluxQueryRunner() {} + + public static DistributedQueryRunner createInfluxQueryRunner(TestingInfluxServer server, Iterable> tables) + throws Exception + { + return createInfluxQueryRunner(server, ImmutableMap.of(), ImmutableList.copyOf(tables)); + } + + public static DistributedQueryRunner createInfluxQueryRunner(TestingInfluxServer server, Map extraProperties, Iterable> tables) + throws Exception + { + return createInfluxQueryRunner(server, extraProperties, ImmutableMap.of(), tables); + } + + public static DistributedQueryRunner createInfluxQueryRunner(TestingInfluxServer server, Map extraProperties, Map connectorProperties, Iterable> tables) + throws Exception + { + DistributedQueryRunner queryRunner = DistributedQueryRunner.builder(createInfluxSession(TPCH_SCHEMA)) + .setExtraProperties(extraProperties) + .build(); + + queryRunner.installPlugin(new TpchPlugin()); + queryRunner.createCatalog("tpch", "tpch"); + + connectorProperties = new HashMap<>(ImmutableMap.copyOf(connectorProperties)); + connectorProperties.putIfAbsent("influx.endpoint", server.getEndpoint()); + connectorProperties.putIfAbsent("influx.username", USERNAME); + connectorProperties.putIfAbsent("influx.password", PASSWORD); + connectorProperties.putIfAbsent("influx.read-timeout", "5m"); + queryRunner.installPlugin(new InfluxPlugin()); + queryRunner.createCatalog("influxdb", "influxdb", connectorProperties); + + TestingTrinoClient trinoClient = queryRunner.getClient(); + LOG.info("Loading data..."); + InfluxSession session = new InfluxSession(server.getEndpoint()); + new InfluxDataTool(session).setUpDatabase(ImmutableList.of(TPCH_SCHEMA)); + long startTime = System.nanoTime(); + for (TpchTable table : tables) { + loadTpchTopic(session, trinoClient, table); + } + LOG.info("Loading complete in %s", nanosSince(startTime).toString(SECONDS)); + + return queryRunner; + } + + public static Session createInfluxSession(String schema) + { + return testSessionBuilder() + .setCatalog("influxdb") + .setSchema(schema) + .build(); + } + + private static void loadTpchTopic(InfluxSession session, TestingTrinoClient trinoClient, TpchTable table) + { + long start = System.nanoTime(); + LOG.info("Running import for %s", table.getTableName()); + String tableName = table.getTableName().toLowerCase(ENGLISH); + InfluxLoader loader = new InfluxLoader(session, TPCH_SCHEMA, tableName, trinoClient.getServer(), trinoClient.getDefaultSession()); + loader.execute(format("SELECT * FROM %s", new QualifiedObjectName(TPCH_SCHEMA, TINY_SCHEMA_NAME, table.getTableName().toLowerCase(ENGLISH)))); + LOG.info("Imported %s in %s", table.getTableName(), nanosSince(start).convertToMostSuccinctTimeUnit()); + } + + public static void main(String[] args) + throws Exception + { + DistributedQueryRunner queryRunner = createInfluxQueryRunner( + new TestingInfluxServer(), +// ImmutableMap.of("http-server.http.port", "8020"), + TpchTable.getTables()); + Logger log = Logger.get(InfluxQueryRunner.class); + log.info("======== SERVER STARTED ========"); + log.info("\n====\n%s\n====", queryRunner.getCoordinator().getBaseUrl()); + } +} diff --git a/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/InfluxSession.java b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/InfluxSession.java new file mode 100644 index 000000000000..c21c795e93f4 --- /dev/null +++ b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/InfluxSession.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.influxdb; + +import org.influxdb.InfluxDB; +import org.influxdb.InfluxDBFactory; +import org.influxdb.dto.BatchPoints; +import org.influxdb.dto.Point; +import org.influxdb.dto.Query; + +import java.io.Closeable; + +import static io.trino.plugin.influxdb.TestingInfluxServer.PASSWORD; +import static io.trino.plugin.influxdb.TestingInfluxServer.USERNAME; + +public class InfluxSession + implements Closeable +{ + private final InfluxDB influxDB; + + public InfluxSession(String endpoint) + { + this.influxDB = InfluxDBFactory.connect(endpoint, USERNAME, PASSWORD); + this.influxDB.disableBatch(); + } + + public String checkConnectivity() + { + return influxDB.version(); + } + + public void execute(String command) + { + execute(null, command); + } + + public void execute(String database, String command) + { + Query query = database == null ? + new Query(command) : new Query(command, database); + influxDB.query(query); + } + + public void write(String database, Point point) + { + influxDB.setDatabase(database); + influxDB.write(point); + } + + public void writeBatch(String database, BatchPoints points) + { + influxDB.setDatabase(database); + influxDB.write(points); + } + + @Override + public void close() + { + influxDB.close(); + } +} diff --git a/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestInfluxConfig.java b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestInfluxConfig.java new file mode 100644 index 000000000000..d718dae5a1d9 --- /dev/null +++ b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestInfluxConfig.java @@ -0,0 +1,79 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.influxdb; + +import com.google.common.collect.ImmutableMap; +import io.airlift.units.Duration; +import jakarta.validation.constraints.AssertFalse; +import org.junit.jupiter.api.Test; + +import java.util.Map; + +import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; +import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; +import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; +import static io.airlift.testing.ValidationAssertions.assertFailsValidation; +import static java.util.concurrent.TimeUnit.SECONDS; + +public class TestInfluxConfig +{ + @Test + public void testDefaults() + { + assertRecordedDefaults(recordDefaults(InfluxConfig.class) + .setEndpoint(null) + .setUsername(null) + .setPassword(null) + .setConnectTimeOut(new Duration(10, SECONDS)) + .setReadTimeOut(new Duration(60, SECONDS))); + } + + @Test + public void testExplicitPropertyMappings() + { + Map properties = ImmutableMap.builder() + .put("influx.endpoint", "http://localhost:8086") + .put("influx.username", "testuser") + .put("influx.password", "testpass") + .put("influx.connect-timeout", "5s") + .put("influx.read-timeout", "30s") + .buildOrThrow(); + + InfluxConfig expected = new InfluxConfig() + .setEndpoint("http://localhost:8086") + .setUsername("testuser") + .setPassword("testpass") + .setConnectTimeOut(new Duration(5, SECONDS)) + .setReadTimeOut(new Duration(30, SECONDS)); + + assertFullMapping(properties, expected); + } + + @Test + public void testValidation() + { + assertFailsPasswordValidation(new InfluxConfig().setUsername("testuser")); + assertFailsPasswordValidation(new InfluxConfig().setPassword("testpass")); + } + + private static void assertFailsPasswordValidation(InfluxConfig config) + { + assertFailsValidation( + config, + "validPasswordConfig", + "'influx.username' and 'influx.password' must be both empty or both non-empty.", + AssertFalse.class); + } +} diff --git a/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestInfluxConnectorTest.java b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestInfluxConnectorTest.java new file mode 100644 index 000000000000..0e1f07a7f302 --- /dev/null +++ b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestInfluxConnectorTest.java @@ -0,0 +1,270 @@ +/* + * 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.influxdb; + +import io.trino.testing.BaseConnectorTest; +import io.trino.testing.MaterializedResult; +import io.trino.testing.QueryRunner; +import io.trino.testing.TestingConnectorBehavior; +import org.intellij.lang.annotations.Language; +import org.junit.jupiter.api.Test; +import org.testcontainers.shaded.com.google.common.collect.ImmutableList; + +import java.util.OptionalInt; + +import static io.trino.plugin.influxdb.InfluxQueryRunner.createInfluxQueryRunner; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static io.trino.testing.MaterializedResult.resultBuilder; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_SCHEMA; +import static io.trino.testing.TestingNames.randomNameSuffix; +import static java.lang.String.format; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assumptions.abort; + +public class TestInfluxConnectorTest + extends BaseConnectorTest +{ + TestingInfluxServer server; + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + server = closeAfterClass(new TestingInfluxServer()); + return createInfluxQueryRunner(server, REQUIRED_TPCH_TABLES); + } + + @SuppressWarnings("DuplicateBranchesInSwitch") + @Override + protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) + { + return switch (connectorBehavior) { + case SUPPORTS_CREATE_SCHEMA -> true; + case SUPPORTS_RENAME_SCHEMA, SUPPORTS_CREATE_VIEW, SUPPORTS_CREATE_MATERIALIZED_VIEW -> false; + + case SUPPORTS_CREATE_TABLE, SUPPORTS_RENAME_TABLE -> false; + case SUPPORTS_ADD_COLUMN, SUPPORTS_RENAME_COLUMN, SUPPORTS_SET_COLUMN_TYPE -> false; + + case SUPPORTS_COMMENT_ON_TABLE, SUPPORTS_COMMENT_ON_COLUMN -> false; + case SUPPORTS_INSERT, SUPPORTS_UPDATE, SUPPORTS_DELETE, SUPPORTS_MERGE -> false; + case SUPPORTS_ARRAY, SUPPORTS_ROW_TYPE -> false; + default -> super.hasBehavior(connectorBehavior); + }; + } + + @Override + protected OptionalInt maxSchemaNameLength() + { + return OptionalInt.of(255); + } + + @Override + protected void verifySchemaNameLengthFailurePermissible(Throwable e) + { + assertThat(e).hasMessageContaining("too long"); + } + + @Test + @Override + public void testShowCreateTable() + { + // List columns explicitly, as there's no defined order in InfluxDB, columns will return by natural order + // to manually order columns, there is an existing issue here. https://github.com/influxdata/influxdb/issues/15957 + String catalog = getSession().getCatalog().orElseThrow(); + String schema = getSession().getSchema().orElseThrow(); + assertThat(computeScalar("SHOW CREATE TABLE orders")) + .isEqualTo(format(""" + CREATE TABLE %s.%s.orders ( + time timestamp(9), + clerk varchar, + comment varchar, + custkey bigint, + orderdate varchar, + orderkey bigint, + orderpriority varchar, + orderstatus varchar, + shippriority bigint, + totalprice double + )""", //"time" is 1st column in influx query result additionally. + catalog, schema)); + } + + @Test + @Override + public void testShowColumns() + { + // List columns explicitly, as there's no defined order in InfluxDB, columns will return by natural order + // to manually order columns, there is an existing issue here. https://github.com/influxdata/influxdb/issues/15957 + MaterializedResult actual = computeActual("SHOW COLUMNS FROM orders"); + MaterializedResult expected = resultBuilder(getSession(), VARCHAR, VARCHAR, VARCHAR, VARCHAR) + .row("time", "timestamp(9)", "", "") //"time" is 1st column in influx query result additionally. + .row("clerk", "varchar", "", "") + .row("comment", "varchar", "", "") + .row("custkey", "bigint", "", "") + .row("orderdate", "varchar", "", "") + .row("orderkey", "bigint", "", "") + .row("orderpriority", "varchar", "", "") + .row("orderstatus", "varchar", "", "") + .row("shippriority", "bigint", "", "") + .row("totalprice", "double", "", "") + .build(); + + assertThat(expected).isEqualTo(actual); + } + + @Test + @Override + public void testDescribeTable() + { + // List columns explicitly, as there's no defined order in InfluxDB, columns will return by natural order + // to manually order columns, there is an existing issue here. https://github.com/influxdata/influxdb/issues/15957 + MaterializedResult expectedColumns = MaterializedResult.resultBuilder(getSession(), VARCHAR, VARCHAR, VARCHAR, VARCHAR) + .row("time", "timestamp(9)", "", "") //"time" is 1st column in influx query result additionally. + .row("clerk", "varchar", "", "") + .row("comment", "varchar", "", "") + .row("custkey", "bigint", "", "") + .row("orderdate", "varchar", "", "") + .row("orderkey", "bigint", "", "") + .row("orderpriority", "varchar", "", "") + .row("orderstatus", "varchar", "", "") + .row("shippriority", "bigint", "", "") + .row("totalprice", "double", "", "") + .build(); + MaterializedResult actualColumns = computeActual("DESCRIBE orders"); + assertThat(actualColumns).isEqualTo(expectedColumns); + } + + @Test + @Override + public void testSelectInformationSchemaColumns() + { + String catalog = getSession().getCatalog().get(); + String schema = getSession().getSchema().get(); + String schemaPattern = schema.replaceAll(".$", "_"); + + // List columns explicitly, as there's no defined order in InfluxDB, columns will return by natural order + // to manually order columns, there is an existing issue here. https://github.com/influxdata/influxdb/issues/15957 + @Language("SQL") String ordersTableWithColumns = "VALUES " + + "('orders', 'time'), " + //"time" is 1st column in influx query result additionally. + "('orders', 'clerk'), " + + "('orders', 'comment')," + + "('orders', 'custkey'), " + + "('orders', 'orderdate'), " + + "('orders', 'orderkey'), " + + "('orders', 'orderpriority'), " + + "('orders', 'orderstatus'), " + + "('orders', 'shippriority'), " + + "('orders', 'totalprice')"; + + assertQuery("SELECT table_schema FROM information_schema.columns WHERE table_schema = '" + schema + "' GROUP BY table_schema", "VALUES '" + schema + "'"); + assertQuery("SELECT table_name FROM information_schema.columns WHERE table_name = 'orders' GROUP BY table_name", "VALUES 'orders'"); + assertQuery("SELECT table_name, column_name FROM information_schema.columns WHERE table_schema = '" + schema + "' AND table_name = 'orders'", ordersTableWithColumns); + assertQuery("SELECT table_name, column_name FROM information_schema.columns WHERE table_schema = '" + schema + "' AND table_name LIKE '%rders'", ordersTableWithColumns); + assertQuery("SELECT table_name, column_name FROM information_schema.columns WHERE table_schema LIKE '" + schemaPattern + "' AND table_name LIKE '_rder_'", ordersTableWithColumns); + assertThat(query( + "SELECT table_name, column_name FROM information_schema.columns " + + "WHERE table_catalog = '" + catalog + "' AND table_schema = '" + schema + "' AND table_name LIKE '%orders%'")) + .skippingTypesCheck() + .containsAll(ordersTableWithColumns); + + assertQuerySucceeds("SELECT * FROM information_schema.columns"); + assertQuery("SELECT DISTINCT table_name, column_name FROM information_schema.columns WHERE table_name LIKE '_rders'", ordersTableWithColumns); + assertQuerySucceeds("SELECT * FROM information_schema.columns WHERE table_catalog = '" + catalog + "'"); + assertQuerySucceeds("SELECT * FROM information_schema.columns WHERE table_catalog = '" + catalog + "' AND table_schema = '" + schema + "'"); + assertQuery("SELECT table_name, column_name FROM information_schema.columns WHERE table_catalog = '" + catalog + "' AND table_schema = '" + schema + "' AND table_name LIKE '_rders'", ordersTableWithColumns); + assertQuerySucceeds("SELECT * FROM information_schema.columns WHERE table_catalog = '" + catalog + "' AND table_name LIKE '%'"); + assertQuery("SELECT column_name FROM information_schema.columns WHERE table_catalog = 'something_else'", "SELECT '' WHERE false"); + + assertQuery( + "SELECT DISTINCT table_name FROM information_schema.columns WHERE table_schema = 'information_schema' OR rand() = 42 ORDER BY 1", + "VALUES " + + "('applicable_roles'), " + + "('columns'), " + + "('enabled_roles'), " + + "('roles'), " + + "('schemata'), " + + "('table_privileges'), " + + "('tables'), " + + "('views')"); + } + + @Test + @Override + public void testSelectAll() + { + // List columns explicitly, as there's no defined order in InfluxDB, columns will return by natural order + // to manually order columns, there is an existing issue here. https://github.com/influxdata/influxdb/issues/15957 + assertQuery("SELECT orderkey, custkey, orderstatus, totalprice, orderdate, orderpriority, clerk, shippriority, comment FROM orders"); + } + + @Test + @Override + public void testDateYearOfEraPredicate() + { + // InfluxDB does not support date columns, except for the special "time" column. + abort("requirement not met"); + } + + @Test + @Override + public void testInsertNegativeDate() { + // InfluxDB does not support date columns, except for the special "time" column. + assertQueryFails("INSERT INTO orders (time) VALUES (DATE '-0001-01-01')", "This connector does not support inserts"); + } + + @Test + @Override + public void testPredicateReflectedInExplain() + { + // The format of the string representation of what gets shown in the table scan is connector-specific. + assertExplain( + "EXPLAIN SELECT name FROM nation WHERE nationkey = 42", + "name=nationkey, type=bigint", "[42]"); + } + + @Test + @Override + public void testSortItemsReflectedInExplain() + { + // The format of the string representation of what gets shown in the table scan is connector-specific. + assertExplain( + "EXPLAIN SELECT name FROM nation ORDER BY nationkey DESC NULLS LAST LIMIT 5", + "TopNPartial\\[count = 5, orderBy = \\[nationkey DESC"); + } + + @Test + @Override + public void testDropNonEmptySchemaWithTable() { + // Override because CREATE TABLE is not supported in InfluxDB, we must insert data into the table to create it. + String schemaName = "test_drop_non_empty_schema_table_" + randomNameSuffix(); + // A connector either supports CREATE SCHEMA and DROP SCHEMA or none of them. + if (!hasBehavior(SUPPORTS_CREATE_SCHEMA)) { + return; + } + + try { + assertUpdate(createSchemaSql(schemaName)); + try (InfluxSession session = new InfluxSession(server.getEndpoint())) { + InfluxDataTool tool = new InfluxDataTool(session); + tool.setUpDataForTest(schemaName, ImmutableList.of("t")); // Create table + } + assertQueryFails("DROP SCHEMA " + schemaName, ".*Cannot drop non-empty schema '\\Q" + schemaName + "\\E'"); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + schemaName + ".t"); + assertUpdate("DROP SCHEMA IF EXISTS " + schemaName); + } + } +} diff --git a/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestInfluxMetadata.java b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestInfluxMetadata.java new file mode 100644 index 000000000000..2946a2444199 --- /dev/null +++ b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestInfluxMetadata.java @@ -0,0 +1,200 @@ +/* + * 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.influxdb; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import io.trino.spi.connector.ColumnMetadata; +import io.trino.spi.connector.ConnectorTableMetadata; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.connector.SchemaTablePrefix; +import io.trino.spi.connector.TableColumnsMetadata; +import io.trino.spi.connector.TableNotFoundException; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.util.Iterator; +import java.util.List; +import java.util.Optional; + +import static io.trino.plugin.influxdb.InfluxConstant.ColumnKind; +import static io.trino.plugin.influxdb.InfluxConstant.ColumnKind.FIELD; +import static io.trino.plugin.influxdb.InfluxConstant.ColumnKind.TAG; +import static io.trino.plugin.influxdb.InfluxConstant.ColumnName; +import static io.trino.plugin.influxdb.InfluxDataTool.TEST_DATABASE; +import static io.trino.plugin.influxdb.InfluxDataTool.TEST_DATABASE_ANOTHER; +import static io.trino.plugin.influxdb.InfluxDataTool.TEST_MEASUREMENT_X; +import static io.trino.plugin.influxdb.InfluxDataTool.TEST_MEASUREMENT_Y; +import static io.trino.plugin.influxdb.InfluxDataTool.TEST_MEASUREMENT_Z; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.BooleanType.BOOLEAN; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static io.trino.spi.type.TimestampType.TIMESTAMP_NANOS; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static io.trino.testing.TestingConnectorSession.SESSION; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +public class TestInfluxMetadata +{ + private static TestingInfluxServer server; + private static InfluxMetadata metadata; + + @BeforeAll + public static void setup() + { + server = new TestingInfluxServer(); + InfluxConfig config = new InfluxConfig(); + config.setEndpoint(server.getEndpoint()); + config.setUsername(TestingInfluxServer.USERNAME); + config.setPassword(TestingInfluxServer.PASSWORD); + metadata = new InfluxMetadata(new NativeInfluxClient(config)); + + try (InfluxSession session = new InfluxSession(server.getEndpoint())) { + InfluxDataTool tool = new InfluxDataTool(session); + tool.setUpDatabase(); + tool.setUpDataForTest(); + } + } + + @AfterAll + public static void destroy() + { + server.close(); + server = null; + } + + @Test + public void testListSchemaNames() + { + assertThat(metadata.listSchemaNames(SESSION)).isEqualTo(ImmutableList.of(TEST_DATABASE, TEST_DATABASE_ANOTHER)); + } + + @Test + public void testListTables() + { + // all schemas + assertThat(ImmutableSet.copyOf(metadata.listTables(SESSION, Optional.empty()))).isEqualTo(ImmutableSet.of( + new SchemaTableName(TEST_DATABASE, TEST_MEASUREMENT_X), + new SchemaTableName(TEST_DATABASE, TEST_MEASUREMENT_Y), + new SchemaTableName(TEST_DATABASE_ANOTHER, TEST_MEASUREMENT_Z))); + + // specific schema + assertThat(ImmutableSet.copyOf(metadata.listTables(SESSION, Optional.of(TEST_DATABASE)))).isEqualTo(ImmutableSet.of( + new SchemaTableName(TEST_DATABASE, TEST_MEASUREMENT_X), + new SchemaTableName(TEST_DATABASE, TEST_MEASUREMENT_Y))); + assertThat(ImmutableSet.copyOf(metadata.listTables(SESSION, Optional.of(TEST_DATABASE_ANOTHER)))).isEqualTo(ImmutableSet.of( + new SchemaTableName(TEST_DATABASE_ANOTHER, TEST_MEASUREMENT_Z))); + + // unknown schema + assertThat(ImmutableSet.copyOf(metadata.listTables(SESSION, Optional.of("unknown")))).isEqualTo(ImmutableSet.of()); + } + + @Test + public void testGetTableHandle() + { + SchemaTableName schemaTableName = new SchemaTableName(TEST_DATABASE, TEST_MEASUREMENT_X); + InfluxTableHandle tableHandle = new InfluxTableHandle(TEST_DATABASE, TEST_MEASUREMENT_X); + assertThat(metadata.getTableHandle(SESSION, schemaTableName)).isEqualTo(tableHandle); + assertThat(metadata.getTableHandle(SESSION, new SchemaTableName(TEST_DATABASE, "unknown"))).isNull(); + assertThat(metadata.getTableHandle(SESSION, new SchemaTableName("unknown", TEST_MEASUREMENT_X))).isNull(); + assertThat(metadata.getTableHandle(SESSION, new SchemaTableName("unknown", "unknown"))).isNull(); + } + + @Test + public void testGetTableMetadata() + { + InfluxTableHandle tableHandle = new InfluxTableHandle(TEST_DATABASE, TEST_MEASUREMENT_X); + ConnectorTableMetadata tableMetadata = metadata.getTableMetadata(SESSION, tableHandle); + SchemaTableName table = tableMetadata.getTable(); + List columns = tableMetadata.getColumns(); + assertThat(table).isEqualTo(new SchemaTableName(TEST_DATABASE, TEST_MEASUREMENT_X)); + assertThat(columns).isEqualTo(ImmutableList.of( + new ColumnMetadata(ColumnName.TIME.getName(), TIMESTAMP_NANOS), + new ColumnMetadata("f1", BIGINT), + new ColumnMetadata("f2", DOUBLE), + new ColumnMetadata("f3", VARCHAR), + new ColumnMetadata("f4", BOOLEAN), + new ColumnMetadata("country", VARCHAR))); + + assertThatThrownBy(() -> metadata.getTableMetadata(SESSION, new InfluxTableHandle("unknown", "unknown"))) + .isInstanceOf(RuntimeException.class) + .hasMessage("The table handle is invalid " + new InfluxTableHandle("unknown", "unknown")); + } + + @Test + public void testGetColumnHandles() + { + // known table + InfluxTableHandle tableHandle = new InfluxTableHandle(TEST_DATABASE, TEST_MEASUREMENT_X); + assertThat(metadata.getColumnHandles(SESSION, tableHandle)).isEqualTo(ImmutableMap.of( + "time", new InfluxColumnHandle("time", TIMESTAMP_NANOS, ColumnKind.TIME), + "f1", new InfluxColumnHandle("f1", BIGINT, FIELD), + "f2", new InfluxColumnHandle("f2", DOUBLE, FIELD), + "f3", new InfluxColumnHandle("f3", VARCHAR, FIELD), + "f4", new InfluxColumnHandle("f4", BOOLEAN, FIELD), + "country", new InfluxColumnHandle("country", VARCHAR, TAG))); + + // unknown table + assertThatThrownBy(() -> metadata.getColumnHandles(SESSION, new InfluxTableHandle(TEST_DATABASE, "unknown"))) + .isInstanceOf(TableNotFoundException.class) + .hasMessage("Table '" + TEST_DATABASE + ".unknown' not found"); + assertThatThrownBy(() -> metadata.getColumnHandles(SESSION, new InfluxTableHandle("unknown", TEST_MEASUREMENT_X))) + .isInstanceOf(TableNotFoundException.class) + .hasMessage("Table '" + "unknown." + TEST_MEASUREMENT_X + "' not found"); + assertThatThrownBy(() -> metadata.getColumnHandles(SESSION, new InfluxTableHandle("unknown", "unknown"))) + .isInstanceOf(TableNotFoundException.class) + .hasMessage("Table 'unknown.unknown' not found"); + } + + @Test + public void testGetColumnMetadata() + { + InfluxTableHandle tableHandle = new InfluxTableHandle(TEST_DATABASE, TEST_MEASUREMENT_X); + InfluxColumnHandle columnHandle = new InfluxColumnHandle("country", VARCHAR, TAG); + ColumnMetadata columnMetadata = metadata.getColumnMetadata(SESSION, tableHandle, columnHandle); + + assertThat(columnMetadata).isEqualTo(ColumnMetadata.builder().setName("country").setType(VARCHAR).build()); + } + + @Test + public void testStreamTableColumns() + { + // known table + SchemaTablePrefix prefix = new SchemaTablePrefix(TEST_DATABASE, TEST_MEASUREMENT_X); + Iterator iterator = metadata.streamTableColumns(SESSION, prefix); + assertThat(iterator).isNotNull(); + if (iterator.hasNext()) { + TableColumnsMetadata next = iterator.next(); + assertThat(next.getTable()).isEqualTo(new SchemaTableName(TEST_DATABASE, TEST_MEASUREMENT_X)); + assertThat(next.getColumns().isEmpty()).isFalse(); + assertThat(next.getColumns().get()).isEqualTo(ImmutableList.of( + new ColumnMetadata("time", TIMESTAMP_NANOS), + new ColumnMetadata("f1", BIGINT), + new ColumnMetadata("f2", DOUBLE), + new ColumnMetadata("f3", VARCHAR), + new ColumnMetadata("f4", BOOLEAN), + new ColumnMetadata("country", VARCHAR))); + } + + // unknown table + iterator = metadata.streamTableColumns(SESSION, new SchemaTablePrefix(TEST_DATABASE, "unknown")); + assertThat(iterator.hasNext()).isFalse(); + iterator = metadata.streamTableColumns(SESSION, new SchemaTablePrefix("unknown", "unknown")); + assertThat(iterator.hasNext()).isFalse(); + } +} diff --git a/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestInfluxMetadataMutation.java b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestInfluxMetadataMutation.java new file mode 100644 index 000000000000..f2e01ca9c048 --- /dev/null +++ b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestInfluxMetadataMutation.java @@ -0,0 +1,136 @@ +/* + * 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.influxdb; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.trino.spi.TrinoException; +import io.trino.spi.connector.SchemaNotFoundException; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.connector.TableNotFoundException; +import io.trino.spi.security.PrincipalType; +import io.trino.spi.security.TrinoPrincipal; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import java.util.Optional; + +import static io.trino.plugin.influxdb.InfluxDataTool.TEST_DATABASE_TEMPORARY; +import static io.trino.plugin.influxdb.InfluxDataTool.TEST_MEASUREMENT_W; +import static io.trino.testing.TestingConnectorSession.SESSION; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +public class TestInfluxMetadataMutation +{ + private TestingInfluxServer server; + private InfluxMetadata metadata; + + @BeforeClass + public void setup() + { + server = new TestingInfluxServer(); + InfluxConfig config = new InfluxConfig(); + config.setEndpoint(server.getEndpoint()); + config.setUsername(TestingInfluxServer.USERNAME); + config.setPassword(TestingInfluxServer.PASSWORD); + metadata = new InfluxMetadata(new NativeInfluxClient(config)); + } + + @AfterClass(alwaysRun = true) + public final void destroy() + { + server.close(); + server = null; + } + + @Test(priority = 1) + public void testCreateSchema() + { + metadata.createSchema(SESSION, TEST_DATABASE_TEMPORARY, ImmutableMap.of(), + new TrinoPrincipal(PrincipalType.USER, "test")); + assertThat(metadata.listSchemaNames(SESSION)).contains(TEST_DATABASE_TEMPORARY); + } + + @Test(priority = 2) + public void testDropTable() + { + // Create a table + try (InfluxSession session = new InfluxSession(server.getEndpoint())) { + InfluxDataTool tool = new InfluxDataTool(session); + tool.setUpDataForTest(TEST_DATABASE_TEMPORARY, ImmutableList.of(TEST_MEASUREMENT_W)); + } + assertThat(metadata.listTables(SESSION, Optional.of(TEST_DATABASE_TEMPORARY))) + .contains(new SchemaTableName(TEST_DATABASE_TEMPORARY, TEST_MEASUREMENT_W)); + + // Drop the table + InfluxTableHandle tableHandle = (InfluxTableHandle) metadata + .getTableHandle(SESSION, new SchemaTableName(TEST_DATABASE_TEMPORARY, TEST_MEASUREMENT_W)); + metadata.dropTable(SESSION, tableHandle); + + assertThat(metadata.listTables(SESSION, Optional.of(TEST_DATABASE_TEMPORARY))) + .doesNotContain(new SchemaTableName(TEST_DATABASE_TEMPORARY, TEST_MEASUREMENT_W)); + } + + @Test(priority = 2) + public void testDropUnknownTable() + { + InfluxTableHandle tableHandle = new InfluxTableHandle(TEST_DATABASE_TEMPORARY, "unknown"); + assertThatThrownBy(() -> metadata.dropTable(SESSION, tableHandle)) + .isInstanceOf(TableNotFoundException.class); + } + + @Test(priority = 2) + public void testDropUnknownSchema() + { + assertThatThrownBy(() -> metadata.dropSchema(SESSION, "unknown", false)) + .isInstanceOf(SchemaNotFoundException.class); + } + + @Test(priority = 3) + public void testDropSchemaCascade() + { + // Create a table + try (InfluxSession session = new InfluxSession(server.getEndpoint())) { + InfluxDataTool tool = new InfluxDataTool(session); + tool.setUpDataForTest(TEST_DATABASE_TEMPORARY, ImmutableList.of(TEST_MEASUREMENT_W)); + } + assertThat(metadata.listTables(SESSION, Optional.of(TEST_DATABASE_TEMPORARY))) + .contains(new SchemaTableName(TEST_DATABASE_TEMPORARY, TEST_MEASUREMENT_W)); + + // Drop the schema with cascade + metadata.dropSchema(SESSION, TEST_DATABASE_TEMPORARY, true); + assertThat(metadata.listSchemaNames(SESSION)).doesNotContain(TEST_DATABASE_TEMPORARY); + } + + @Test(priority = 4) + public void testDropSchema() + { + // Create a schema and a table + metadata.createSchema(SESSION, TEST_DATABASE_TEMPORARY, ImmutableMap.of(), + new TrinoPrincipal(PrincipalType.USER, "test")); + try (InfluxSession session = new InfluxSession(server.getEndpoint())) { + InfluxDataTool tool = new InfluxDataTool(session); + tool.setUpDataForTest(TEST_DATABASE_TEMPORARY, ImmutableList.of(TEST_MEASUREMENT_W)); + } + assertThat(metadata.listTables(SESSION, Optional.of(TEST_DATABASE_TEMPORARY))) + .contains(new SchemaTableName(TEST_DATABASE_TEMPORARY, TEST_MEASUREMENT_W)); + + assertThatThrownBy(() -> metadata.dropSchema(SESSION, TEST_DATABASE_TEMPORARY, false)) + .isInstanceOf(TrinoException.class) + .hasMessage("Cannot drop non-empty schema '%s'".formatted(TEST_DATABASE_TEMPORARY)); + } +} diff --git a/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestInfluxPlugin.java b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestInfluxPlugin.java new file mode 100644 index 000000000000..515949c51587 --- /dev/null +++ b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestInfluxPlugin.java @@ -0,0 +1,35 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.influxdb; + +import com.google.common.collect.ImmutableMap; +import io.trino.spi.Plugin; +import io.trino.spi.connector.Connector; +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; + +public class TestInfluxPlugin +{ + @Test + public void testCreateConnector() + { + Plugin plugin = new InfluxPlugin(); + ConnectorFactory factory = getOnlyElement(plugin.getConnectorFactories()); + Connector test = factory.create("test", ImmutableMap.of("influx.endpoint", "http://localhost:8086"), new TestingConnectorContext()); + test.shutdown(); + } +} diff --git a/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestInfluxRecordSet.java b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestInfluxRecordSet.java new file mode 100644 index 000000000000..1baaef8161c4 --- /dev/null +++ b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestInfluxRecordSet.java @@ -0,0 +1,113 @@ +/* + * 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.influxdb; + +import com.google.common.collect.ImmutableList; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import static io.trino.plugin.influxdb.InfluxConstant.ColumnKind.FIELD; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.BooleanType.BOOLEAN; +import static io.trino.spi.type.VarcharType.VARCHAR; +import static io.trino.spi.type.VarcharType.createUnboundedVarcharType; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +public abstract class TestInfluxRecordSet +{ + static InfluxRecord sourceData; + + @BeforeAll + public static void setup() + { + sourceData = new InfluxRecord(ImmutableList.of( + "col1", "col2", "col3" + ), ImmutableList.of( + ImmutableList.of("text1", -10L, false), + ImmutableList.of("text2", 0L, false), + ImmutableList.of("text3", 10L, true))); + } + + @Test + public void testFieldRequires() + { + assertThatThrownBy(() -> new InfluxRecordSet(null, ImmutableList.of(), null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("tableHandle is null"); + assertThatThrownBy(() -> new InfluxRecordSet(new InfluxTableHandle("a", "b"), null, null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("columnHandles is null"); + } + + @Test + public void testCursorSimple() + { + try (InfluxRecordCursor cursor = new InfluxRecordCursor( + ImmutableList.of( + new InfluxColumnHandle("col1", VARCHAR, FIELD), + new InfluxColumnHandle("col2", BIGINT, FIELD), + new InfluxColumnHandle("col3", BOOLEAN, FIELD)), + sourceData)) { + assertThat(cursor.getType(0)).isEqualTo(createUnboundedVarcharType()); + assertThat(cursor.getType(1)).isEqualTo(BIGINT); + + int idx = 0; + while (cursor.advanceNextPosition()) { + idx++; + if (idx == 1) { + assertThat(ImmutableList.of("text1", -10L, false)) + .isEqualTo(ImmutableList.of(cursor.getSlice(0).toStringUtf8(), cursor.getLong(1), cursor.getBoolean(2))); + } + if (idx == 2) { + assertThat(ImmutableList.of("text2", 0L, false)) + .isEqualTo(ImmutableList.of(cursor.getSlice(0).toStringUtf8(), cursor.getLong(1), cursor.getBoolean(2))); + } + if (idx == 3) { + assertThat(ImmutableList.of("text3", 10L, true)) + .isEqualTo(ImmutableList.of(cursor.getSlice(0).toStringUtf8(), cursor.getLong(1), cursor.getBoolean(2))); + } + } + } + } + + @Test + public void testCursorMixedOrder() + { + try (InfluxRecordCursor cursor = new InfluxRecordCursor( + ImmutableList.of( + new InfluxColumnHandle("col1", VARCHAR, FIELD), + new InfluxColumnHandle("col2", BIGINT, FIELD), + new InfluxColumnHandle("col3", BOOLEAN, FIELD)), + sourceData)) { + int idx = 0; + while (cursor.advanceNextPosition()) { + idx++; + if (idx == 1) { + assertThat(ImmutableList.of(-10L, "text1", false)) + .isEqualTo(ImmutableList.of(cursor.getLong(1), cursor.getSlice(0).toStringUtf8(), cursor.getBoolean(2))); + } + if (idx == 2) { + assertThat(ImmutableList.of(0L, false, "text2")) + .isEqualTo(ImmutableList.of(cursor.getLong(1), cursor.getBoolean(2), cursor.getSlice(0).toStringUtf8())); + } + if (idx == 3) { + assertThat(ImmutableList.of("text3", 10L, true)) + .isEqualTo(ImmutableList.of(cursor.getSlice(0).toStringUtf8(), cursor.getLong(1), cursor.getBoolean(2))); + } + } + } + } +} diff --git a/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestInfluxRecordSetProvider.java b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestInfluxRecordSetProvider.java new file mode 100644 index 000000000000..055faa7dda61 --- /dev/null +++ b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestInfluxRecordSetProvider.java @@ -0,0 +1,76 @@ +/* + * 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.influxdb; + +import com.google.common.collect.ImmutableList; +import io.trino.spi.connector.RecordSet; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import static io.trino.plugin.influxdb.InfluxDataTool.TEST_DATABASE; +import static io.trino.plugin.influxdb.InfluxDataTool.TEST_MEASUREMENT_X; +import static io.trino.plugin.influxdb.InfluxTransactionHandle.INSTANCE; +import static io.trino.plugin.influxdb.TestingInfluxServer.PASSWORD; +import static io.trino.plugin.influxdb.TestingInfluxServer.USERNAME; +import static io.trino.testing.TestingConnectorSession.SESSION; +import static org.assertj.core.api.Assertions.assertThat; + +public class TestInfluxRecordSetProvider +{ + private static TestingInfluxServer server; + private static InfluxClient client; + private static InfluxSplit split; + + @BeforeAll + public static void setupServer() + { + server = new TestingInfluxServer(); + InfluxConfig config = new InfluxConfig(); + config.setEndpoint(server.getEndpoint()); + config.setUsername(USERNAME); + config.setPassword(PASSWORD); + client = new NativeInfluxClient(config); + split = new InfluxSplit(); + + try (InfluxSession session = new InfluxSession(server.getEndpoint())) { + InfluxDataTool tool = new InfluxDataTool(session); + tool.setUpDatabase(); + tool.setUpDataForTest(); + } + } + + @AfterAll + public static void destroy() + { + server.close(); + server = null; + } + + @Test + public void testGetRecordSet() + { + InfluxTableHandle tableHandle = new InfluxTableHandle(TEST_DATABASE, TEST_MEASUREMENT_X); + InfluxRecordSetProvider provider = new InfluxRecordSetProvider(client); + RecordSet recordSet = provider.getRecordSet(INSTANCE, SESSION, split, tableHandle, ImmutableList.of()); + InfluxRecordSet influxRecordSet = (InfluxRecordSet) recordSet; + assertThat(influxRecordSet).isNotNull(); + assertThat(influxRecordSet.getSourceData()).isNotNull(); + assertThat(influxRecordSet.getSourceData().getColumns().containsAll( + ImmutableList.of("time", "f1", "f2", "f3", "f4", "country")) + ).isTrue(); + assertThat(influxRecordSet.getSourceData().getValues().size()).isEqualTo(100); + } +} diff --git a/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestInfluxTypeMapping.java b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestInfluxTypeMapping.java new file mode 100644 index 000000000000..49fe24457fb2 --- /dev/null +++ b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestInfluxTypeMapping.java @@ -0,0 +1,157 @@ +/* + * 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.influxdb; + +import com.google.common.collect.ImmutableList; +import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.QueryRunner; +import io.trino.testing.datatype.DataSetup; +import io.trino.testing.datatype.SqlDataTypeTest; +import org.junit.jupiter.api.Test; + +import static io.trino.plugin.influxdb.InfluxQueryRunner.createInfluxQueryRunner; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.BooleanType.BOOLEAN; +import static io.trino.spi.type.DoubleType.DOUBLE; +import static io.trino.spi.type.TimestampType.TIMESTAMP_NANOS; +import static io.trino.spi.type.VarcharType.VARCHAR; + +public class TestInfluxTypeMapping + extends AbstractTestQueryFramework +{ + private TestingInfluxServer server; + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + server = closeAfterClass(new TestingInfluxServer()); + return createInfluxQueryRunner(server, ImmutableList.of()); + } + + @Test + public void testBoolean() + { + SqlDataTypeTest.create() + .addRoundTrip("time", "timestamp", "2020-01-01T00:00:00Z", TIMESTAMP_NANOS, "CAST('2020-01-01 00:00:00' as timestamp(9))") + .addRoundTrip("col_1", "boolean", "true", BOOLEAN, "true") + .addRoundTrip("col_2", "boolean", "false", BOOLEAN, "false") + .execute(getQueryRunner(), influxCreateAndInsert("test_boolean")); + } + + @Test + public void testInteger() + { + SqlDataTypeTest.create() + .addRoundTrip("time", "timestamp", "2020-01-01T00:00:00Z", TIMESTAMP_NANOS, "CAST('2020-01-01 00:00:00' as timestamp(9))") + .addRoundTrip("col_1", "integer", "-128", BIGINT, "CAST('-128' as bigint)") + .addRoundTrip("col_2", "integer", "5", BIGINT, "CAST('5' as bigint)") + .addRoundTrip("col_3", "integer", "127", BIGINT, "CAST('127' as bigint)") + .execute(getQueryRunner(), influxCreateAndInsert("test_integer")); + + SqlDataTypeTest.create() + .addRoundTrip("time", "timestamp", "2020-01-02T00:00:00Z", TIMESTAMP_NANOS, "CAST('2020-01-02 00:00:00' as timestamp(9))") + .addRoundTrip("col_1", "integer", "-32768", BIGINT, "CAST('-32768' as bigint)") + .addRoundTrip("col_2", "integer", "32456", BIGINT, "CAST('32456' as bigint)") + .addRoundTrip("col_3", "integer", "32767", BIGINT, "CAST('32767' as bigint)") + .execute(getQueryRunner(), influxCreateAndInsert("test_integer")); + + SqlDataTypeTest.create() + .addRoundTrip("time", "timestamp", "2020-01-03T00:00:00Z", TIMESTAMP_NANOS, "CAST('2020-01-03 00:00:00' as timestamp(9))") + .addRoundTrip("col_1", "integer", "-2147483648", BIGINT, "CAST('-2147483648' as bigint)") + .addRoundTrip("col_2", "integer", "1234567890", BIGINT, "CAST('1234567890' as bigint)") + .addRoundTrip("col_3", "integer", "2147483647", BIGINT, "CAST('2147483647' as bigint)") + .execute(getQueryRunner(), influxCreateAndInsert("test_integer")); + + SqlDataTypeTest.create() + .addRoundTrip("time", "timestamp", "2020-01-04T00:00:00Z", TIMESTAMP_NANOS, "CAST('2020-01-04 00:00:00' as timestamp(9))") + .addRoundTrip("col_1", "integer", "-9223372036854775808", BIGINT, "-9223372036854775808") + .addRoundTrip("col_2", "integer", "123456789012", BIGINT, "123456789012") + .addRoundTrip("col_3", "integer", "9223372036854775807", BIGINT, "9223372036854775807") + .execute(getQueryRunner(), influxCreateAndInsert("test_integer")); + } + + @Test + public void testFloat() + { + SqlDataTypeTest.create() + .addRoundTrip("time", "timestamp", "2020-01-01T00:00:00Z", TIMESTAMP_NANOS, "CAST('2020-01-01 00:00:00' as timestamp(9))") + .addRoundTrip("col_1", "float", "-193", DOUBLE, "CAST('-193' as double)") + .addRoundTrip("col_2", "float", "-10.1", DOUBLE, "CAST('-10.1' as double)") + .addRoundTrip("col_3", "float", "123456789.3", DOUBLE, "CAST('123456789.3' as double)") + .addRoundTrip("col_4", "float", "12345678901234567890.31", DOUBLE, "CAST('12345678901234567890.31' as double)") + .execute(getQueryRunner(), influxCreateAndInsert("test_float")); + + SqlDataTypeTest.create() + .addRoundTrip("time", "timestamp", "2020-01-02T00:00:00Z", TIMESTAMP_NANOS, "CAST('2020-01-02 00:00:00' as timestamp(9))") + .addRoundTrip("col_1", "float", "1.0E100", DOUBLE, "1.0E100") + .addRoundTrip("col_2", "float", "123.456E10", DOUBLE, "123.456E10") + .execute(getQueryRunner(), influxCreateAndInsert("test_float")); + } + + @Test + public void testString() + { + SqlDataTypeTest.create() + .addRoundTrip("time", "timestamp", "2020-01-01T00:00:00Z", TIMESTAMP_NANOS, "CAST('2020-01-01 00:00:00' as timestamp(9))") + .addRoundTrip("col_1", "string", "text_a", VARCHAR, "CAST('text_a' as varchar)") + .addRoundTrip("col_2", "string", "12345678", VARCHAR, "CAST('12345678' as varchar)") + .addRoundTrip("col_3", "string", "攻殻機動隊", VARCHAR, "CAST('攻殻機動隊' as varchar)") + .execute(getQueryRunner(), influxCreateAndInsert("test_string")); + } + + @Test + public void testTimestamp() + { + SqlDataTypeTest.create() + .addRoundTrip("time", "timestamp", "2020-01-01T00:00:00Z", TIMESTAMP_NANOS, "CAST('2020-01-01 00:00:00' as timestamp(9))") + .addRoundTrip("string", "required", VARCHAR, "CAST('required' as varchar)") + .execute(getQueryRunner(), influxCreateAndInsert("test_timestamp")); + + SqlDataTypeTest.create() + .addRoundTrip("time", "timestamp", "2020-01-02T00:00:00.000Z", TIMESTAMP_NANOS, "CAST('2020-01-02 00:00:00' as timestamp(9))") + .addRoundTrip("string", "required", VARCHAR, "CAST('required' as varchar)") + .execute(getQueryRunner(), influxCreateAndInsert("test_timestamp")); + + SqlDataTypeTest.create() + .addRoundTrip("time", "timestamp", "2020-01-03T00:00:00.000000Z", TIMESTAMP_NANOS, "CAST('2020-01-03 00:00:00' as timestamp(9))") + .addRoundTrip("string", "required", VARCHAR, "CAST('required' as varchar)") + .execute(getQueryRunner(), influxCreateAndInsert("test_timestamp")); + + SqlDataTypeTest.create() + .addRoundTrip("time", "timestamp", "2020-01-04T00:00:00.000000000Z", TIMESTAMP_NANOS, "CAST('2020-01-04 00:00:00' as timestamp(9))") + .addRoundTrip("string", "required", VARCHAR, "CAST('required' as varchar)") + .execute(getQueryRunner(), influxCreateAndInsert("test_timestamp")); + + SqlDataTypeTest.create() + .addRoundTrip("time", "timestamp", "2020-01-01T00:00:00.123Z", TIMESTAMP_NANOS, "CAST('2020-01-01 00:00:00.123' as timestamp(9))") + .addRoundTrip("string", "required", VARCHAR, "CAST('required' as varchar)") + .execute(getQueryRunner(), influxCreateAndInsert("test_timestamp")); + + SqlDataTypeTest.create() + .addRoundTrip("time", "timestamp", "2020-01-01T00:00:00.123456Z", TIMESTAMP_NANOS, "CAST('2020-01-01 00:00:00.123456' as timestamp(9))") + .addRoundTrip("string", "required", VARCHAR, "CAST('required' as varchar)") + .execute(getQueryRunner(), influxCreateAndInsert("test_timestamp")); + + SqlDataTypeTest.create() + .addRoundTrip("time", "timestamp", "2020-01-01T00:00:00.123456789Z", TIMESTAMP_NANOS, "CAST('2020-01-01 00:00:00.123456789' as timestamp(9))") + .addRoundTrip("string", "required", VARCHAR, "CAST('required' as varchar)") + .execute(getQueryRunner(), influxCreateAndInsert("test_timestamp")); + } + + private DataSetup influxCreateAndInsert(String tableNamePrefix) + { + return new InfluxDataSetup(new InfluxSession(server.getEndpoint()), tableNamePrefix); + } +} diff --git a/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestNativeInfluxClient.java b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestNativeInfluxClient.java new file mode 100644 index 000000000000..c17df66121ad --- /dev/null +++ b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestNativeInfluxClient.java @@ -0,0 +1,28 @@ +/* + * 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.influxdb; + +import org.junit.jupiter.api.Test; + +import static io.trino.spi.testing.InterfaceTestUtils.assertAllMethodsOverridden; + +public class TestNativeInfluxClient +{ + @Test + public void testEverythingImplemented() + { + assertAllMethodsOverridden(InfluxClient.class, NativeInfluxClient.class); + } +} diff --git a/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestingInfluxServer.java b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestingInfluxServer.java new file mode 100644 index 000000000000..52270c71b1b2 --- /dev/null +++ b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestingInfluxServer.java @@ -0,0 +1,86 @@ +/* + * 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.influxdb; + +import io.airlift.log.Logger; +import io.trino.testing.ResourcePresence; +import org.testcontainers.containers.InfluxDBContainer; +import org.testcontainers.utility.DockerImageName; + +import static com.google.common.base.Strings.isNullOrEmpty; + +public class TestingInfluxServer + implements AutoCloseable +{ + private static final Logger log = Logger.get(TestingInfluxServer.class); + + public static final String VERSION = "1.8.10"; + public static final String USERNAME = "admin"; + public static final String PASSWORD = "password"; + private static final int PORT = 8086; + private InfluxDBContainer dockerContainer; + + public TestingInfluxServer() + { + this(VERSION); + } + + public TestingInfluxServer(String influxVersion) + { + log.info("Influx server starting..."); + + DockerImageName dockerImageName = DockerImageName.parse("influxdb").withTag(influxVersion); + try { + this.dockerContainer = new InfluxDBContainer<>(dockerImageName) + .withExposedPorts(PORT) + .withAdmin(USERNAME) + .withAdminPassword(PASSWORD) + .withUsername("test") + .withPassword("password") + .withNetworkAliases("influxdb"); + this.dockerContainer.start(); + + try (InfluxSession session = new InfluxSession(getEndpoint())) { + String version = session.checkConnectivity(); + log.info("Influx server connectivity check is %s.", !isNullOrEmpty(version) ? "success" : "fail"); + } + } + catch (Exception e) { + if (this.dockerContainer != null) { + this.dockerContainer.close(); + } + throw e; + } + + log.info("Influx server started."); + } + + public String getEndpoint() + { + return this.dockerContainer.getUrl(); + } + + @Override + public void close() + { + this.dockerContainer.close(); + } + + @ResourcePresence + public boolean isRunning() + { + return dockerContainer.getContainerId() != null; + } +} diff --git a/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestingInfluxTable.java b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestingInfluxTable.java new file mode 100644 index 000000000000..43a193f98f2a --- /dev/null +++ b/plugin/trino-influxdb/src/test/java/io/trino/plugin/influxdb/TestingInfluxTable.java @@ -0,0 +1,42 @@ +/* + * 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.influxdb; + +import io.trino.testing.sql.TemporaryRelation; + +import static io.trino.testing.TestingNames.randomNameSuffix; +import static java.util.Objects.requireNonNull; + +public class TestingInfluxTable + implements TemporaryRelation +{ + private final String tableName; + + public TestingInfluxTable(String namePrefix) + { + requireNonNull(namePrefix, "namePrefix is null"); + this.tableName = namePrefix + randomNameSuffix(); + } + + @Override + public String getName() + { + return tableName; + } + + @Override + public void close() + { + } +} diff --git a/pom.xml b/pom.xml index 88b50585fc4e..a186b21d5f9e 100644 --- a/pom.xml +++ b/pom.xml @@ -83,6 +83,7 @@ plugin/trino-hudi plugin/trino-iceberg plugin/trino-ignite + plugin/trino-influxdb plugin/trino-jmx plugin/trino-kafka plugin/trino-kafka-event-listener @@ -1164,6 +1165,12 @@ ${project.version} + + io.trino + trino-influxdb + ${project.version} + + io.trino trino-jdbc diff --git a/testing/trino-product-tests-groups/src/main/java/io/trino/tests/product/TestGroups.java b/testing/trino-product-tests-groups/src/main/java/io/trino/tests/product/TestGroups.java index 4c9057bd9b5a..c975b2ebed07 100644 --- a/testing/trino-product-tests-groups/src/main/java/io/trino/tests/product/TestGroups.java +++ b/testing/trino-product-tests-groups/src/main/java/io/trino/tests/product/TestGroups.java @@ -105,6 +105,7 @@ public final class TestGroups public static final String HUDI = "hudi"; public static final String PARQUET = "parquet"; public static final String IGNITE = "ignite"; + public static final String INFLUXDB = "influxdb"; public static final String FAULT_TOLERANT = "fault-tolerant"; private TestGroups() {} diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeAllConnectors.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeAllConnectors.java index 6b3b46c5b305..2c4c9a1fcea2 100644 --- a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeAllConnectors.java +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeAllConnectors.java @@ -59,6 +59,7 @@ public void extendEnvironment(Environment.Builder builder) "hudi", "iceberg", "ignite", + "influxdb", "kafka", "kudu", "loki", diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeInfluxdb.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeInfluxdb.java new file mode 100644 index 000000000000..953429f850b2 --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvMultinodeInfluxdb.java @@ -0,0 +1,82 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.tests.product.launcher.env.environment; + +import com.google.inject.Inject; +import io.trino.tests.product.launcher.docker.DockerFiles; +import io.trino.tests.product.launcher.env.DockerContainer; +import io.trino.tests.product.launcher.env.Environment; +import io.trino.tests.product.launcher.env.EnvironmentProvider; +import io.trino.tests.product.launcher.env.common.StandardMultinode; +import io.trino.tests.product.launcher.env.common.TestsEnvironment; +import io.trino.tests.product.launcher.testcontainers.PortBinder; +import org.testcontainers.containers.startupcheck.IsRunningStartupCheckStrategy; + +import java.time.Duration; + +import static io.trino.tests.product.launcher.env.EnvironmentContainers.configureTempto; +import static java.util.Objects.requireNonNull; +import static org.testcontainers.containers.wait.strategy.Wait.forHttp; +import static org.testcontainers.utility.MountableFile.forHostPath; + +@TestsEnvironment +public class EnvMultinodeInfluxdb + extends EnvironmentProvider +{ + private static final String INFLUXDB = "influxdb"; + private static final int INFLUXDB_PORT = 8086; + public static final String USERNAME = "admin"; + public static final String PASSWORD = "password"; + + private final DockerFiles.ResourceProvider configDir; + private final PortBinder portBinder; + + @Inject + public EnvMultinodeInfluxdb(StandardMultinode standardMultinode, DockerFiles dockerFiles, PortBinder portBinder) + { + super(standardMultinode); + this.configDir = dockerFiles.getDockerFilesHostDirectory("conf/environment/multinode-influxdb/"); + this.portBinder = requireNonNull(portBinder, "portBinder is null"); + } + + @Override + public void extendEnvironment(Environment.Builder builder) + { + builder.addContainer(createInfluxDb()); + builder.addConnector(INFLUXDB, forHostPath(configDir.getPath("influxdb.properties"))); + configureTempto(builder, configDir); + } + + private DockerContainer createInfluxDb() + { + DockerContainer container = new DockerContainer("influxdb:1.8.10", "influxdb") + .withEnv("INFLUXDB_DB", "presto") + .withEnv("INFLUXDB_ADMIN_USER", USERNAME) + .withEnv("INFLUXDB_ADMIN_PASSWORD", PASSWORD) + .withEnv("INFLUXDB_USER", "test") + .withEnv("INFLUXDB_USER_PASSWORD", "password") + .withEnv("INFLUXDB_INIT_DATAFILE", "init-influxdb.data") + .withExposedPorts(INFLUXDB_PORT) + .withCopyFileToContainer(forHostPath(configDir.getPath("init-influxdb.sh")), "/init-influxdb.sh") + .withCopyFileToContainer(forHostPath(configDir.getPath("init-influxdb.data")), "/init-influxdb.data") + .withStartupCheckStrategy(new IsRunningStartupCheckStrategy()) + .waitingFor(forHttp("/health").forPort(INFLUXDB_PORT).forStatusCode(200)) + .withStartupTimeout(Duration.ofMinutes(1)); + + portBinder.exposePort(container, INFLUXDB_PORT); + + return container; + } +} diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/SuiteInfluxDB.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/SuiteInfluxDB.java new file mode 100644 index 000000000000..0dba85002e20 --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/suite/suites/SuiteInfluxDB.java @@ -0,0 +1,39 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.tests.product.launcher.suite.suites; + +import com.google.common.collect.ImmutableList; +import io.trino.tests.product.launcher.env.EnvironmentConfig; +import io.trino.tests.product.launcher.env.environment.EnvMultinodeInfluxdb; +import io.trino.tests.product.launcher.suite.Suite; +import io.trino.tests.product.launcher.suite.SuiteTestRun; + +import java.util.List; + +import static io.trino.tests.product.TestGroups.CONFIGURED_FEATURES; +import static io.trino.tests.product.TestGroups.INFLUXDB; +import static io.trino.tests.product.launcher.suite.SuiteTestRun.testOnEnvironment; + +public class SuiteInfluxDB + extends Suite +{ + @Override + public List getTestRuns(EnvironmentConfig config) + { + return ImmutableList.of( + testOnEnvironment(EnvMultinodeInfluxdb.class) + .withGroups(CONFIGURED_FEATURES, INFLUXDB) + .build()); + } +} diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-all/influxdb.properties b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-all/influxdb.properties new file mode 100644 index 000000000000..08a64d58e00e --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-all/influxdb.properties @@ -0,0 +1,4 @@ +connector.name=influxdb +influx.endpoint=http://influxdb:8086 +influx.username=admin +influx.password=password diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-influxdb/influxdb.properties b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-influxdb/influxdb.properties new file mode 100644 index 000000000000..08a64d58e00e --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-influxdb/influxdb.properties @@ -0,0 +1,4 @@ +connector.name=influxdb +influx.endpoint=http://influxdb:8086 +influx.username=admin +influx.password=password diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-influxdb/init-influxdb.data b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-influxdb/init-influxdb.data new file mode 100644 index 000000000000..ddb030a1c787 --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-influxdb/init-influxdb.data @@ -0,0 +1,120 @@ +cpu_load_1,host=server1 value=3.1 1673353784866000000 +cpu_load_1,host=server2 value=7.8 1673353784866000000 +cpu_load_1,host=server3 value=6.0 1673353784866000000 +cpu_load_1,host=server4 value=1.3 1673353784866000000 +cpu_load_5,host=server1 value=2.0 1673353784866000000 +cpu_load_5,host=server2 value=0.9 1673353784866000000 +cpu_load_5,host=server3 value=0.3 1673353784866000000 +cpu_load_5,host=server4 value=1.3 1673353784866000000 +cpu_load_15,host=server1 value=5.9 1673353784866000000 +cpu_load_15,host=server2 value=8.7 1673353784866000000 +cpu_load_15,host=server3 value=5.2 1673353784866000000 +cpu_load_15,host=server4 value=4.2 1673353784866000000 +cpu_load_1,host=server1 value=4.3 1673353785866000000 +cpu_load_1,host=server2 value=9.3 1673353785866000000 +cpu_load_1,host=server3 value=7.9 1673353785866000000 +cpu_load_1,host=server4 value=1.7 1673353785866000000 +cpu_load_5,host=server1 value=4.8 1673353785866000000 +cpu_load_5,host=server2 value=5.5 1673353785866000000 +cpu_load_5,host=server3 value=7.7 1673353785866000000 +cpu_load_5,host=server4 value=0.2 1673353785866000000 +cpu_load_15,host=server1 value=7.9 1673353785866000000 +cpu_load_15,host=server2 value=9.8 1673353785866000000 +cpu_load_15,host=server3 value=7.1 1673353785866000000 +cpu_load_15,host=server4 value=5.5 1673353785866000000 +cpu_load_1,host=server1 value=2.7 1673353786866000000 +cpu_load_1,host=server2 value=7.4 1673353786866000000 +cpu_load_1,host=server3 value=5.8 1673353786866000000 +cpu_load_1,host=server4 value=2.7 1673353786866000000 +cpu_load_5,host=server1 value=9.8 1673353786866000000 +cpu_load_5,host=server2 value=9.2 1673353786866000000 +cpu_load_5,host=server3 value=9.3 1673353786866000000 +cpu_load_5,host=server4 value=5.0 1673353786866000000 +cpu_load_15,host=server1 value=6.8 1673353786866000000 +cpu_load_15,host=server2 value=4.4 1673353786866000000 +cpu_load_15,host=server3 value=2.1 1673353786866000000 +cpu_load_15,host=server4 value=1.8 1673353786866000000 +cpu_load_1,host=server1 value=7.2 1673353787866000000 +cpu_load_1,host=server2 value=3.7 1673353787866000000 +cpu_load_1,host=server3 value=7.0 1673353787866000000 +cpu_load_1,host=server4 value=7.3 1673353787866000000 +cpu_load_5,host=server1 value=4.9 1673353787866000000 +cpu_load_5,host=server2 value=6.0 1673353787866000000 +cpu_load_5,host=server3 value=7.0 1673353787866000000 +cpu_load_5,host=server4 value=7.2 1673353787866000000 +cpu_load_15,host=server1 value=4.9 1673353787866000000 +cpu_load_15,host=server2 value=3.8 1673353787866000000 +cpu_load_15,host=server3 value=2.2 1673353787866000000 +cpu_load_15,host=server4 value=0.9 1673353787866000000 +cpu_load_1,host=server1 value=3.0 1673353788866000000 +cpu_load_1,host=server2 value=6.3 1673353788866000000 +cpu_load_1,host=server3 value=2.9 1673353788866000000 +cpu_load_1,host=server4 value=4.1 1673353788866000000 +cpu_load_5,host=server1 value=3.6 1673353788866000000 +cpu_load_5,host=server2 value=7.0 1673353788866000000 +cpu_load_5,host=server3 value=0.4 1673353788866000000 +cpu_load_5,host=server4 value=3.5 1673353788866000000 +cpu_load_15,host=server1 value=9.2 1673353788866000000 +cpu_load_15,host=server2 value=7.1 1673353788866000000 +cpu_load_15,host=server3 value=8.6 1673353788866000000 +cpu_load_15,host=server4 value=7.6 1673353788866000000 +cpu_load_1,host=server1 value=8.0 1673353789866000000 +cpu_load_1,host=server2 value=3.6 1673353789866000000 +cpu_load_1,host=server3 value=7.7 1673353789866000000 +cpu_load_1,host=server4 value=3.4 1673353789866000000 +cpu_load_5,host=server1 value=1.6 1673353789866000000 +cpu_load_5,host=server2 value=4.0 1673353789866000000 +cpu_load_5,host=server3 value=4.4 1673353789866000000 +cpu_load_5,host=server4 value=2.2 1673353789866000000 +cpu_load_15,host=server1 value=7.7 1673353789866000000 +cpu_load_15,host=server2 value=8.7 1673353789866000000 +cpu_load_15,host=server3 value=5.4 1673353789866000000 +cpu_load_15,host=server4 value=0.0 1673353789866000000 +cpu_load_1,host=server1 value=8.9 1673353790866000000 +cpu_load_1,host=server2 value=9.0 1673353790866000000 +cpu_load_1,host=server3 value=2.4 1673353790866000000 +cpu_load_1,host=server4 value=5.8 1673353790866000000 +cpu_load_5,host=server1 value=8.6 1673353790866000000 +cpu_load_5,host=server2 value=2.8 1673353790866000000 +cpu_load_5,host=server3 value=0.5 1673353790866000000 +cpu_load_5,host=server4 value=9.9 1673353790866000000 +cpu_load_15,host=server1 value=0.2 1673353790866000000 +cpu_load_15,host=server2 value=1.6 1673353790866000000 +cpu_load_15,host=server3 value=9.4 1673353790866000000 +cpu_load_15,host=server4 value=3.7 1673353790866000000 +cpu_load_1,host=server1 value=3.2 1673353791866000000 +cpu_load_1,host=server2 value=0.4 1673353791866000000 +cpu_load_1,host=server3 value=0.1 1673353791866000000 +cpu_load_1,host=server4 value=2.0 1673353791866000000 +cpu_load_5,host=server1 value=7.3 1673353791866000000 +cpu_load_5,host=server2 value=1.4 1673353791866000000 +cpu_load_5,host=server3 value=4.0 1673353791866000000 +cpu_load_5,host=server4 value=8.4 1673353791866000000 +cpu_load_15,host=server1 value=5.2 1673353791866000000 +cpu_load_15,host=server2 value=2.5 1673353791866000000 +cpu_load_15,host=server3 value=4.2 1673353791866000000 +cpu_load_15,host=server4 value=7.2 1673353791866000000 +cpu_load_1,host=server1 value=5.1 1673353792866000000 +cpu_load_1,host=server2 value=9.9 1673353792866000000 +cpu_load_1,host=server3 value=2.5 1673353792866000000 +cpu_load_1,host=server4 value=7.6 1673353792866000000 +cpu_load_5,host=server1 value=2.1 1673353792866000000 +cpu_load_5,host=server2 value=0.7 1673353792866000000 +cpu_load_5,host=server3 value=1.7 1673353792866000000 +cpu_load_5,host=server4 value=3.2 1673353792866000000 +cpu_load_15,host=server1 value=3.6 1673353792866000000 +cpu_load_15,host=server2 value=1.5 1673353792866000000 +cpu_load_15,host=server3 value=2.8 1673353792866000000 +cpu_load_15,host=server4 value=1.9 1673353792866000000 +cpu_load_1,host=server1 value=6.3 1673353793866000000 +cpu_load_1,host=server2 value=4.7 1673353793866000000 +cpu_load_1,host=server3 value=0.6 1673353793866000000 +cpu_load_1,host=server4 value=0.5 1673353793866000000 +cpu_load_5,host=server1 value=0.9 1673353793866000000 +cpu_load_5,host=server2 value=3.4 1673353793866000000 +cpu_load_5,host=server3 value=2.4 1673353793866000000 +cpu_load_5,host=server4 value=7.8 1673353793866000000 +cpu_load_15,host=server1 value=0.4 1673353793866000000 +cpu_load_15,host=server2 value=2.2 1673353793866000000 +cpu_load_15,host=server3 value=5.9 1673353793866000000 +cpu_load_15,host=server4 value=2.7 1673353793866000000 diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-influxdb/init-influxdb.sh b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-influxdb/init-influxdb.sh new file mode 100644 index 000000000000..431e8f79a07d --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-influxdb/init-influxdb.sh @@ -0,0 +1,138 @@ +#!/bin/bash +set -e + +AUTH_ENABLED="$INFLUXDB_HTTP_AUTH_ENABLED" + +if [ -z "$AUTH_ENABLED" ]; then + AUTH_ENABLED="$(grep -iE '^\s*auth-enabled\s*=\s*true' /etc/influxdb/influxdb.conf | grep -io 'true' | cat)" +else + AUTH_ENABLED="$(echo "$INFLUXDB_HTTP_AUTH_ENABLED" | grep -io 'true' | cat)" +fi + +INIT_USERS=$([ ! -z "$AUTH_ENABLED" ] && [ ! -z "$INFLUXDB_ADMIN_USER" ] && echo 1 || echo) + +# Check if an environment variable for where to put meta is set. +# If so, then use that directory, otherwise use the default. +if [ -z "$INFLUXDB_META_DIR" ]; then + META_DIR="/var/lib/influxdb/meta" +else + META_DIR="$INFLUXDB_META_DIR" +fi + +if ( [ ! -z "$INIT_USERS" ] || [ ! -z "$INFLUXDB_DB" ] || [ "$(ls -A /docker-entrypoint-initdb.d 2> /dev/null)" ] ) && [ ! "$(ls -d "$META_DIR" 2>/dev/null)" ]; then + + INIT_QUERY="" + CREATE_DB_QUERY="CREATE DATABASE $INFLUXDB_DB" + + if [ ! -z "$INIT_USERS" ]; then + + if [ -z "$INFLUXDB_ADMIN_PASSWORD" ]; then + INFLUXDB_ADMIN_PASSWORD="$(< /dev/urandom tr -dc _A-Z-a-z-0-9 | head -c32;echo;)" + echo "INFLUXDB_ADMIN_PASSWORD:$INFLUXDB_ADMIN_PASSWORD" + fi + + INIT_QUERY="CREATE USER \"$INFLUXDB_ADMIN_USER\" WITH PASSWORD '$INFLUXDB_ADMIN_PASSWORD' WITH ALL PRIVILEGES" + elif [ ! -z "$INFLUXDB_DB" ]; then + INIT_QUERY="$CREATE_DB_QUERY" + else + INIT_QUERY="SHOW DATABASES" + fi + + INFLUXDB_INIT_PORT="8086" + + INFLUXDB_HTTP_BIND_ADDRESS=127.0.0.1:$INFLUXDB_INIT_PORT INFLUXDB_HTTP_HTTPS_ENABLED=false influxd "$@" & + pid="$!" + + INFLUX_CMD="influx -host 127.0.0.1 -port $INFLUXDB_INIT_PORT -execute " + + for i in {30..0}; do + if $INFLUX_CMD "$INIT_QUERY" &> /dev/null; then + break + fi + echo 'influxdb init process in progress...' + sleep 1 + done + + #custom initialized for trino product tests + if [ ! -z "$INFLUXDB_DB" ] && [ ! -z "$INFLUXDB_INIT_DATAFILE" ]; then + INFLUX_CMD="influx -database $INFLUXDB_DB -host 127.0.0.1 -port $INFLUXDB_INIT_PORT -execute " + echo "init data test.." + while read -r line + do + $INFLUX_CMD "INSERT $line" &> /dev/null; + done < "$INFLUXDB_INIT_DATAFILE" + fi + + if [ "$i" = 0 ]; then + echo >&2 'influxdb init process failed.' + exit 1 + fi + + if [ ! -z "$INIT_USERS" ]; then + + INFLUX_CMD="influx -host 127.0.0.1 -port $INFLUXDB_INIT_PORT -username ${INFLUXDB_ADMIN_USER} -password ${INFLUXDB_ADMIN_PASSWORD} -execute " + + if [ ! -z "$INFLUXDB_DB" ]; then + $INFLUX_CMD "$CREATE_DB_QUERY" + fi + + if [ ! -z "$INFLUXDB_USER" ] && [ -z "$INFLUXDB_USER_PASSWORD" ]; then + INFLUXDB_USER_PASSWORD="$(< /dev/urandom tr -dc _A-Z-a-z-0-9 | head -c32;echo;)" + echo "INFLUXDB_USER_PASSWORD:$INFLUXDB_USER_PASSWORD" + fi + + if [ ! -z "$INFLUXDB_USER" ]; then + $INFLUX_CMD "CREATE USER \"$INFLUXDB_USER\" WITH PASSWORD '$INFLUXDB_USER_PASSWORD'" + + $INFLUX_CMD "REVOKE ALL PRIVILEGES FROM \"$INFLUXDB_USER\"" + + if [ ! -z "$INFLUXDB_DB" ]; then + $INFLUX_CMD "GRANT ALL ON \"$INFLUXDB_DB\" TO \"$INFLUXDB_USER\"" + fi + fi + + if [ ! -z "$INFLUXDB_WRITE_USER" ] && [ -z "$INFLUXDB_WRITE_USER_PASSWORD" ]; then + INFLUXDB_WRITE_USER_PASSWORD="$(< /dev/urandom tr -dc _A-Z-a-z-0-9 | head -c32;echo;)" + echo "INFLUXDB_WRITE_USER_PASSWORD:$INFLUXDB_WRITE_USER_PASSWORD" + fi + + if [ ! -z "$INFLUXDB_WRITE_USER" ]; then + $INFLUX_CMD "CREATE USER \"$INFLUXDB_WRITE_USER\" WITH PASSWORD '$INFLUXDB_WRITE_USER_PASSWORD'" + $INFLUX_CMD "REVOKE ALL PRIVILEGES FROM \"$INFLUXDB_WRITE_USER\"" + + if [ ! -z "$INFLUXDB_DB" ]; then + $INFLUX_CMD "GRANT WRITE ON \"$INFLUXDB_DB\" TO \"$INFLUXDB_WRITE_USER\"" + fi + fi + + if [ ! -z "$INFLUXDB_READ_USER" ] && [ -z "$INFLUXDB_READ_USER_PASSWORD" ]; then + INFLUXDB_READ_USER_PASSWORD="$(< /dev/urandom tr -dc _A-Z-a-z-0-9 | head -c32;echo;)" + echo "INFLUXDB_READ_USER_PASSWORD:$INFLUXDB_READ_USER_PASSWORD" + fi + + if [ ! -z "$INFLUXDB_READ_USER" ]; then + $INFLUX_CMD "CREATE USER \"$INFLUXDB_READ_USER\" WITH PASSWORD '$INFLUXDB_READ_USER_PASSWORD'" + $INFLUX_CMD "REVOKE ALL PRIVILEGES FROM \"$INFLUXDB_READ_USER\"" + + if [ ! -z "$INFLUXDB_DB" ]; then + $INFLUX_CMD "GRANT READ ON \"$INFLUXDB_DB\" TO \"$INFLUXDB_READ_USER\"" + fi + fi + + fi + + for f in /docker-entrypoint-initdb.d/*; do + case "$f" in + *.sh) echo "$0: running $f"; . "$f" ;; + *.iql) echo "$0: running $f"; $INFLUX_CMD "$(cat ""$f"")"; echo ;; + *) echo "$0: ignoring $f" ;; + esac + echo + done + + if ! kill -s TERM "$pid" || ! wait "$pid"; then + echo >&2 'influxdb init process failed. (Could not stop influxdb)' + exit 1 + fi + +fi diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-influxdb/tempto-configuration.yaml b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-influxdb/tempto-configuration.yaml new file mode 100644 index 000000000000..5f6a7d157024 --- /dev/null +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/multinode-influxdb/tempto-configuration.yaml @@ -0,0 +1,3 @@ +databases: + presto: + jdbc_url: "jdbc:trino://${databases.presto.host}:${databases.presto.port}/influxdb/test" diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/influxdb/TestInfluxdb.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/influxdb/TestInfluxdb.java new file mode 100644 index 000000000000..c9599f32987c --- /dev/null +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/influxdb/TestInfluxdb.java @@ -0,0 +1,67 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.tests.product.influxdb; + +import io.trino.tempto.ProductTest; +import io.trino.tempto.query.QueryResult; +import org.testng.annotations.Test; + +import static io.trino.tempto.assertions.QueryAssert.Row.row; +import static io.trino.tempto.assertions.QueryAssert.assertThat; +import static io.trino.tests.product.TestGroups.INFLUXDB; +import static io.trino.tests.product.TestGroups.PROFILE_SPECIFIC_TESTS; +import static io.trino.tests.product.utils.QueryExecutors.onTrino; + +public class TestInfluxdb + extends ProductTest +{ + @Test(groups = {INFLUXDB, PROFILE_SPECIFIC_TESTS}) + public void testShowSchemas() + { + QueryResult result = onTrino().executeQuery("SHOW SCHEMAS FROM influxdb"); + assertThat(result).contains(row("presto")); + } + + @Test(groups = {INFLUXDB, PROFILE_SPECIFIC_TESTS}) + public void testShowTable() + { + QueryResult result = onTrino().executeQuery("SHOW TABLES FROM influxdb.presto"); + assertThat(result).contains(row("cpu_load_1"), row("cpu_load_5"), row("cpu_load_15")); + } + + @Test(groups = {INFLUXDB, PROFILE_SPECIFIC_TESTS}) + public void testSelect() + { + assertThat(onTrino().executeQuery("SELECT * FROM influxdb.presto.cpu_load_1 LIMIT 5")) + .hasRowsCount(5) + .hasColumnsCount(3); + + assertThat(onTrino().executeQuery("SELECT * FROM influxdb.presto.cpu_load_1 WHERE host='none'")) + .hasNoRows(); + + assertThat(onTrino().executeQuery("SELECT host, value FROM influxdb.presto.cpu_load_5 LIMIT 5")) + .hasRowsCount(5) + .hasColumnsCount(2); + + assertThat(onTrino().executeQuery("SELECT host, value FROM influxdb.presto.cpu_load_5 where value < -1")) + .hasNoRows(); + + assertThat(onTrino().executeQuery("SELECT time, value FROM influxdb.presto.cpu_load_15 WHERE host='server1' LIMIT 10")) + .hasRowsCount(10) + .hasColumnsCount(2); + + assertThat(onTrino().executeQuery("SELECT time, value FROM influxdb.presto.cpu_load_15 where value < -1")) + .hasNoRows(); + } +}