Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Influx Connector #2397

Closed
wants to merge 20 commits into from
Closed
Show file tree
Hide file tree
Changes from 13 commits
Commits
Show all changes
20 commits
Select commit Hold shift + click to select a range
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions .gitignore
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ temp-testng-customsuite.xml
test-output
.externalToolBuilders
*~
\#*
williame marked this conversation as resolved.
Show resolved Hide resolved
benchmark_outputs
*.pyc
*.class
Expand Down
1 change: 1 addition & 0 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -142,6 +142,7 @@
<module>presto-elasticsearch</module>
<module>presto-iceberg</module>
<module>presto-google-sheets</module>
<module>presto-influx</module>
</modules>

<dependencyManagement>
Expand Down
43 changes: 43 additions & 0 deletions presto-docs/src/main/sphinx/connector/influx.rst
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
================
Influx Connector
================

.. contents::
:local:
:backlinks: none
:depth: 1

Overview
--------

The Influx connector allows querying data-points stored in an
`InfluxDB <https://www.influxdata.com/products/influxdb-overview/>`_
Time Series Database.

Configuration
-------------

The following configuration properties are available:

================================================== ======================================================================
Property Name Description
================================================== ======================================================================
``connector.name=influx``
``influx.host=`` Default localhost
``influx.port=`` Default 8086
``influx.use-https=`` Default false
``influx.database=`` The database name must be specified. Each instance of the connector
can only connect to a single database on a server
``influx.username=``
``influx.password=``
``influx.cache-meta-data-millis=`` How long to cache schema info e.g. measurement names before refreshing
================================================== ======================================================================

Limitations
-----------

* Only SELECT queries are supported
* InfluxDB has case-sensitive identifiers, whereas prestosql is case-insenstive. The influx connector will report an error
if two identifiers differ only in case, and therefore are ambiguous
* authentication and https support is untested
* LDAP on InfluxDB Enterprise editions is not supported
130 changes: 130 additions & 0 deletions presto-influx/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,130 @@
<?xml version="1.0" encoding="UTF-8"?>
<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>

<parent>
<artifactId>presto-root</artifactId>
<groupId>io.prestosql</groupId>
<version>327-SNAPSHOT</version>
williame marked this conversation as resolved.
Show resolved Hide resolved
</parent>

<artifactId>presto-influx</artifactId>
<description>Presto - Influx Connector</description>
<packaging>presto-plugin</packaging>

<properties>
<air.main.basedir>${project.parent.basedir}</air.main.basedir>
</properties>

<dependencies>
<dependency>
<groupId>io.airlift</groupId>
<artifactId>bootstrap</artifactId>
</dependency>

<dependency>
<groupId>io.airlift</groupId>
<artifactId>configuration</artifactId>
</dependency>

<dependency>
<groupId>io.airlift</groupId>
<artifactId>json</artifactId>
</dependency>

<dependency>
<groupId>io.airlift</groupId>
<artifactId>log</artifactId>
</dependency>

<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
</dependency>

<dependency>
<groupId>com.google.inject</groupId>
<artifactId>guice</artifactId>
</dependency>

<dependency>
<groupId>javax.inject</groupId>
<artifactId>javax.inject</artifactId>
</dependency>

<dependency>
<groupId>javax.validation</groupId>
<artifactId>validation-api</artifactId>
</dependency>

<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
</dependency>

<!-- Presto SPI -->
<dependency>
<groupId>io.prestosql</groupId>
<artifactId>presto-spi</artifactId>
<scope>provided</scope>
</dependency>

<dependency>
<groupId>io.airlift</groupId>
<artifactId>slice</artifactId>
<scope>provided</scope>
</dependency>

<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-annotations</artifactId>
<scope>provided</scope>
</dependency>

<dependency>
<groupId>org.openjdk.jol</groupId>
<artifactId>jol-core</artifactId>
<scope>provided</scope>
</dependency>

<!-- for testing -->
<dependency>
<groupId>io.prestosql</groupId>
<artifactId>presto-main</artifactId>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.testng</groupId>
<artifactId>testng</artifactId>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.assertj</groupId>
<artifactId>assertj-core</artifactId>
<scope>test</scope>
</dependency>

<dependency>
<groupId>io.prestosql</groupId>
<artifactId>presto-testing</artifactId>
<scope>test</scope>
</dependency>

<dependency>
<groupId>io.airlift</groupId>
<artifactId>testing</artifactId>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.testcontainers</groupId>
<artifactId>influxdb</artifactId>
<scope>test</scope>
</dependency>
</dependencies>

</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,204 @@
/*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package io.prestosql.plugin.influx;

import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.node.JsonNodeType;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import io.airlift.log.Logger;
import io.prestosql.spi.HostAddress;

import javax.inject.Inject;

import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.function.Supplier;

import static java.util.Objects.requireNonNull;

public class InfluxClient
{
final Logger logger;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please make it private static final

private final InfluxConfig config;
private final InfluxHttp http;
// the various metadata are cached for a configurable number of milliseconds so we don't hammer the server
private final CachedMetaData<Map<String, String>> retentionPolicies; // schema name (lower-case) -> retention policy (case-sensitive)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Basically, we use guava for cache. You can see the existing code in MongoSession

While I think it worth to cache table definitions, I'm not sure if we should store schema and table names. Are those operations slow in InfluxDB?

private final CachedMetaData<Map<String, String>> measurements; // table name (lower-case) -> measurement (case-sensitive)
private final Map<String, CachedMetaData<Map<String, InfluxColumn>>> tagKeys; // column name (lower-case) -> tags
private final Map<String, Map<String, CachedMetaData<Map<String, InfluxColumn>>>> fields; // column name (lower-case) -> measurement -> fields

@Inject
public InfluxClient(InfluxConfig config)
{
this.logger = Logger.get(getClass());
this.config = requireNonNull(config, "config is null");
this.http = new InfluxHttp(config.getHost(), config.getPort(), config.isUseHttps(), config.getDatabase(), config.getUserName(), config.getPassword());
this.retentionPolicies = new CachedMetaData<>(() -> showNames("SHOW RETENTION POLICIES"));
this.measurements = new CachedMetaData<>(() -> showNames("SHOW MEASUREMENTS"));
this.tagKeys = new ConcurrentHashMap<>();
this.fields = new ConcurrentHashMap<>();
}

public Collection<String> getSchemaNames()
{
return retentionPolicies.get().keySet();
}

public String getRetentionPolicy(String schemaName)
{
return retentionPolicies.get().get(schemaName);
}

public Collection<String> getTableNames()
{
return measurements.get().keySet();
}

public String getMeasurement(String tableName)
{
return measurements.get().get(tableName);
}

// Influx tracks the tags in each measurement, but not which retention-policy they are used in
private Map<String, InfluxColumn> getTags(String tableName)
{
return tagKeys.computeIfAbsent(tableName,
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This method and getFields are too nested. Please try to avoid it.

k -> new CachedMetaData<>(() -> {
String measurement = measurements.get().get(tableName);
if (measurement == null) {
return Collections.emptyMap();
}
String query = new InfluxQL("SHOW TAG KEYS FROM ")
.addIdentifier(measurement)
.toString();
ImmutableMap.Builder<String, InfluxColumn> tags = new ImmutableMap.Builder<>();
for (Map.Entry<String, String> name : showNames(query).entrySet()) {
tags.put(name.getKey(), new InfluxColumn(name.getValue(), "string", InfluxColumn.Kind.TAG));
}
return tags.build();
}))
.get();
}

private Map<String, InfluxColumn> getFields(String schemaName, String tableName)
{
return fields.computeIfAbsent(schemaName,
k -> new HashMap<>())
.computeIfAbsent(tableName,
k -> new CachedMetaData<>(() -> {
String retentionPolicy = retentionPolicies.get().get(schemaName);
String measurement = measurements.get().get(tableName);
if (retentionPolicy == null || measurement == null) {
return Collections.emptyMap();
}
String query = new InfluxQL("SHOW FIELD KEYS FROM ")
.addIdentifier(retentionPolicy).append('.')
.addIdentifier(measurement)
.toString();
Map<String, InfluxColumn> fields = new HashMap<>();
JsonNode results = execute(query);
if (results != null) {
for (JsonNode series : results) {
if (series.has("values")) {
for (JsonNode row : series.get("values")) {
String name = row.get(0).textValue();
String influxType = row.get(1).textValue();
InfluxColumn collision = fields.put(name.toLowerCase(Locale.ENGLISH), new InfluxColumn(name, influxType, InfluxColumn.Kind.FIELD));
if (collision != null) {
InfluxError.IDENTIFIER_CASE_SENSITIVITY.fail("identifier " + name + " collides with " + collision.getInfluxName(), query);
}
}
}
}
}
return ImmutableMap.copyOf(fields);
}))
.get();
}

public boolean tableExistsInSchema(String schemaName, String tableName)
{
return !getFields(schemaName, tableName).isEmpty();
}

public List<InfluxColumn> getColumns(String schemaName, String tableName)
{
Collection<InfluxColumn> fields = getFields(schemaName, tableName).values();
if (fields.isEmpty()) {
return Collections.emptyList();
}
ImmutableList.Builder<InfluxColumn> columns = new ImmutableList.Builder<>();
columns.add(InfluxColumn.TIME);
columns.addAll(getTags(tableName).values());
columns.addAll(fields);
return columns.build();
}

private Map<String, String> showNames(String query)
{
Map<String, String> names = new HashMap<>();
JsonNode series = execute(query);
InfluxError.GENERAL.check(series != null && series.getNodeType().equals(JsonNodeType.ARRAY), "expecting an array, not " + series, query);
InfluxError.GENERAL.check(series != null && series.size() == 1, "expecting one element, not " + series, query);
series = series.get(0);
if (series.has("values")) {
for (JsonNode row : series.get("values")) {
String name = row.get(0).textValue();
String collision = names.put(name.toLowerCase(Locale.ENGLISH), name);
if (collision != null) {
InfluxError.IDENTIFIER_CASE_SENSITIVITY.fail("identifier " + name + " collides with " + collision, query);
}
}
}
return ImmutableMap.copyOf(names);
}

JsonNode execute(String query)
{
return http.query(query);
}

public HostAddress getHostAddress()
{
return HostAddress.fromParts(config.getHost(), config.getPort());
}

private class CachedMetaData<T>
{
private final Supplier<T> loader;
private T value;
private long lastLoaded;

private CachedMetaData(Supplier<T> loader)
{
this.loader = loader;
}

public synchronized T get()
{
if (System.currentTimeMillis() > lastLoaded + config.getCacheMetaDataMillis()) {
value = loader.get();
lastLoaded = System.currentTimeMillis();
}
return value;
}
}
}
Loading