From 3ceba4d8d36c5766601f352c612f14daa4534a1d Mon Sep 17 00:00:00 2001 From: Will Edwards Date: Wed, 18 Dec 2019 21:52:51 +0100 Subject: [PATCH 01/19] Does metadata; queries return a fake row, without any clause push-down --- pom.xml | 1 + presto-influx/pom.xml | 105 ++++++++++++++ .../prestosql/plugin/influx/InfluxClient.java | 135 ++++++++++++++++++ .../prestosql/plugin/influx/InfluxColumn.java | 56 ++++++++ .../plugin/influx/InfluxColumnHandle.java | 42 ++++++ .../prestosql/plugin/influx/InfluxConfig.java | 82 +++++++++++ .../plugin/influx/InfluxConnector.java | 61 ++++++++ .../plugin/influx/InfluxConnectorFactory.java | 43 ++++++ .../plugin/influx/InfluxErrorCode.java | 42 ++++++ .../plugin/influx/InfluxHandleResolver.java | 35 +++++ .../plugin/influx/InfluxMetadata.java | 115 +++++++++++++++ .../prestosql/plugin/influx/InfluxModule.java | 21 +++ .../prestosql/plugin/influx/InfluxPlugin.java | 14 ++ .../plugin/influx/InfluxRecordCursor.java | 76 ++++++++++ .../plugin/influx/InfluxRecordSet.java | 36 +++++ .../influx/InfluxRecordSetProvider.java | 38 +++++ .../prestosql/plugin/influx/InfluxSplit.java | 62 ++++++++ .../plugin/influx/InfluxSplitManager.java | 23 +++ .../plugin/influx/InfluxTableHandle.java | 15 ++ .../influx/InfluxTransactionHandle.java | 8 ++ 20 files changed, 1010 insertions(+) create mode 100644 presto-influx/pom.xml create mode 100644 presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java create mode 100644 presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumn.java create mode 100644 presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumnHandle.java create mode 100644 presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConfig.java create mode 100644 presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConnector.java create mode 100644 presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConnectorFactory.java create mode 100644 presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxErrorCode.java create mode 100644 presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxHandleResolver.java create mode 100644 presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java create mode 100644 presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxModule.java create mode 100644 presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxPlugin.java create mode 100644 presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordCursor.java create mode 100644 presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java create mode 100644 presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSetProvider.java create mode 100644 presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxSplit.java create mode 100644 presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxSplitManager.java create mode 100644 presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTableHandle.java create mode 100644 presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTransactionHandle.java diff --git a/pom.xml b/pom.xml index 115d35b4c308..fe346f26766c 100644 --- a/pom.xml +++ b/pom.xml @@ -142,6 +142,7 @@ presto-elasticsearch presto-iceberg presto-google-sheets + presto-influx diff --git a/presto-influx/pom.xml b/presto-influx/pom.xml new file mode 100644 index 000000000000..9e448236af57 --- /dev/null +++ b/presto-influx/pom.xml @@ -0,0 +1,105 @@ + + + 4.0.0 + + + presto-root + io.prestosql + 327-SNAPSHOT + + + presto-influx + Presto - Influx Connector + presto-plugin + + + + + org.influxdb + influxdb-java + 2.14 + + + + io.airlift + bootstrap + + + + io.airlift + configuration + + + + io.airlift + json + + + + com.google.guava + guava + + + + com.google.inject + guice + + + + javax.inject + javax.inject + + + + com.fasterxml.jackson.core + jackson-databind + + + + + io.prestosql + presto-spi + provided + + + + io.airlift + slice + provided + + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + + org.openjdk.jol + jol-core + provided + + + + + io.prestosql + presto-main + test + + + + org.testng + testng + test + + + + io.airlift + testing + test + + + + diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java new file mode 100644 index 000000000000..663e3db4a1d8 --- /dev/null +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java @@ -0,0 +1,135 @@ +package io.prestosql.plugin.influx; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.prestosql.spi.HostAddress; +import org.influxdb.InfluxDB; +import org.influxdb.InfluxDBFactory; +import org.influxdb.dto.Query; +import org.influxdb.dto.QueryResult; + +import javax.inject.Inject; +import java.util.*; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Supplier; + +import static java.util.Objects.requireNonNull; + +public class InfluxClient { + + private final InfluxConfig config; + private final InfluxDB influxDB; + // the various metadata are cached for a configurable number of milliseconds so we don't hammer the server + private final CachedMetaData> retentionPolicies; + private final CachedMetaData> measurements; + private final Map>> tagKeys; + private final Map>> fields; + + @Inject + public InfluxClient(InfluxConfig config) { + this.config = requireNonNull(config, "config is null"); + this.influxDB = InfluxDBFactory.connect("http://" + config.getHost() + ":" + config.getPort(), + 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 Map getRetentionPolicies() { + return retentionPolicies.get(); + } + + public Map getMeasurements() { + return measurements.get(); + } + + public Map getTags(String measurement) { + return tagKeys.computeIfAbsent(measurement, + k -> new CachedMetaData<>(() -> { + String query = "SHOW TAG KEYS FROM " + getMeasurements().get(measurement); + ImmutableMap.Builder tags = new ImmutableMap.Builder<>(); + for (Map.Entry name: showNames(query).entrySet()) { + tags.put(name.getKey(), new InfluxColumn(name.getValue(), "string", InfluxColumn.Kind.TAG)); + } + return tags.build(); + } + )) + .get(); + } + + public Map getFields(String measurement) { + return fields.computeIfAbsent(measurement, + k -> new CachedMetaData<>(() -> { + String query = "SHOW FIELD KEYS FROM " + getMeasurements().get(measurement); + Map fields = new HashMap<>(); + for (QueryResult.Series series : execute(query)) { + int nameIndex = series.getColumns().indexOf("fieldKey"); + int typeIndex = series.getColumns().indexOf("fieldType"); + for (List row : series.getValues()) { + String name = row.get(nameIndex).toString(); + String influxType = row.get(typeIndex).toString(); + InfluxColumn collision = fields.put(name.toLowerCase(), new InfluxColumn(name, influxType, InfluxColumn.Kind.FIELD)); + if (collision != null) { + InfluxErrorCode.IDENTIFIER_CASE_SENSITIVITY.fail("identifier " + name + " collides with " + collision.getInfluxName(), query); + } + } + } + return ImmutableMap.copyOf(fields); + })) + .get(); + } + + public List getColumns(String measurement) { + ImmutableList.Builder columns = new ImmutableList.Builder<>(); + columns.add(InfluxColumn.TIME); + columns.addAll(getTags(measurement).values()); + columns.addAll(getFields(measurement).values()); + return columns.build(); + } + + private Map showNames(String query) { + Map names = new HashMap<>(); + for (QueryResult.Series series: execute(query)) { + for (List row: series.getValues()) { + String name = row.get(0).toString(); + String collision = names.put(name.toLowerCase(), name); + if (collision != null) { + InfluxErrorCode.IDENTIFIER_CASE_SENSITIVITY.fail("identifier " + name + " collides with " + collision, query); + } + } + } + return ImmutableMap.copyOf(names); + } + + List execute(String query) { + QueryResult result = influxDB.query(new Query(query, config.getDatabase())); + InfluxErrorCode.GENERAL_ERROR.check(!result.hasError(), result.getError(), query); + InfluxErrorCode.GENERAL_ERROR.check(result.getResults().size() == 1, "expecting 1 series", query); + return result.getResults().get(0).getSeries(); + } + + public HostAddress getHostAddress() { + return HostAddress.fromParts(config.getHost(), config.getPort()); + } + + private class CachedMetaData { + + private final Supplier loader; + private T value; + private long lastLoaded; + + private CachedMetaData(Supplier loader) { + this.loader = loader; + } + + public synchronized T get() { + if (System.currentTimeMillis() > lastLoaded + config.getCacheMetaDataMillis()) { + value = loader.get(); + lastLoaded = System.currentTimeMillis(); + } + return value; + } + + } +} diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumn.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumn.java new file mode 100644 index 000000000000..9f56e57eb434 --- /dev/null +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumn.java @@ -0,0 +1,56 @@ +package io.prestosql.plugin.influx; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableMap; +import io.prestosql.spi.connector.ColumnHandle; +import io.prestosql.spi.connector.ColumnMetadata; +import io.prestosql.spi.type.*; + +public class InfluxColumn extends ColumnMetadata { + + // map InfluxDB types to Presto types + private final static ImmutableMap TYPES_MAPPING = new ImmutableMap.Builder() + .put("string", VarcharType.VARCHAR) + .put("boolean", BooleanType.BOOLEAN) + .put("integer", BigintType.BIGINT) + .put("float", DoubleType.DOUBLE) + .put("_time", TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE) + .build(); + public static final InfluxColumn TIME = new InfluxColumn("time", "_time", Kind.TIME); + + private final String influxName; + private final String influxType; + private final Kind kind; + + @JsonCreator + public InfluxColumn(@JsonProperty("influxName") String influxName, + @JsonProperty("influxType") String influxType, + @JsonProperty("kind") Kind kind) { + super(influxName.toLowerCase(), TYPES_MAPPING.get(influxType), null, kind.name().toLowerCase(), false); + this.influxName = influxName; + this.influxType = influxType; + this.kind = kind; + } + + @JsonProperty + public String getInfluxName() { + return influxName; + } + + @JsonProperty + public String getInfluxType() { + return influxType; + } + + @JsonProperty + public Kind getKind() { + return kind; + } + + public enum Kind { + TIME, + TAG, + FIELD, + } +} diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumnHandle.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumnHandle.java new file mode 100644 index 000000000000..720382604c60 --- /dev/null +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumnHandle.java @@ -0,0 +1,42 @@ +package io.prestosql.plugin.influx; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.prestosql.spi.connector.ColumnHandle; +import io.prestosql.spi.connector.ColumnMetadata; +import io.prestosql.spi.connector.SchemaTableName; + +public class InfluxColumnHandle extends InfluxColumn implements ColumnHandle { + + private final SchemaTableName schemaTableName; + + @JsonCreator + public InfluxColumnHandle(@JsonProperty("retentionPolicy") String retentionPolicy, + @JsonProperty("measurement") String measurement, + @JsonProperty("influxName") String influxName, + @JsonProperty("influxType") String influxType, + @JsonProperty("kind") Kind kind) { + super(influxName, influxType, kind); + this.schemaTableName = new SchemaTableName(retentionPolicy, measurement); + } + + public InfluxColumnHandle(String retentionPolicy, + String measurement, + InfluxColumn column) { + this(retentionPolicy, measurement, column.getInfluxName(), column.getInfluxType(), column.getKind()); + } + + public SchemaTableName getSchemaTableName() { + return schemaTableName; + } + + @JsonProperty + public String getRetentionPolicy() { + return schemaTableName.getSchemaName(); + } + + @JsonProperty + public String getMeasurement() { + return schemaTableName.getTableName(); + } +} diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConfig.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConfig.java new file mode 100644 index 000000000000..b8cbdf470507 --- /dev/null +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConfig.java @@ -0,0 +1,82 @@ +package io.prestosql.plugin.influx; + +import io.airlift.configuration.Config; + +import javax.validation.constraints.NotNull; +import java.net.URI; + +public class InfluxConfig { + + private long cacheMetaDataMillis = 10000; + private String host = "localhost"; + private int port = 8086; + private String database; + private String userName; + private String password; + + @NotNull + public long getCacheMetaDataMillis() { + return cacheMetaDataMillis; + } + + @Config("cache-meta-data-millis") + public InfluxConfig setCacheMetaDataMillis(long cacheMetaDataMillis) { + this.cacheMetaDataMillis = cacheMetaDataMillis; + return this; + } + + @NotNull + public String getHost() + { + return host; + } + + @Config("host") + public InfluxConfig setHost(String host) { + this.host = host; + return this; + } + + public int getPort() { + return port; + } + + @Config("port") + public InfluxConfig setPort(int port) { + this.port = port; + return this; + } + + @NotNull + public String getDatabase() { + return database; + } + + @Config("database") + public InfluxConfig setDatabase(String database) { + this.database = database; + return this; + } + + @NotNull + public String getUserName() { + return userName; + } + + @Config("user") + public InfluxConfig setUserName(String userName) { + this.userName = userName; + return this; + } + + @NotNull + public String getPassword() { + return password; + } + + @Config("password") + public InfluxConfig setPassword(String password) { + this.password = password; + return this; + } +} diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConnector.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConnector.java new file mode 100644 index 000000000000..9f4e57dcdc2b --- /dev/null +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConnector.java @@ -0,0 +1,61 @@ +package io.prestosql.plugin.influx; + +import io.airlift.bootstrap.LifeCycleManager; +import io.prestosql.spi.connector.*; +import io.prestosql.spi.transaction.IsolationLevel; + +import javax.inject.Inject; + +import static io.prestosql.plugin.influx.InfluxTransactionHandle.INSTANCE; +import static io.prestosql.spi.transaction.IsolationLevel.READ_COMMITTED; +import static io.prestosql.spi.transaction.IsolationLevel.checkConnectorSupports; +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 = lifeCycleManager; + this.metadata = metadata; + this.splitManager = splitManager; + this.recordSetProvider = recordSetProvider; + } + + @Override + public ConnectorTransactionHandle beginTransaction(IsolationLevel isolationLevel, boolean readOnly) { + checkConnectorSupports(READ_COMMITTED, isolationLevel); + return INSTANCE; + } + + @Override + public ConnectorMetadata getMetadata(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/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConnectorFactory.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConnectorFactory.java new file mode 100644 index 000000000000..a341ed13de66 --- /dev/null +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConnectorFactory.java @@ -0,0 +1,43 @@ +package io.prestosql.plugin.influx; + +import com.google.inject.Injector; +import io.airlift.bootstrap.Bootstrap; +import io.airlift.json.JsonModule; +import io.prestosql.spi.connector.Connector; +import io.prestosql.spi.connector.ConnectorContext; +import io.prestosql.spi.connector.ConnectorFactory; +import io.prestosql.spi.connector.ConnectorHandleResolver; + +import java.util.Map; + +import static java.util.Objects.requireNonNull; + +public class InfluxConnectorFactory implements ConnectorFactory { + + @Override + public String getName() { + return "influx"; + } + + @Override + public ConnectorHandleResolver getHandleResolver() { + return new InfluxHandleResolver(); + } + + @Override + public Connector create(String catalogName, Map config, ConnectorContext context) { + requireNonNull(config, "config is null"); + + Bootstrap app = new Bootstrap( + new JsonModule(), + new InfluxModule()); + + Injector injector = app + .strictConfig() + .doNotInitializeLogging() + .setRequiredConfigurationProperties(config) + .initialize(); + + return injector.getInstance(InfluxConnector.class); + } +} diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxErrorCode.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxErrorCode.java new file mode 100644 index 000000000000..83246e0d2d89 --- /dev/null +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxErrorCode.java @@ -0,0 +1,42 @@ +package io.prestosql.plugin.influx; + +import io.prestosql.spi.ErrorCode; +import io.prestosql.spi.ErrorCodeSupplier; +import io.prestosql.spi.ErrorType; +import io.prestosql.spi.PrestoException; + +public enum InfluxErrorCode implements ErrorCodeSupplier { + + GENERAL_ERROR(1, ErrorType.EXTERNAL), + IDENTIFIER_CASE_SENSITIVITY(2, ErrorType.EXTERNAL); + + private static final int ERROR_BASE = 0; // FIXME needs allocating + private ErrorCode errorCode; + + InfluxErrorCode(int code, ErrorType type) { + this.errorCode = new ErrorCode(code + ERROR_BASE, name(), ErrorType.EXTERNAL); + } + + public void check(boolean condition, String message, String context) { + if (!condition) { + fail(message, context); + } + } + + public void check(boolean condition, String message) { + check(condition, message, null); + } + + public void fail(String message, String context) { + throw new PrestoException(this, message + (context != null && !context.isEmpty()? " " + context: "")); + } + + public void fail(String message) { + fail(message, null); + } + + @Override + public ErrorCode toErrorCode() { + return errorCode; + } +} diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxHandleResolver.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxHandleResolver.java new file mode 100644 index 000000000000..91c40b093589 --- /dev/null +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxHandleResolver.java @@ -0,0 +1,35 @@ +package io.prestosql.plugin.influx; + +import io.prestosql.spi.connector.ColumnHandle; +import io.prestosql.spi.connector.ConnectorHandleResolver; +import io.prestosql.spi.connector.ConnectorSplit; +import io.prestosql.spi.connector.ConnectorTableHandle; +import io.prestosql.spi.connector.ConnectorTransactionHandle; + +public class InfluxHandleResolver + implements ConnectorHandleResolver +{ + @Override + public Class getTableHandleClass() + { + return InfluxTableHandle.class; + } + + @Override + public Class getColumnHandleClass() + { + return InfluxColumnHandle.class; + } + + @Override + public Class getSplitClass() + { + return InfluxSplit.class; + } + + @Override + public Class getTransactionHandleClass() + { + return InfluxTransactionHandle.class; + } +} diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java new file mode 100644 index 000000000000..afc2a2af3d83 --- /dev/null +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java @@ -0,0 +1,115 @@ +package io.prestosql.plugin.influx; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.prestosql.spi.connector.*; + +import javax.inject.Inject; +import java.util.*; + +import static java.util.Objects.requireNonNull; + +public class InfluxMetadata implements ConnectorMetadata { + + private InfluxClient client; + + @Inject + public InfluxMetadata(InfluxClient client) { + this.client = requireNonNull(client, "client is null"); + } + + @Override + public List listSchemaNames(ConnectorSession session) { + return ImmutableList.copyOf(client.getRetentionPolicies().keySet()); + } + + @Override + public List listTables(ConnectorSession session, Optional schemaName) { + Collection retentionPolicies; + if (schemaName.isPresent()) { + if (client.getRetentionPolicies().containsKey(schemaName.get())) { + retentionPolicies = Collections.singletonList(schemaName.get()); + } else { + return Collections.emptyList(); + } + } else { + retentionPolicies = client.getRetentionPolicies().keySet(); + } + // in Influx, all measurements can exist in all retention policies, + // (and all tickets asking for a way to know which measurements are actually + // used in which retention policy are closed as wont-fix) + ImmutableList.Builder builder = ImmutableList.builder(); + for (String measurementName: client.getMeasurements().keySet()) { + for (String retentionPolicy: retentionPolicies) { + builder.add(new SchemaTableName(retentionPolicy, measurementName)); + } + } + return builder.build(); + } + + @Override + public InfluxTableHandle getTableHandle(ConnectorSession session, SchemaTableName tableName) + { + if (client.getRetentionPolicies().containsKey(tableName.getSchemaName()) && + client.getMeasurements().containsKey(tableName.getTableName())) { + return new InfluxTableHandle(tableName.getSchemaName(), tableName.getTableName()); + } + return null; + } + + @Override + public Map> listTableColumns(ConnectorSession session, SchemaTablePrefix prefix) { + requireNonNull(prefix, "prefix is null"); + ImmutableMap.Builder> result = ImmutableMap.builder(); + Collection retentionPolicies = client.getRetentionPolicies().keySet(); + Collection measurements = client.getMeasurements().keySet(); + Map metadata = new HashMap<>(); + for (String retentionPolicy: retentionPolicies) { + for (String measurement: measurements) { + SchemaTableName schemaTableName = new SchemaTableName(retentionPolicy, measurement); + if (prefix.matches(schemaTableName)) { + result.put(schemaTableName, metadata.computeIfAbsent(measurement, + k -> getTableMetadata(session, new InfluxTableHandle(schemaTableName.getSchemaName(), schemaTableName.getTableName()))) + .getColumns()); + } + } + } + return result.build(); + } + + @Override + public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle table) { + InfluxTableHandle influxTable = (InfluxTableHandle)table; + ImmutableList.Builder columns = new ImmutableList.Builder<>(); + for (InfluxColumn column: client.getColumns(influxTable.getTableName())) { + columns.add(new InfluxColumnHandle(influxTable.getSchemaName(), influxTable.getTableName(), column)); + } + return new ConnectorTableMetadata(influxTable, columns.build()); + } + + @Override + public Map getColumnHandles(ConnectorSession session, ConnectorTableHandle tableHandle) { + ImmutableMap.Builder handles = new ImmutableMap.Builder<>(); + for (ColumnMetadata column: getTableMetadata(session, tableHandle).getColumns()) { + handles.put(column.getName(), (InfluxColumnHandle)column); + } + return handles.build(); + } + + @Override + public ColumnMetadata getColumnMetadata(ConnectorSession session, ConnectorTableHandle tableHandle, ColumnHandle columnHandle) + { + return (InfluxColumnHandle) columnHandle; + } + + @Override + public boolean usesLegacyTableLayouts() { + return false; + } + + @Override + public ConnectorTableProperties getTableProperties(ConnectorSession session, ConnectorTableHandle table) + { + return new ConnectorTableProperties(); + } +} diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxModule.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxModule.java new file mode 100644 index 000000000000..4f2fd8b52e29 --- /dev/null +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxModule.java @@ -0,0 +1,21 @@ +package io.prestosql.plugin.influx; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Scopes; + +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).in(Scopes.SINGLETON); + binder.bind(InfluxConnector.class).in(Scopes.SINGLETON); + binder.bind(InfluxMetadata.class).in(Scopes.SINGLETON); + binder.bind(InfluxSplitManager.class).in(Scopes.SINGLETON); + binder.bind(InfluxRecordSetProvider.class).in(Scopes.SINGLETON); + binder.bind(InfluxHandleResolver.class).in(Scopes.SINGLETON); + } +} diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxPlugin.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxPlugin.java new file mode 100644 index 000000000000..8d4b8321cef8 --- /dev/null +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxPlugin.java @@ -0,0 +1,14 @@ +package io.prestosql.plugin.influx; + +import com.google.common.collect.ImmutableList; +import io.prestosql.spi.Plugin; +import io.prestosql.spi.connector.ConnectorFactory; + +public class InfluxPlugin implements Plugin { + + @Override + public Iterable getConnectorFactories() + { + return ImmutableList.of(new InfluxConnectorFactory()); + } +} diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordCursor.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordCursor.java new file mode 100644 index 000000000000..c2f96d3f5c47 --- /dev/null +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordCursor.java @@ -0,0 +1,76 @@ +package io.prestosql.plugin.influx; + +import io.airlift.slice.Slice; +import io.prestosql.spi.connector.RecordCursor; +import io.prestosql.spi.type.Type; +import org.influxdb.dto.QueryResult; + +import java.util.List; + +public class InfluxRecordCursor implements RecordCursor { + + private final List results; + private final List columns; + private int rowId; + + public InfluxRecordCursor(List results, List columns) { + this.results = results; + this.columns = columns; + this.rowId = -1; + } + + @Override + public long getCompletedBytes() { + return 0; + } + + @Override + public long getReadTimeNanos() { + return 0; + } + + @Override + public Type getType(int field) { + return null; + } + + @Override + public boolean advanceNextPosition() { + rowId++; + return rowId < 1; + } + + @Override + public boolean getBoolean(int field) { + return false; + } + + @Override + public long getLong(int field) { + return 0; + } + + @Override + public double getDouble(int field) { + return 0; + } + + @Override + public Slice getSlice(int field) { + return null; + } + + @Override + public Object getObject(int field) { + return null; + } + + @Override + public boolean isNull(int field) { + return true; + } + + @Override + public void close() { + } +} diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java new file mode 100644 index 000000000000..79cbdc95f478 --- /dev/null +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java @@ -0,0 +1,36 @@ +package io.prestosql.plugin.influx; + +import com.google.common.collect.ImmutableList; +import io.prestosql.spi.connector.RecordCursor; +import io.prestosql.spi.connector.RecordSet; +import io.prestosql.spi.type.Type; +import org.influxdb.dto.QueryResult; + +import java.util.List; + +public class InfluxRecordSet implements RecordSet { + + private final List columns; + private final List columnTypes; + private final List results; + + public InfluxRecordSet(List columns, List results) { + this.columns = columns; + ImmutableList.Builder columnTypes = new ImmutableList.Builder<>(); + for (InfluxColumnHandle column: columns) { + columnTypes.add(column.getType()); + } + this.columnTypes = columnTypes.build(); + this.results = results; + } + + @Override + public List getColumnTypes() { + return columnTypes; + } + + @Override + public RecordCursor cursor() { + return new InfluxRecordCursor(results, columns); + } +} diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSetProvider.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSetProvider.java new file mode 100644 index 000000000000..36d37bc6126a --- /dev/null +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSetProvider.java @@ -0,0 +1,38 @@ +package io.prestosql.plugin.influx; + +import com.google.common.collect.ImmutableList; +import io.prestosql.spi.connector.*; +import org.influxdb.dto.QueryResult; + +import javax.inject.Inject; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +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) { + ImmutableList.Builder handles = ImmutableList.builder(); + List columnNames = new ArrayList<>(); + for (ColumnHandle handle : columns) { + InfluxColumnHandle influxColumnHandle = (InfluxColumnHandle) handle; + handles.add(influxColumnHandle); + columnNames.add(influxColumnHandle.getInfluxName()); + } + QueryResult.Series series = new QueryResult.Series(); + series.setColumns(columnNames); + return new InfluxRecordSet(handles.build(), Collections.singletonList(series)); + } +} diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxSplit.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxSplit.java new file mode 100644 index 000000000000..f93c47c961a7 --- /dev/null +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxSplit.java @@ -0,0 +1,62 @@ +package io.prestosql.plugin.influx; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import io.prestosql.spi.HostAddress; +import io.prestosql.spi.connector.ConnectorSplit; + +import java.util.List; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +public class InfluxSplit implements ConnectorSplit { + + private final HostAddress address; + + @JsonCreator + public InfluxSplit(@JsonProperty("host") String host, @JsonProperty("port") int port) { + this.address = HostAddress.fromParts(requireNonNull(host, "host is null"), port); + } + + public InfluxSplit(HostAddress address) { + this.address = requireNonNull(address, "address is null"); + } + + @JsonProperty + public String getHost() { + return address.getHostText(); + } + + @JsonProperty + public int getPort() { + return address.getPort(); + } + + + @Override + public boolean isRemotelyAccessible() { + return true; + } + + @Override + public List getAddresses() + { + return ImmutableList.of(address); + } + + @Override + public Object getInfo() + { + return this; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("host", address) + .toString(); + } +} diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxSplitManager.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxSplitManager.java new file mode 100644 index 000000000000..87431051091d --- /dev/null +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxSplitManager.java @@ -0,0 +1,23 @@ +package io.prestosql.plugin.influx; + +import io.prestosql.spi.connector.*; + +import javax.inject.Inject; +import java.util.Collections; + +public class InfluxSplitManager implements ConnectorSplitManager { + + private final InfluxClient client; + + @Inject + public InfluxSplitManager(InfluxClient client) + { + this.client = client; + } + + @Override + public ConnectorSplitSource getSplits(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorTableHandle table, SplitSchedulingStrategy splitSchedulingStrategy) + { + return new FixedSplitSource(Collections.singletonList(new InfluxSplit(client.getHostAddress()))); + } +} diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTableHandle.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTableHandle.java new file mode 100644 index 000000000000..e970ee66013e --- /dev/null +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTableHandle.java @@ -0,0 +1,15 @@ +package io.prestosql.plugin.influx; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.prestosql.spi.connector.ConnectorTableHandle; +import io.prestosql.spi.connector.SchemaTableName; + +public class InfluxTableHandle extends SchemaTableName implements ConnectorTableHandle { + + @JsonCreator + public InfluxTableHandle(@JsonProperty("schema") String schemaName, + @JsonProperty("table") String tableName) { + super(schemaName, tableName); + } +} diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTransactionHandle.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTransactionHandle.java new file mode 100644 index 000000000000..4f43562d877f --- /dev/null +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTransactionHandle.java @@ -0,0 +1,8 @@ +package io.prestosql.plugin.influx; + +import io.prestosql.spi.connector.ConnectorTransactionHandle; + +public enum InfluxTransactionHandle implements ConnectorTransactionHandle +{ + INSTANCE +} From 6644d71a80132cf7c87a576ec117dcdcfb734c0a Mon Sep 17 00:00:00 2001 From: Will Edwards Date: Fri, 20 Dec 2019 15:07:05 +0100 Subject: [PATCH 02/19] Basic implementation of Influx connector, using old influx lib --- .../prestosql/plugin/influx/InfluxClient.java | 25 +++-- .../prestosql/plugin/influx/InfluxColumn.java | 22 ++++- .../plugin/influx/InfluxColumnHandle.java | 24 +++-- ...{InfluxErrorCode.java => InfluxError.java} | 22 +++-- .../plugin/influx/InfluxMetadata.java | 89 ++++++++++++++++-- .../io/prestosql/plugin/influx/InfluxQL.java | 93 +++++++++++++++++++ .../plugin/influx/InfluxRecordCursor.java | 55 ++++++++--- .../plugin/influx/InfluxRecordSet.java | 39 ++++++-- .../influx/InfluxRecordSetProvider.java | 33 ++++--- .../plugin/influx/InfluxTableHandle.java | 49 +++++++++- 10 files changed, 388 insertions(+), 63 deletions(-) rename presto-influx/src/main/java/io/prestosql/plugin/influx/{InfluxErrorCode.java => InfluxError.java} (61%) create mode 100644 presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java index 663e3db4a1d8..4e338640a1c2 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java @@ -2,8 +2,10 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import io.airlift.log.Logger; import io.prestosql.spi.HostAddress; import org.influxdb.InfluxDB; +import org.influxdb.InfluxDBException; import org.influxdb.InfluxDBFactory; import org.influxdb.dto.Query; import org.influxdb.dto.QueryResult; @@ -17,6 +19,7 @@ public class InfluxClient { + final Logger logger; private final InfluxConfig config; private final InfluxDB influxDB; // the various metadata are cached for a configurable number of milliseconds so we don't hammer the server @@ -27,6 +30,7 @@ public class InfluxClient { @Inject public InfluxClient(InfluxConfig config) { + this.logger = Logger.get(getClass()); this.config = requireNonNull(config, "config is null"); this.influxDB = InfluxDBFactory.connect("http://" + config.getHost() + ":" + config.getPort(), config.getUserName(), config.getPassword()); @@ -71,7 +75,7 @@ public Map getFields(String measurement) { String influxType = row.get(typeIndex).toString(); InfluxColumn collision = fields.put(name.toLowerCase(), new InfluxColumn(name, influxType, InfluxColumn.Kind.FIELD)); if (collision != null) { - InfluxErrorCode.IDENTIFIER_CASE_SENSITIVITY.fail("identifier " + name + " collides with " + collision.getInfluxName(), query); + InfluxError.IDENTIFIER_CASE_SENSITIVITY.fail("identifier " + name + " collides with " + collision.getInfluxName(), query); } } } @@ -95,7 +99,7 @@ private Map showNames(String query) { String name = row.get(0).toString(); String collision = names.put(name.toLowerCase(), name); if (collision != null) { - InfluxErrorCode.IDENTIFIER_CASE_SENSITIVITY.fail("identifier " + name + " collides with " + collision, query); + InfluxError.IDENTIFIER_CASE_SENSITIVITY.fail("identifier " + name + " collides with " + collision, query); } } } @@ -103,10 +107,19 @@ private Map showNames(String query) { } List execute(String query) { - QueryResult result = influxDB.query(new Query(query, config.getDatabase())); - InfluxErrorCode.GENERAL_ERROR.check(!result.hasError(), result.getError(), query); - InfluxErrorCode.GENERAL_ERROR.check(result.getResults().size() == 1, "expecting 1 series", query); - return result.getResults().get(0).getSeries(); + logger.debug("executing: " + query); + QueryResult result; + try { + result = influxDB.query(new Query(query, config.getDatabase())); + } catch (InfluxDBException e) { + InfluxError.GENERAL.fail(e.toString(), query); + return Collections.emptyList(); + } + InfluxError.GENERAL.check(!result.hasError(), result.getError(), query); + InfluxError.GENERAL.check(result.getResults().size() == 1, "expecting 1 series", query); + InfluxError.GENERAL.check(!result.getResults().get(0).hasError(), result.getResults().get(0).getError(), query); + List series = result.getResults().get(0).getSeries(); + return series != null? series: Collections.emptyList(); } public HostAddress getHostAddress() { diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumn.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumn.java index 9f56e57eb434..2cf4337cefe1 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumn.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumn.java @@ -2,8 +2,8 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.MoreObjects; import com.google.common.collect.ImmutableMap; -import io.prestosql.spi.connector.ColumnHandle; import io.prestosql.spi.connector.ColumnMetadata; import io.prestosql.spi.type.*; @@ -15,9 +15,9 @@ public class InfluxColumn extends ColumnMetadata { .put("boolean", BooleanType.BOOLEAN) .put("integer", BigintType.BIGINT) .put("float", DoubleType.DOUBLE) - .put("_time", TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE) + .put("time", TimestampType.TIMESTAMP) .build(); - public static final InfluxColumn TIME = new InfluxColumn("time", "_time", Kind.TIME); + public static final InfluxColumn TIME = new InfluxColumn("time", "time", Kind.TIME); private final String influxName; private final String influxType; @@ -48,6 +48,22 @@ public Kind getKind() { return kind; } + protected MoreObjects.ToStringHelper toStringHelper(Object self) { + MoreObjects.ToStringHelper helper = com.google.common.base.MoreObjects.toStringHelper(self) + .addValue(getName()) + .addValue(getType()) + .addValue(kind); + if (!getName().equals(getInfluxName())) { + helper.add("influx-name", getInfluxName()); + } + return helper; + } + + @Override + public String toString() { + return toStringHelper(this).toString(); + } + public enum Kind { TIME, TAG, diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumnHandle.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumnHandle.java index 720382604c60..6fac484db5d9 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumnHandle.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumnHandle.java @@ -3,12 +3,13 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import io.prestosql.spi.connector.ColumnHandle; -import io.prestosql.spi.connector.ColumnMetadata; import io.prestosql.spi.connector.SchemaTableName; + public class InfluxColumnHandle extends InfluxColumn implements ColumnHandle { - private final SchemaTableName schemaTableName; + private final String retentionPolicy; + private final String measurement; @JsonCreator public InfluxColumnHandle(@JsonProperty("retentionPolicy") String retentionPolicy, @@ -17,7 +18,8 @@ public InfluxColumnHandle(@JsonProperty("retentionPolicy") String retentionPolic @JsonProperty("influxType") String influxType, @JsonProperty("kind") Kind kind) { super(influxName, influxType, kind); - this.schemaTableName = new SchemaTableName(retentionPolicy, measurement); + this.retentionPolicy = retentionPolicy; + this.measurement = measurement; } public InfluxColumnHandle(String retentionPolicy, @@ -26,17 +28,21 @@ public InfluxColumnHandle(String retentionPolicy, this(retentionPolicy, measurement, column.getInfluxName(), column.getInfluxType(), column.getKind()); } - public SchemaTableName getSchemaTableName() { - return schemaTableName; - } - @JsonProperty public String getRetentionPolicy() { - return schemaTableName.getSchemaName(); + return retentionPolicy; } @JsonProperty public String getMeasurement() { - return schemaTableName.getTableName(); + return measurement; + } + + @Override + public String toString() { + return toStringHelper(this) + .addValue(getRetentionPolicy()) + .addValue(getMeasurement()) + .toString(); } } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxErrorCode.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxError.java similarity index 61% rename from presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxErrorCode.java rename to presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxError.java index 83246e0d2d89..a04d636a4cde 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxErrorCode.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxError.java @@ -5,16 +5,19 @@ import io.prestosql.spi.ErrorType; import io.prestosql.spi.PrestoException; -public enum InfluxErrorCode implements ErrorCodeSupplier { +import static com.google.common.base.MoreObjects.toStringHelper; - GENERAL_ERROR(1, ErrorType.EXTERNAL), - IDENTIFIER_CASE_SENSITIVITY(2, ErrorType.EXTERNAL); +public enum InfluxError implements ErrorCodeSupplier { + + GENERAL (ErrorType.INTERNAL_ERROR), + IDENTIFIER_CASE_SENSITIVITY (ErrorType.EXTERNAL), + BAD_VALUE (ErrorType.USER_ERROR); private static final int ERROR_BASE = 0; // FIXME needs allocating - private ErrorCode errorCode; + private final ErrorCode errorCode; - InfluxErrorCode(int code, ErrorType type) { - this.errorCode = new ErrorCode(code + ERROR_BASE, name(), ErrorType.EXTERNAL); + InfluxError(ErrorType type) { + this.errorCode = new ErrorCode(ERROR_BASE + ordinal(), name(), type); } public void check(boolean condition, String message, String context) { @@ -39,4 +42,11 @@ public void fail(String message) { public ErrorCode toErrorCode() { return errorCode; } + + @Override + public String toString() { + return toStringHelper(this) + .add("code", errorCode) + .toString(); + } } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java index afc2a2af3d83..9aaf81e957de 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java @@ -3,6 +3,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.prestosql.spi.connector.*; +import io.prestosql.spi.predicate.*; import javax.inject.Inject; import java.util.*; @@ -11,7 +12,7 @@ public class InfluxMetadata implements ConnectorMetadata { - private InfluxClient client; + private final InfluxClient client; @Inject public InfluxMetadata(InfluxClient client) { @@ -50,9 +51,10 @@ public List listTables(ConnectorSession session, Optional> listTableColumns(ConnectorSess SchemaTableName schemaTableName = new SchemaTableName(retentionPolicy, measurement); if (prefix.matches(schemaTableName)) { result.put(schemaTableName, metadata.computeIfAbsent(measurement, - k -> getTableMetadata(session, new InfluxTableHandle(schemaTableName.getSchemaName(), schemaTableName.getTableName()))) + k -> getTableMetadata(session, new InfluxTableHandle(retentionPolicy, measurement))) .getColumns()); } } @@ -82,7 +84,7 @@ public ConnectorTableMetadata getTableMetadata(ConnectorSession session, Connect InfluxTableHandle influxTable = (InfluxTableHandle)table; ImmutableList.Builder columns = new ImmutableList.Builder<>(); for (InfluxColumn column: client.getColumns(influxTable.getTableName())) { - columns.add(new InfluxColumnHandle(influxTable.getSchemaName(), influxTable.getTableName(), column)); + columns.add(new InfluxColumnHandle(influxTable.getRetentionPolicy(), influxTable.getMeasurement(), column)); } return new ConnectorTableMetadata(influxTable, columns.build()); } @@ -112,4 +114,79 @@ public ConnectorTableProperties getTableProperties(ConnectorSession session, Con { return new ConnectorTableProperties(); } + + @Override + public Optional> applyFilter(ConnectorSession session, ConnectorTableHandle handle, Constraint constraint) { + boolean all = true; + InfluxQL where = new InfluxQL(); + for (Map.Entry predicate: constraint.getSummary().getDomains().orElse(Collections.emptyMap()).entrySet()) { + InfluxColumnHandle column = (InfluxColumnHandle) predicate.getKey(); + ValueSet values = predicate.getValue().getValues(); + if (values instanceof SortedRangeSet) { + boolean first = true; + for (Range range : values.getRanges().getOrderedRanges()) { + where.append(first ? where.isEmpty() ? "WHERE ((" : " AND ((" : ") OR ("); + if (range.isSingleValue()) { + where.add(column).append(" = ").add(range.getSingleValue()); + } else { + final String low; + switch (range.getLow().getBound()) { + case EXACTLY: + low = " >= "; + break; + case ABOVE: + low = " > "; + break; + default: + InfluxError.GENERAL.fail("bad low bound", range.toString(session)); + continue; + } + final String high; + switch (range.getHigh().getBound()) { + case EXACTLY: + high = " <= "; + break; + case BELOW: + high = " < "; + break; + default: + InfluxError.GENERAL.fail("bad high bound", range.toString(session)); + continue; + } + where.add(column).append(low).add(range.getLow().getValue()).append(" AND ") + .add(column).append(high).add(range.getHigh().getValue()); + } + first = false; + } + if (first) { + client.logger.warn("unhandled SortedRangeSet " + column + ":" + values.getClass().getName() + "=" + values.toString(session)); + all = false; + } else { + where.append("))"); + } + } else if (values instanceof EquatableValueSet) { + boolean first = true; + for (Object value: values.getDiscreteValues().getValues()) { + where.append(first? where.isEmpty()? "WHERE (": " AND (": " OR ") + .add(column).append(" = ").add(value); + first = false; + } + if (first) { + client.logger.warn("unhandled EquatableValueSet " + column + ":" + values.getClass().getName() + "=" + values.toString(session)); + all = false; + } else { + where.append(')'); + } + } else { + client.logger.warn("unhandled predicate " + column + ":" + values.getClass().getName() + "=" + values.toString(session)); + all = false; + } + } + client.logger.debug("applyFilter(" + handle + ", " + constraint.getSummary().toString(session) + ") = " + all + ", " + where); + InfluxTableHandle table = (InfluxTableHandle) handle; + return Optional.of(new ConstraintApplicationResult<>(new InfluxTableHandle( + table.getRetentionPolicy(), + table.getMeasurement(), + where), all? TupleDomain.all(): constraint.getSummary())); + } } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java new file mode 100644 index 000000000000..85e131e7dc57 --- /dev/null +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java @@ -0,0 +1,93 @@ +package io.prestosql.plugin.influx; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.airlift.slice.Slice; + +public class InfluxQL { + + private final StringBuilder influxQL; + + public InfluxQL() { + influxQL = new StringBuilder(); + } + + @JsonCreator + public InfluxQL(@JsonProperty("q") String prefix) { + influxQL = new StringBuilder(prefix); + } + + public InfluxQL append(InfluxQL fragment) { + influxQL.append(fragment); + return this; + } + + public InfluxQL append(String s) { + influxQL.append(s); + return this; + } + + public InfluxQL append(char ch) { + influxQL.append(ch); + return this; + } + + public InfluxQL add(InfluxColumn column) { + addIdentifier(column.getInfluxName()); + return this; + } + + public InfluxQL add(Object value) { + assert !(value instanceof InfluxColumn): value; + if (value == null) { + influxQL.append("null"); + } else if (value instanceof Slice) { + quote(((Slice) value).toStringUtf8(), '\''); + } else if (value instanceof Number || value instanceof Boolean) { + influxQL.append(value); + } else { + quote(value.toString(), '\''); + } + return this; + } + + public InfluxQL addIdentifier(String identifier) { + boolean safe = true; + for (int i = 0; i < identifier.length() && safe; i++) { + char ch = identifier.charAt(i); + safe = (ch >= 'A' && ch <= 'Z') || (ch >= 'a' && ch <= 'z') || (i > 0 && ch >= '0' && ch <= '9') || ch == '_'; + } + if (safe) { + influxQL.append(identifier); + } else { + quote(identifier, '"'); + } + return this; + } + + public void quote(String value, char delimiter) { + append(delimiter); + for (int i = 0; i < value.length(); i++) { + char ch = value.charAt(i); + if (ch < ' ' || ch > 127) { + InfluxError.BAD_VALUE.fail("illegal value", value); + } + if (ch == delimiter || ch == '\\') { + append('\\'); + } + append(ch); + } + append(delimiter); + } + + public boolean isEmpty() { + return influxQL.length() == 0; + } + + @JsonProperty("q") + @Override + public String toString() { + return influxQL.toString(); + } + +} diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordCursor.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordCursor.java index c2f96d3f5c47..d8cb3a8cb031 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordCursor.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordCursor.java @@ -1,21 +1,25 @@ package io.prestosql.plugin.influx; import io.airlift.slice.Slice; +import io.airlift.slice.Slices; import io.prestosql.spi.connector.RecordCursor; import io.prestosql.spi.type.Type; -import org.influxdb.dto.QueryResult; +import java.time.Instant; import java.util.List; +import java.util.Objects; + +import static java.time.format.DateTimeFormatter.ISO_DATE_TIME; public class InfluxRecordCursor implements RecordCursor { - private final List results; - private final List columns; + private final List columns; + private final List> rows; private int rowId; - public InfluxRecordCursor(List results, List columns) { - this.results = results; + public InfluxRecordCursor(List columns, List> rows) { this.columns = columns; + this.rows = rows; this.rowId = -1; } @@ -31,46 +35,75 @@ public long getReadTimeNanos() { @Override public Type getType(int field) { - return null; + return columns.get(field).getType(); } @Override public boolean advanceNextPosition() { - rowId++; - return rowId < 1; + return ++rowId < rows.size(); } @Override public boolean getBoolean(int field) { + Object value = getObject(field); + if (value instanceof Boolean) { + return (Boolean) value; + } + if (value instanceof Number) { + return ((Number) value).intValue() != 0; + } + if (value != null) { + InfluxError.BAD_VALUE.fail("cannot cast " + columns.get(field) + ": " + value.getClass() + ": " + value + " to boolean"); + } return false; } @Override public long getLong(int field) { + Object value = getObject(field); + if (value instanceof Number) { + return ((Number) value).longValue(); + } + if (value != null) { + InfluxError.BAD_VALUE.fail("cannot cast " + columns.get(field) + ": " + value.getClass() + ": " + value + " to long"); + } return 0; } @Override public double getDouble(int field) { + Object value = getObject(field); + if (value instanceof Number) { + return ((Number) value).doubleValue(); + } + if (value != null) { + InfluxError.BAD_VALUE.fail("cannot cast " + columns.get(field) + ": " + value.getClass() + ": " + value + " to double"); + } return 0; } @Override public Slice getSlice(int field) { - return null; + String value = Objects.toString(getObject(field), null); + return value != null? Slices.utf8Slice(value): null; } @Override public Object getObject(int field) { - return null; + Object value = rows.get(rowId).get(field); + if (columns.get(field).getKind() == InfluxColumn.Kind.TIME && value instanceof String) { + return Instant.parse((String) value).toEpochMilli(); + } + return value; } @Override public boolean isNull(int field) { - return true; + return rows.get(rowId).get(field) == null; } @Override public void close() { + rowId = rows.size(); } } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java index 79cbdc95f478..2d8d5c80af77 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java @@ -6,22 +6,47 @@ import io.prestosql.spi.type.Type; import org.influxdb.dto.QueryResult; -import java.util.List; +import java.time.ZoneId; +import java.util.*; public class InfluxRecordSet implements RecordSet { - private final List columns; + private final List columns; private final List columnTypes; - private final List results; + private final List> rows; - public InfluxRecordSet(List columns, List results) { + public InfluxRecordSet(List columns, List results) { this.columns = columns; ImmutableList.Builder columnTypes = new ImmutableList.Builder<>(); - for (InfluxColumnHandle column: columns) { + Map mapping = new HashMap<>(); + for (InfluxColumn column: columns) { columnTypes.add(column.getType()); + mapping.put(column.getInfluxName(), mapping.size()); } this.columnTypes = columnTypes.build(); - this.results = results; + this.rows = new ArrayList<>(); + Object[] row = new Object[columns.size()]; + for (QueryResult.Series series: results) { + if (series.getValues().isEmpty()) { + continue; + } + Arrays.fill(row, null); + if (series.getTags() != null) { + for (Map.Entry tag : series.getTags().entrySet()) { + row[mapping.get(tag.getKey())] = tag.getValue(); + } + } + int[] fields = new int[series.getColumns().size()]; + for (int i = 0; i < fields.length; i++) { + fields[i] = mapping.get(series.getColumns().get(i)); + } + for (List values: series.getValues()) { + for (int i = 0; i < fields.length; i++) { + row[fields[i]] = values.get(i); + } + rows.add(ImmutableList.copyOf(row)); + } + } } @Override @@ -31,6 +56,6 @@ public List getColumnTypes() { @Override public RecordCursor cursor() { - return new InfluxRecordCursor(results, columns); + return new InfluxRecordCursor(columns, rows); } } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSetProvider.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSetProvider.java index 36d37bc6126a..bd3267707428 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSetProvider.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSetProvider.java @@ -5,11 +5,10 @@ import org.influxdb.dto.QueryResult; import javax.inject.Inject; +import java.time.ZoneId; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.List; -import java.util.stream.Collectors; import static java.util.Objects.requireNonNull; @@ -23,16 +22,26 @@ public InfluxRecordSetProvider(InfluxClient client) { } @Override - public RecordSet getRecordSet(ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorSplit split, ConnectorTableHandle table, List columns) { - ImmutableList.Builder handles = ImmutableList.builder(); - List columnNames = new ArrayList<>(); - for (ColumnHandle handle : columns) { - InfluxColumnHandle influxColumnHandle = (InfluxColumnHandle) handle; - handles.add(influxColumnHandle); - columnNames.add(influxColumnHandle.getInfluxName()); + public RecordSet getRecordSet(ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorSplit split, ConnectorTableHandle tableHandle, List columns) { + InfluxTableHandle table = (InfluxTableHandle) tableHandle; + client.logger.info("getRecordSet(" + split + ", " + table + ", " + columns + ")"); + InfluxQL query = new InfluxQL(); + ImmutableList.Builder handles = ImmutableList.builder(); + boolean hasTime = false; + for (ColumnHandle handle: columns) { + InfluxColumnHandle column = (InfluxColumnHandle) handle; + hasTime |= column.getInfluxName().equals("time"); + InfluxError.GENERAL.check(column.getMeasurement().equals(table.getMeasurement()), "bad measurement for " + column + " in " + table); + InfluxError.GENERAL.check(column.getRetentionPolicy().equals(table.getRetentionPolicy()), "bad retention-policy for " + column + " in " + table); + query.append(query.isEmpty()? "SELECT ": ", ").add(column); + handles.add(column); } - QueryResult.Series series = new QueryResult.Series(); - series.setColumns(columnNames); - return new InfluxRecordSet(handles.build(), Collections.singletonList(series)); + if (!hasTime) { + query.append(query.isEmpty()? "SELECT time": ", time"); + handles.add(InfluxColumn.TIME); + } + query.append(' ').append(table.getFromWhere()); + List results = client.execute(query.toString()); // actually run the query against our Influx server + return new InfluxRecordSet(handles.build(), results); } } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTableHandle.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTableHandle.java index e970ee66013e..3bfb2da01759 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTableHandle.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTableHandle.java @@ -5,11 +5,54 @@ import io.prestosql.spi.connector.ConnectorTableHandle; import io.prestosql.spi.connector.SchemaTableName; +import static java.util.Objects.requireNonNull; + public class InfluxTableHandle extends SchemaTableName implements ConnectorTableHandle { + private final String retentionPolicy; + private final String measurement; + private final InfluxQL where; + @JsonCreator - public InfluxTableHandle(@JsonProperty("schema") String schemaName, - @JsonProperty("table") String tableName) { - super(schemaName, tableName); + public InfluxTableHandle(@JsonProperty("retentionPolicy") String retentionPolicy, + @JsonProperty("measurement") String measurement, + @JsonProperty("where") InfluxQL where) { + super(retentionPolicy, measurement); + this.retentionPolicy = requireNonNull(retentionPolicy, "retentionPolicy is null"); + this.measurement = requireNonNull(measurement, "measurement is null"); + this.where = requireNonNull(where, "where is null"); + } + + public InfluxTableHandle(String retentionPolicy, String measurement) { + this(retentionPolicy, measurement, new InfluxQL()); + } + + @JsonProperty + public String getRetentionPolicy() { + return retentionPolicy; + } + + @JsonProperty + public String getMeasurement() { + return measurement; + } + + @JsonProperty + public InfluxQL getWhere() { + return where; + } + + public InfluxQL getFromWhere() { + InfluxQL from = new InfluxQL("FROM ") + .addIdentifier(getRetentionPolicy()).append('.').addIdentifier(getMeasurement()); + if (!getWhere().isEmpty()) { + from.append(' ').append(getWhere().toString()); + } + return from; + } + + @Override + public String toString() { + return getFromWhere().toString(); } } From bb80718e68affc3fe6fa17934dec81bba9ecd635 Mon Sep 17 00:00:00 2001 From: Will Edwards Date: Fri, 20 Dec 2019 16:58:29 +0100 Subject: [PATCH 03/19] Sort out what measurements are populated in which retention policies --- .../prestosql/plugin/influx/InfluxClient.java | 88 ++++++++++----- .../plugin/influx/InfluxMetadata.java | 105 ++++++++++-------- .../plugin/influx/InfluxRecordCursor.java | 14 ++- .../plugin/influx/InfluxRecordSet.java | 22 ++-- .../influx/InfluxRecordSetProvider.java | 10 +- 5 files changed, 140 insertions(+), 99 deletions(-) diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java index 4e338640a1c2..433b45a64024 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java @@ -26,7 +26,7 @@ public class InfluxClient { private final CachedMetaData> retentionPolicies; private final CachedMetaData> measurements; private final Map>> tagKeys; - private final Map>> fields; + private final Map>>> fields; @Inject public InfluxClient(InfluxConfig config) { @@ -40,18 +40,32 @@ public InfluxClient(InfluxConfig config) { this.fields = new ConcurrentHashMap<>(); } - public Map getRetentionPolicies() { - return retentionPolicies.get(); + public Collection getSchemaNames() { + return retentionPolicies.get().keySet(); } - public Map getMeasurements() { - return measurements.get(); + public String getRetentionPolicy(String schemaName) { + return retentionPolicies.get().get(schemaName); } - public Map getTags(String measurement) { - return tagKeys.computeIfAbsent(measurement, + public Collection getTableNames() { + return measurements.get().keySet(); + } + + public String getMeasurement(String tableName) { + return measurements.get().get(tableName); + } + + private Map getTags(String tableName) { + return tagKeys.computeIfAbsent(tableName, k -> new CachedMetaData<>(() -> { - String query = "SHOW TAG KEYS FROM " + getMeasurements().get(measurement); + 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 tags = new ImmutableMap.Builder<>(); for (Map.Entry name: showNames(query).entrySet()) { tags.put(name.getKey(), new InfluxColumn(name.getValue(), "string", InfluxColumn.Kind.TAG)); @@ -62,33 +76,51 @@ public Map getTags(String measurement) { .get(); } - public Map getFields(String measurement) { - return fields.computeIfAbsent(measurement, - k -> new CachedMetaData<>(() -> { - String query = "SHOW FIELD KEYS FROM " + getMeasurements().get(measurement); - Map fields = new HashMap<>(); - for (QueryResult.Series series : execute(query)) { - int nameIndex = series.getColumns().indexOf("fieldKey"); - int typeIndex = series.getColumns().indexOf("fieldType"); - for (List row : series.getValues()) { - String name = row.get(nameIndex).toString(); - String influxType = row.get(typeIndex).toString(); - InfluxColumn collision = fields.put(name.toLowerCase(), new InfluxColumn(name, influxType, InfluxColumn.Kind.FIELD)); - if (collision != null) { - InfluxError.IDENTIFIER_CASE_SENSITIVITY.fail("identifier " + name + " collides with " + collision.getInfluxName(), query); + private Map 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 fields = new HashMap<>(); + for (QueryResult.Series series : execute(query)) { + int nameIndex = series.getColumns().indexOf("fieldKey"); + int typeIndex = series.getColumns().indexOf("fieldType"); + for (List row : series.getValues()) { + String name = row.get(nameIndex).toString(); + String influxType = row.get(typeIndex).toString(); + InfluxColumn collision = fields.put(name.toLowerCase(), 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); - })) + return ImmutableMap.copyOf(fields); + })) .get(); } - public List getColumns(String measurement) { + public boolean tableExistsInSchema(String schemaName, String tableName) { + return !getFields(schemaName, tableName).isEmpty(); + } + + public List getColumns(String schemaName, String tableName) { + Collection fields = getFields(schemaName, tableName).values(); + if (fields.isEmpty()) { + return Collections.emptyList(); + } ImmutableList.Builder columns = new ImmutableList.Builder<>(); columns.add(InfluxColumn.TIME); - columns.addAll(getTags(measurement).values()); - columns.addAll(getFields(measurement).values()); + columns.addAll(getTags(tableName).values()); + columns.addAll(fields); return columns.build(); } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java index 9aaf81e957de..5461df36bb23 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java @@ -21,28 +21,30 @@ public InfluxMetadata(InfluxClient client) { @Override public List listSchemaNames(ConnectorSession session) { - return ImmutableList.copyOf(client.getRetentionPolicies().keySet()); + return ImmutableList.copyOf(client.getSchemaNames()); } @Override public List listTables(ConnectorSession session, Optional schemaName) { - Collection retentionPolicies; + Collection schemaNames; if (schemaName.isPresent()) { - if (client.getRetentionPolicies().containsKey(schemaName.get())) { - retentionPolicies = Collections.singletonList(schemaName.get()); + if (client.getSchemaNames().contains(schemaName.get())) { + schemaNames = Collections.singletonList(schemaName.get()); } else { return Collections.emptyList(); } } else { - retentionPolicies = client.getRetentionPolicies().keySet(); + schemaNames = client.getSchemaNames(); } // in Influx, all measurements can exist in all retention policies, // (and all tickets asking for a way to know which measurements are actually // used in which retention policy are closed as wont-fix) ImmutableList.Builder builder = ImmutableList.builder(); - for (String measurementName: client.getMeasurements().keySet()) { - for (String retentionPolicy: retentionPolicies) { - builder.add(new SchemaTableName(retentionPolicy, measurementName)); + for (String tableName: client.getTableNames()) { + for (String matchingSchemaName: schemaNames) { + if (client.tableExistsInSchema(matchingSchemaName, tableName)) { + builder.add(new SchemaTableName(matchingSchemaName, tableName)); + } } } return builder.build(); @@ -51,8 +53,8 @@ public List listTables(ConnectorSession session, Optional> listTableColumns(ConnectorSession session, SchemaTablePrefix prefix) { requireNonNull(prefix, "prefix is null"); ImmutableMap.Builder> result = ImmutableMap.builder(); - Collection retentionPolicies = client.getRetentionPolicies().keySet(); - Collection measurements = client.getMeasurements().keySet(); - Map metadata = new HashMap<>(); - for (String retentionPolicy: retentionPolicies) { - for (String measurement: measurements) { - SchemaTableName schemaTableName = new SchemaTableName(retentionPolicy, measurement); + Collection schemaNames = client.getSchemaNames(); + Collection tableNames = client.getTableNames(); + for (String schemaName: schemaNames) { + for (String tableName: tableNames) { + SchemaTableName schemaTableName = new SchemaTableName(schemaName, tableName); if (prefix.matches(schemaTableName)) { - result.put(schemaTableName, metadata.computeIfAbsent(measurement, - k -> getTableMetadata(session, new InfluxTableHandle(retentionPolicy, measurement))) - .getColumns()); + List columns = client.getColumns(schemaName, tableName); + if (!columns.isEmpty()) { + result.put(schemaTableName, ImmutableList.copyOf(columns)); + } } } } @@ -83,7 +85,7 @@ public Map> listTableColumns(ConnectorSess public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle table) { InfluxTableHandle influxTable = (InfluxTableHandle)table; ImmutableList.Builder columns = new ImmutableList.Builder<>(); - for (InfluxColumn column: client.getColumns(influxTable.getTableName())) { + for (InfluxColumn column: client.getColumns(influxTable.getSchemaName(), influxTable.getTableName())) { columns.add(new InfluxColumnHandle(influxTable.getRetentionPolicy(), influxTable.getMeasurement(), column)); } return new ConnectorTableMetadata(influxTable, columns.build()); @@ -125,36 +127,51 @@ public Optional> applyFilter(C if (values instanceof SortedRangeSet) { boolean first = true; for (Range range : values.getRanges().getOrderedRanges()) { + if (!range.isSingleValue() && !range.getLow().getValueBlock().isPresent() && !range.getHigh().getValueBlock().isPresent()) { + // can't do an IS NULL + client.logger.debug("can't tackle range " + column + ": " + range.toString(session)); + all = false; + continue; + } where.append(first ? where.isEmpty() ? "WHERE ((" : " AND ((" : ") OR ("); if (range.isSingleValue()) { where.add(column).append(" = ").add(range.getSingleValue()); } else { - final String low; - switch (range.getLow().getBound()) { - case EXACTLY: - low = " >= "; - break; - case ABOVE: - low = " > "; - break; - default: - InfluxError.GENERAL.fail("bad low bound", range.toString(session)); - continue; + boolean hasLow = false; + if (range.getLow().getValueBlock().isPresent()) { + final String low; + switch (range.getLow().getBound()) { + case EXACTLY: + low = " >= "; + break; + case ABOVE: + low = " > "; + break; + default: + InfluxError.GENERAL.fail("bad low bound", range.toString(session)); + continue; + } + where.add(column).append(low).add(range.getLow().getValue()); + hasLow = true; } - final String high; - switch (range.getHigh().getBound()) { - case EXACTLY: - high = " <= "; - break; - case BELOW: - high = " < "; - break; - default: - InfluxError.GENERAL.fail("bad high bound", range.toString(session)); - continue; + if (range.getHigh().getValueBlock().isPresent()) { + final String high; + switch (range.getHigh().getBound()) { + case EXACTLY: + high = " <= "; + break; + case BELOW: + high = " < "; + break; + default: + InfluxError.GENERAL.fail("bad high bound", range.toString(session)); + continue; + } + if (hasLow) { + where.append(" AND "); + } + where.add(column).append(high).add(range.getHigh().getValue()); } - where.add(column).append(low).add(range.getLow().getValue()).append(" AND ") - .add(column).append(high).add(range.getHigh().getValue()); } first = false; } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordCursor.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordCursor.java index d8cb3a8cb031..96ddead08941 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordCursor.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordCursor.java @@ -9,15 +9,15 @@ import java.util.List; import java.util.Objects; -import static java.time.format.DateTimeFormatter.ISO_DATE_TIME; public class InfluxRecordCursor implements RecordCursor { private final List columns; - private final List> rows; + private final List rows; + private Object[] row; private int rowId; - public InfluxRecordCursor(List columns, List> rows) { + public InfluxRecordCursor(List columns, List rows) { this.columns = columns; this.rows = rows; this.rowId = -1; @@ -40,7 +40,8 @@ public Type getType(int field) { @Override public boolean advanceNextPosition() { - return ++rowId < rows.size(); + row = ++rowId < rows.size()? rows.get(rowId): null; + return row != null; } @Override @@ -90,7 +91,7 @@ public Slice getSlice(int field) { @Override public Object getObject(int field) { - Object value = rows.get(rowId).get(field); + Object value = row[field]; if (columns.get(field).getKind() == InfluxColumn.Kind.TIME && value instanceof String) { return Instant.parse((String) value).toEpochMilli(); } @@ -99,11 +100,12 @@ public Object getObject(int field) { @Override public boolean isNull(int field) { - return rows.get(rowId).get(field) == null; + return row[field] == null; } @Override public void close() { rowId = rows.size(); + row = null; } } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java index 2d8d5c80af77..32674b697f32 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java @@ -6,14 +6,13 @@ import io.prestosql.spi.type.Type; import org.influxdb.dto.QueryResult; -import java.time.ZoneId; import java.util.*; public class InfluxRecordSet implements RecordSet { private final List columns; private final List columnTypes; - private final List> rows; + private final List rows; public InfluxRecordSet(List columns, List results) { this.columns = columns; @@ -25,26 +24,25 @@ public InfluxRecordSet(List columns, List resu } this.columnTypes = columnTypes.build(); this.rows = new ArrayList<>(); - Object[] row = new Object[columns.size()]; + final int IGNORE = -1; for (QueryResult.Series series: results) { if (series.getValues().isEmpty()) { continue; } - Arrays.fill(row, null); - if (series.getTags() != null) { - for (Map.Entry tag : series.getTags().entrySet()) { - row[mapping.get(tag.getKey())] = tag.getValue(); - } - } + // we can't push down group-bys so we have no tags to consider int[] fields = new int[series.getColumns().size()]; for (int i = 0; i < fields.length; i++) { - fields[i] = mapping.get(series.getColumns().get(i)); + fields[i] = mapping.getOrDefault(series.getColumns().get(i), IGNORE); } for (List values: series.getValues()) { + Object[] row = new Object[columns.size()]; for (int i = 0; i < fields.length; i++) { - row[fields[i]] = values.get(i); + int slot = fields[i]; + if (slot != IGNORE) { + row[slot] = values.get(i); + } } - rows.add(ImmutableList.copyOf(row)); + rows.add(row); } } } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSetProvider.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSetProvider.java index bd3267707428..2c72cb3b0d28 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSetProvider.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSetProvider.java @@ -25,22 +25,14 @@ public InfluxRecordSetProvider(InfluxClient client) { public RecordSet getRecordSet(ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorSplit split, ConnectorTableHandle tableHandle, List columns) { InfluxTableHandle table = (InfluxTableHandle) tableHandle; client.logger.info("getRecordSet(" + split + ", " + table + ", " + columns + ")"); - InfluxQL query = new InfluxQL(); ImmutableList.Builder handles = ImmutableList.builder(); - boolean hasTime = false; for (ColumnHandle handle: columns) { InfluxColumnHandle column = (InfluxColumnHandle) handle; - hasTime |= column.getInfluxName().equals("time"); InfluxError.GENERAL.check(column.getMeasurement().equals(table.getMeasurement()), "bad measurement for " + column + " in " + table); InfluxError.GENERAL.check(column.getRetentionPolicy().equals(table.getRetentionPolicy()), "bad retention-policy for " + column + " in " + table); - query.append(query.isEmpty()? "SELECT ": ", ").add(column); handles.add(column); } - if (!hasTime) { - query.append(query.isEmpty()? "SELECT time": ", time"); - handles.add(InfluxColumn.TIME); - } - query.append(' ').append(table.getFromWhere()); + InfluxQL query = new InfluxQL("SELECT * ").append(table.getFromWhere()); List results = client.execute(query.toString()); // actually run the query against our Influx server return new InfluxRecordSet(handles.build(), results); } From aff0f885584e7ad05ba5a2beb168a361363eca55 Mon Sep 17 00:00:00 2001 From: Will Edwards Date: Fri, 20 Dec 2019 18:53:07 +0100 Subject: [PATCH 04/19] Timestamps now UTC; support LIMIT --- .../io/prestosql/plugin/influx/InfluxColumn.java | 2 +- .../prestosql/plugin/influx/InfluxMetadata.java | 13 ++++++++++++- .../java/io/prestosql/plugin/influx/InfluxQL.java | 10 ++++++++++ .../plugin/influx/InfluxRecordCursor.java | 5 ++++- .../plugin/influx/InfluxTableHandle.java | 15 +++++++++++++-- 5 files changed, 40 insertions(+), 5 deletions(-) diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumn.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumn.java index 2cf4337cefe1..76764219951a 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumn.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumn.java @@ -15,7 +15,7 @@ public class InfluxColumn extends ColumnMetadata { .put("boolean", BooleanType.BOOLEAN) .put("integer", BigintType.BIGINT) .put("float", DoubleType.DOUBLE) - .put("time", TimestampType.TIMESTAMP) + .put("time", TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE) .build(); public static final InfluxColumn TIME = new InfluxColumn("time", "time", Kind.TIME); diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java index 5461df36bb23..8821de148a8e 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java @@ -117,6 +117,16 @@ public ConnectorTableProperties getTableProperties(ConnectorSession session, Con return new ConnectorTableProperties(); } + @Override + public Optional> applyLimit(ConnectorSession session, ConnectorTableHandle handle, long limit) { + InfluxTableHandle table = (InfluxTableHandle) handle; + return Optional.of(new LimitApplicationResult<>(new InfluxTableHandle( + table.getRetentionPolicy(), + table.getMeasurement(), + table.getWhere(), + limit), true)); + } + @Override public Optional> applyFilter(ConnectorSession session, ConnectorTableHandle handle, Constraint constraint) { boolean all = true; @@ -204,6 +214,7 @@ public Optional> applyFilter(C return Optional.of(new ConstraintApplicationResult<>(new InfluxTableHandle( table.getRetentionPolicy(), table.getMeasurement(), - where), all? TupleDomain.all(): constraint.getSummary())); + where, + table.getLimit()), all? TupleDomain.all(): constraint.getSummary())); } } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java index 85e131e7dc57..7796b8d5047e 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java @@ -32,6 +32,16 @@ public InfluxQL append(char ch) { return this; } + public InfluxQL append(long l) { + influxQL.append(l); + return this; + } + + public InfluxQL append(int i) { + influxQL.append(i); + return this; + } + public InfluxQL add(InfluxColumn column) { addIdentifier(column.getInfluxName()); return this; diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordCursor.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordCursor.java index 96ddead08941..4aaf40d15610 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordCursor.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordCursor.java @@ -3,6 +3,8 @@ import io.airlift.slice.Slice; import io.airlift.slice.Slices; import io.prestosql.spi.connector.RecordCursor; +import io.prestosql.spi.type.DateTimeEncoding; +import io.prestosql.spi.type.TimeZoneKey; import io.prestosql.spi.type.Type; import java.time.Instant; @@ -93,7 +95,8 @@ public Slice getSlice(int field) { public Object getObject(int field) { Object value = row[field]; if (columns.get(field).getKind() == InfluxColumn.Kind.TIME && value instanceof String) { - return Instant.parse((String) value).toEpochMilli(); + Instant timestamp = Instant.parse((String) value); + return DateTimeEncoding.packDateTimeWithZone(timestamp.toEpochMilli(), TimeZoneKey.UTC_KEY); } return value; } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTableHandle.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTableHandle.java index 3bfb2da01759..e0be0b700b69 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTableHandle.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTableHandle.java @@ -12,19 +12,22 @@ public class InfluxTableHandle extends SchemaTableName implements ConnectorTable private final String retentionPolicy; private final String measurement; private final InfluxQL where; + private final Long limit; @JsonCreator public InfluxTableHandle(@JsonProperty("retentionPolicy") String retentionPolicy, @JsonProperty("measurement") String measurement, - @JsonProperty("where") InfluxQL where) { + @JsonProperty("where") InfluxQL where, + @JsonProperty("limit") Long limit) { super(retentionPolicy, measurement); this.retentionPolicy = requireNonNull(retentionPolicy, "retentionPolicy is null"); this.measurement = requireNonNull(measurement, "measurement is null"); this.where = requireNonNull(where, "where is null"); + this.limit = limit; } public InfluxTableHandle(String retentionPolicy, String measurement) { - this(retentionPolicy, measurement, new InfluxQL()); + this(retentionPolicy, measurement, new InfluxQL(), null); } @JsonProperty @@ -42,12 +45,20 @@ public InfluxQL getWhere() { return where; } + @JsonProperty + public Long getLimit() { + return limit; + } + public InfluxQL getFromWhere() { InfluxQL from = new InfluxQL("FROM ") .addIdentifier(getRetentionPolicy()).append('.').addIdentifier(getMeasurement()); if (!getWhere().isEmpty()) { from.append(' ').append(getWhere().toString()); } + if (getLimit() != null) { + from.append(" LIMIT ").append(getLimit()); + } return from; } From baae62ef88cead32096b5fec3bcedc8b7e0bc255 Mon Sep 17 00:00:00 2001 From: Will Edwards Date: Fri, 20 Dec 2019 20:20:03 +0100 Subject: [PATCH 05/19] Removed dependency on InfluxDB client libs, as they caused mvn package dependency problems --- presto-influx/pom.xml | 7 -- .../prestosql/plugin/influx/InfluxClient.java | 68 ++++++++++--------- .../prestosql/plugin/influx/InfluxError.java | 5 ++ .../plugin/influx/InfluxMetadata.java | 14 +++- .../plugin/influx/InfluxRecordCursor.java | 7 +- .../plugin/influx/InfluxRecordSet.java | 47 ++++++++++--- .../influx/InfluxRecordSetProvider.java | 9 +-- 7 files changed, 97 insertions(+), 60 deletions(-) diff --git a/presto-influx/pom.xml b/presto-influx/pom.xml index 9e448236af57..b6f4bfa46e89 100644 --- a/presto-influx/pom.xml +++ b/presto-influx/pom.xml @@ -15,13 +15,6 @@ presto-plugin - - - org.influxdb - influxdb-java - 2.14 - - io.airlift bootstrap diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java index 433b45a64024..04a4a4bf1524 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java @@ -1,16 +1,17 @@ package io.prestosql.plugin.influx; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +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 org.influxdb.InfluxDB; -import org.influxdb.InfluxDBException; -import org.influxdb.InfluxDBFactory; -import org.influxdb.dto.Query; -import org.influxdb.dto.QueryResult; import javax.inject.Inject; +import java.net.URL; +import java.net.URLEncoder; +import java.nio.charset.StandardCharsets; import java.util.*; import java.util.concurrent.ConcurrentHashMap; import java.util.function.Supplier; @@ -21,7 +22,6 @@ public class InfluxClient { final Logger logger; private final InfluxConfig config; - private final InfluxDB influxDB; // the various metadata are cached for a configurable number of milliseconds so we don't hammer the server private final CachedMetaData> retentionPolicies; private final CachedMetaData> measurements; @@ -32,8 +32,6 @@ public class InfluxClient { public InfluxClient(InfluxConfig config) { this.logger = Logger.get(getClass()); this.config = requireNonNull(config, "config is null"); - this.influxDB = InfluxDBFactory.connect("http://" + config.getHost() + ":" + config.getPort(), - config.getUserName(), config.getPassword()); this.retentionPolicies = new CachedMetaData<>(() -> showNames("SHOW RETENTION POLICIES")); this.measurements = new CachedMetaData<>(() -> showNames("SHOW MEASUREMENTS")); this.tagKeys = new ConcurrentHashMap<>(); @@ -91,15 +89,15 @@ private Map getFields(String schemaName, String tableName) .addIdentifier(measurement) .toString(); Map fields = new HashMap<>(); - for (QueryResult.Series series : execute(query)) { - int nameIndex = series.getColumns().indexOf("fieldKey"); - int typeIndex = series.getColumns().indexOf("fieldType"); - for (List row : series.getValues()) { - String name = row.get(nameIndex).toString(); - String influxType = row.get(typeIndex).toString(); - InfluxColumn collision = fields.put(name.toLowerCase(), new InfluxColumn(name, influxType, InfluxColumn.Kind.FIELD)); - if (collision != null) { - InfluxError.IDENTIFIER_CASE_SENSITIVITY.fail("identifier " + name + " collides with " + collision.getInfluxName(), query); + for (JsonNode series : execute(query)) { + 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(), new InfluxColumn(name, influxType, InfluxColumn.Kind.FIELD)); + if (collision != null) { + InfluxError.IDENTIFIER_CASE_SENSITIVITY.fail("identifier " + name + " collides with " + collision.getInfluxName(), query); + } } } } @@ -126,9 +124,13 @@ public List getColumns(String schemaName, String tableName) { private Map showNames(String query) { Map names = new HashMap<>(); - for (QueryResult.Series series: execute(query)) { - for (List row: series.getValues()) { - String name = row.get(0).toString(); + JsonNode series = execute(query); + InfluxError.GENERAL.check(series.getNodeType().equals(JsonNodeType.ARRAY), "expecting an array, not " + series, query); + InfluxError.GENERAL.check(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(), name); if (collision != null) { InfluxError.IDENTIFIER_CASE_SENSITIVITY.fail("identifier " + name + " collides with " + collision, query); @@ -138,20 +140,24 @@ private Map showNames(String query) { return ImmutableMap.copyOf(names); } - List execute(String query) { + JsonNode execute(String query) { logger.debug("executing: " + query); - QueryResult result; + final JsonNode response; try { - result = influxDB.query(new Query(query, config.getDatabase())); - } catch (InfluxDBException e) { - InfluxError.GENERAL.fail(e.toString(), query); - return Collections.emptyList(); + URL url = new URL("http://" + config.getUserName() + ":" + config.getPassword() + "@" + config.getHost() + ":" + config.getPort() + + "/query?db=" + config.getDatabase() + "&q=" + URLEncoder.encode(query, StandardCharsets.UTF_8.toString())); + response = new ObjectMapper().readTree(url); + } catch (Throwable t) { + InfluxError.EXTERNAL.fail(t); + return null; + } + JsonNode results = response.get("results"); + InfluxError.GENERAL.check(results.size() == 1, "expecting one result", query); + JsonNode result = results.get(0); + if (result.has("error")) { + InfluxError.GENERAL.fail(result.get("error").asText(), query); } - InfluxError.GENERAL.check(!result.hasError(), result.getError(), query); - InfluxError.GENERAL.check(result.getResults().size() == 1, "expecting 1 series", query); - InfluxError.GENERAL.check(!result.getResults().get(0).hasError(), result.getResults().get(0).getError(), query); - List series = result.getResults().get(0).getSeries(); - return series != null? series: Collections.emptyList(); + return result.get("series"); } public HostAddress getHostAddress() { diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxError.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxError.java index a04d636a4cde..0117ad607e7b 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxError.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxError.java @@ -10,6 +10,7 @@ public enum InfluxError implements ErrorCodeSupplier { GENERAL (ErrorType.INTERNAL_ERROR), + EXTERNAL (ErrorType.EXTERNAL), IDENTIFIER_CASE_SENSITIVITY (ErrorType.EXTERNAL), BAD_VALUE (ErrorType.USER_ERROR); @@ -38,6 +39,10 @@ public void fail(String message) { fail(message, null); } + public void fail(Throwable t) { + throw new PrestoException(this, t); + } + @Override public ErrorCode toErrorCode() { return errorCode; diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java index 8821de148a8e..294d9a0a6ca6 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java @@ -4,8 +4,10 @@ import com.google.common.collect.ImmutableMap; import io.prestosql.spi.connector.*; import io.prestosql.spi.predicate.*; +import io.prestosql.spi.type.DateTimeEncoding; import javax.inject.Inject; +import java.time.Instant; import java.util.*; import static java.util.Objects.requireNonNull; @@ -161,7 +163,11 @@ public Optional> applyFilter(C InfluxError.GENERAL.fail("bad low bound", range.toString(session)); continue; } - where.add(column).append(low).add(range.getLow().getValue()); + Object value = range.getLow().getValue(); + if (column.getKind() == InfluxColumn.Kind.TIME && value instanceof Long) { + value = Instant.ofEpochMilli(DateTimeEncoding.unpackMillisUtc((Long)value)).toString(); + } + where.add(column).append(low).add(value); hasLow = true; } if (range.getHigh().getValueBlock().isPresent()) { @@ -180,7 +186,11 @@ public Optional> applyFilter(C if (hasLow) { where.append(" AND "); } - where.add(column).append(high).add(range.getHigh().getValue()); + Object value = range.getHigh().getValue(); + if (column.getKind() == InfluxColumn.Kind.TIME && value instanceof Long) { + value = Instant.ofEpochMilli(DateTimeEncoding.unpackMillisUtc((Long)value)).toString(); + } + where.add(column).append(high).add(value); } } first = false; diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordCursor.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordCursor.java index 4aaf40d15610..f54f12def27d 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordCursor.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordCursor.java @@ -93,12 +93,7 @@ public Slice getSlice(int field) { @Override public Object getObject(int field) { - Object value = row[field]; - if (columns.get(field).getKind() == InfluxColumn.Kind.TIME && value instanceof String) { - Instant timestamp = Instant.parse((String) value); - return DateTimeEncoding.packDateTimeWithZone(timestamp.toEpochMilli(), TimeZoneKey.UTC_KEY); - } - return value; + return row[field]; } @Override diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java index 32674b697f32..f81a84946461 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java @@ -1,11 +1,14 @@ package io.prestosql.plugin.influx; +import com.fasterxml.jackson.databind.JsonNode; import com.google.common.collect.ImmutableList; import io.prestosql.spi.connector.RecordCursor; import io.prestosql.spi.connector.RecordSet; +import io.prestosql.spi.type.DateTimeEncoding; +import io.prestosql.spi.type.TimeZoneKey; import io.prestosql.spi.type.Type; -import org.influxdb.dto.QueryResult; +import java.time.Instant; import java.util.*; public class InfluxRecordSet implements RecordSet { @@ -14,7 +17,7 @@ public class InfluxRecordSet implements RecordSet { private final List columnTypes; private final List rows; - public InfluxRecordSet(List columns, List results) { + public InfluxRecordSet(List columns, JsonNode results) { this.columns = columns; ImmutableList.Builder columnTypes = new ImmutableList.Builder<>(); Map mapping = new HashMap<>(); @@ -25,21 +28,49 @@ public InfluxRecordSet(List columns, List resu this.columnTypes = columnTypes.build(); this.rows = new ArrayList<>(); final int IGNORE = -1; - for (QueryResult.Series series: results) { - if (series.getValues().isEmpty()) { + for (JsonNode series: results) { + if (!series.has("values")) { continue; } // we can't push down group-bys so we have no tags to consider - int[] fields = new int[series.getColumns().size()]; + JsonNode header = series.get("columns"); + int[] fields = new int[header.size()]; for (int i = 0; i < fields.length; i++) { - fields[i] = mapping.getOrDefault(series.getColumns().get(i), IGNORE); + fields[i] = mapping.getOrDefault(header.get(i).textValue(), IGNORE); } - for (List values: series.getValues()) { + for (JsonNode values: series.get("values")) { Object[] row = new Object[columns.size()]; for (int i = 0; i < fields.length; i++) { int slot = fields[i]; if (slot != IGNORE) { - row[slot] = values.get(i); + final Object value; + JsonNode node = values.get(i); + if (node.isNull()) { + value = null; + } else { + switch (columns.get(slot).getInfluxType()) { + case "string": + value = node.textValue(); + break; + case "boolean": + value = node.booleanValue(); + break; + case "integer": + value = node.longValue(); + break; + case "float": + value = node.doubleValue(); + break; + case "time": + Instant timestamp = Instant.parse(node.textValue()); + value = DateTimeEncoding.packDateTimeWithZone(timestamp.toEpochMilli(), TimeZoneKey.UTC_KEY); + break; + default: + InfluxError.GENERAL.fail("cannot map " + node + " to " + columns.get(slot)); + value = null; + } + } + row[slot] = value; } } rows.add(row); diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSetProvider.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSetProvider.java index 2c72cb3b0d28..d156619d62fe 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSetProvider.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSetProvider.java @@ -1,13 +1,10 @@ package io.prestosql.plugin.influx; +import com.fasterxml.jackson.databind.JsonNode; import com.google.common.collect.ImmutableList; import io.prestosql.spi.connector.*; -import org.influxdb.dto.QueryResult; import javax.inject.Inject; -import java.time.ZoneId; -import java.util.ArrayList; -import java.util.Collections; import java.util.List; import static java.util.Objects.requireNonNull; @@ -32,8 +29,8 @@ public RecordSet getRecordSet(ConnectorTransactionHandle transaction, ConnectorS InfluxError.GENERAL.check(column.getRetentionPolicy().equals(table.getRetentionPolicy()), "bad retention-policy for " + column + " in " + table); handles.add(column); } - InfluxQL query = new InfluxQL("SELECT * ").append(table.getFromWhere()); - List results = client.execute(query.toString()); // actually run the query against our Influx server + String query = new InfluxQL("SELECT * ").append(table.getFromWhere()).toString(); + JsonNode results = client.execute(query); // actually run the query against our Influx server return new InfluxRecordSet(handles.build(), results); } } From 9da423262e0dd58eeac18a17a3e5fa7d7d470152 Mon Sep 17 00:00:00 2001 From: Will Edwards Date: Fri, 20 Dec 2019 20:39:55 +0100 Subject: [PATCH 06/19] Okay, using the Airlift codestyle and it *still* doesn't pass mvn checkstyle! --- .../prestosql/plugin/influx/InfluxClient.java | 79 ++++++++---- .../prestosql/plugin/influx/InfluxColumn.java | 53 ++++++-- .../plugin/influx/InfluxColumnHandle.java | 44 +++++-- .../prestosql/plugin/influx/InfluxConfig.java | 51 ++++++-- .../plugin/influx/InfluxConnector.java | 35 ++++- .../plugin/influx/InfluxConnectorFactory.java | 27 +++- .../prestosql/plugin/influx/InfluxError.java | 52 ++++++-- .../plugin/influx/InfluxHandleResolver.java | 14 ++ .../plugin/influx/InfluxMetadata.java | 121 +++++++++++++----- .../prestosql/plugin/influx/InfluxModule.java | 21 ++- .../prestosql/plugin/influx/InfluxPlugin.java | 18 ++- .../io/prestosql/plugin/influx/InfluxQL.java | 71 +++++++--- .../plugin/influx/InfluxRecordCursor.java | 62 ++++++--- .../plugin/influx/InfluxRecordSet.java | 41 ++++-- .../influx/InfluxRecordSetProvider.java | 35 ++++- .../prestosql/plugin/influx/InfluxSplit.java | 33 ++++- .../plugin/influx/InfluxSplitManager.java | 26 +++- .../plugin/influx/InfluxTableHandle.java | 47 +++++-- .../influx/InfluxTransactionHandle.java | 17 ++- 19 files changed, 651 insertions(+), 196 deletions(-) diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java index 04a4a4bf1524..e14b10cf7333 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java @@ -1,3 +1,17 @@ +/* + * 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; @@ -9,16 +23,22 @@ import io.prestosql.spi.HostAddress; import javax.inject.Inject; + import java.net.URL; import java.net.URLEncoder; import java.nio.charset.StandardCharsets; -import java.util.*; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.function.Supplier; import static java.util.Objects.requireNonNull; -public class InfluxClient { +public class InfluxClient +{ final Logger logger; private final InfluxConfig config; @@ -29,7 +49,8 @@ public class InfluxClient { private final Map>>> fields; @Inject - public InfluxClient(InfluxConfig config) { + public InfluxClient(InfluxConfig config) + { this.logger = Logger.get(getClass()); this.config = requireNonNull(config, "config is null"); this.retentionPolicies = new CachedMetaData<>(() -> showNames("SHOW RETENTION POLICIES")); @@ -38,23 +59,28 @@ public InfluxClient(InfluxConfig config) { this.fields = new ConcurrentHashMap<>(); } - public Collection getSchemaNames() { + public Collection getSchemaNames() + { return retentionPolicies.get().keySet(); } - public String getRetentionPolicy(String schemaName) { + public String getRetentionPolicy(String schemaName) + { return retentionPolicies.get().get(schemaName); } - public Collection getTableNames() { + public Collection getTableNames() + { return measurements.get().keySet(); } - public String getMeasurement(String tableName) { + public String getMeasurement(String tableName) + { return measurements.get().get(tableName); } - private Map getTags(String tableName) { + private Map getTags(String tableName) + { return tagKeys.computeIfAbsent(tableName, k -> new CachedMetaData<>(() -> { String measurement = measurements.get().get(tableName); @@ -65,7 +91,7 @@ private Map getTags(String tableName) { .addIdentifier(measurement) .toString(); ImmutableMap.Builder tags = new ImmutableMap.Builder<>(); - for (Map.Entry name: showNames(query).entrySet()) { + for (Map.Entry name : showNames(query).entrySet()) { tags.put(name.getKey(), new InfluxColumn(name.getValue(), "string", InfluxColumn.Kind.TAG)); } return tags.build(); @@ -74,12 +100,13 @@ private Map getTags(String tableName) { .get(); } - private Map getFields(String schemaName, String tableName) { + private Map getFields(String schemaName, String tableName) + { return fields.computeIfAbsent(schemaName, k -> new HashMap<>()) .computeIfAbsent(tableName, k -> new CachedMetaData<>(() -> { - String retentionPolicy = retentionPolicies.get().get(schemaName); + String retentionPolicy = retentionPolicies.get().get(schemaName); String measurement = measurements.get().get(tableName); if (retentionPolicy == null || measurement == null) { return Collections.emptyMap(); @@ -106,11 +133,13 @@ private Map getFields(String schemaName, String tableName) .get(); } - public boolean tableExistsInSchema(String schemaName, String tableName) { + public boolean tableExistsInSchema(String schemaName, String tableName) + { return !getFields(schemaName, tableName).isEmpty(); } - public List getColumns(String schemaName, String tableName) { + public List getColumns(String schemaName, String tableName) + { Collection fields = getFields(schemaName, tableName).values(); if (fields.isEmpty()) { return Collections.emptyList(); @@ -122,14 +151,15 @@ public List getColumns(String schemaName, String tableName) { return columns.build(); } - private Map showNames(String query) { + private Map showNames(String query) + { Map names = new HashMap<>(); JsonNode series = execute(query); InfluxError.GENERAL.check(series.getNodeType().equals(JsonNodeType.ARRAY), "expecting an array, not " + series, query); InfluxError.GENERAL.check(series.size() == 1, "expecting one element, not " + series, query); series = series.get(0); if (series.has("values")) { - for (JsonNode row: series.get("values")) { + for (JsonNode row : series.get("values")) { String name = row.get(0).textValue(); String collision = names.put(name.toLowerCase(), name); if (collision != null) { @@ -140,14 +170,16 @@ private Map showNames(String query) { return ImmutableMap.copyOf(names); } - JsonNode execute(String query) { + JsonNode execute(String query) + { logger.debug("executing: " + query); final JsonNode response; try { URL url = new URL("http://" + config.getUserName() + ":" + config.getPassword() + "@" + config.getHost() + ":" + config.getPort() + "/query?db=" + config.getDatabase() + "&q=" + URLEncoder.encode(query, StandardCharsets.UTF_8.toString())); response = new ObjectMapper().readTree(url); - } catch (Throwable t) { + } + catch (Throwable t) { InfluxError.EXTERNAL.fail(t); return null; } @@ -160,27 +192,30 @@ JsonNode execute(String query) { return result.get("series"); } - public HostAddress getHostAddress() { + public HostAddress getHostAddress() + { return HostAddress.fromParts(config.getHost(), config.getPort()); } - private class CachedMetaData { + private class CachedMetaData + { private final Supplier loader; private T value; private long lastLoaded; - private CachedMetaData(Supplier loader) { + private CachedMetaData(Supplier loader) + { this.loader = loader; } - public synchronized T get() { + public synchronized T get() + { if (System.currentTimeMillis() > lastLoaded + config.getCacheMetaDataMillis()) { value = loader.get(); lastLoaded = System.currentTimeMillis(); } return value; } - } } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumn.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumn.java index 76764219951a..4b55a7a129a9 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumn.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumn.java @@ -1,3 +1,17 @@ +/* + * 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.annotation.JsonCreator; @@ -5,10 +19,18 @@ import com.google.common.base.MoreObjects; import com.google.common.collect.ImmutableMap; import io.prestosql.spi.connector.ColumnMetadata; -import io.prestosql.spi.type.*; +import io.prestosql.spi.type.BigintType; +import io.prestosql.spi.type.BooleanType; +import io.prestosql.spi.type.DoubleType; +import io.prestosql.spi.type.TimestampWithTimeZoneType; +import io.prestosql.spi.type.Type; +import io.prestosql.spi.type.VarcharType; -public class InfluxColumn extends ColumnMetadata { +public class InfluxColumn + extends ColumnMetadata +{ + public static final InfluxColumn TIME = new InfluxColumn("time", "time", Kind.TIME); // map InfluxDB types to Presto types private final static ImmutableMap TYPES_MAPPING = new ImmutableMap.Builder() .put("string", VarcharType.VARCHAR) @@ -17,16 +39,15 @@ public class InfluxColumn extends ColumnMetadata { .put("float", DoubleType.DOUBLE) .put("time", TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE) .build(); - public static final InfluxColumn TIME = new InfluxColumn("time", "time", Kind.TIME); - private final String influxName; private final String influxType; private final Kind kind; @JsonCreator public InfluxColumn(@JsonProperty("influxName") String influxName, - @JsonProperty("influxType") String influxType, - @JsonProperty("kind") Kind kind) { + @JsonProperty("influxType") String influxType, + @JsonProperty("kind") Kind kind) + { super(influxName.toLowerCase(), TYPES_MAPPING.get(influxType), null, kind.name().toLowerCase(), false); this.influxName = influxName; this.influxType = influxType; @@ -34,21 +55,25 @@ public InfluxColumn(@JsonProperty("influxName") String influxName, } @JsonProperty - public String getInfluxName() { + public String getInfluxName() + { return influxName; } @JsonProperty - public String getInfluxType() { + public String getInfluxType() + { return influxType; } @JsonProperty - public Kind getKind() { + public Kind getKind() + { return kind; } - protected MoreObjects.ToStringHelper toStringHelper(Object self) { + protected MoreObjects.ToStringHelper toStringHelper(Object self) + { MoreObjects.ToStringHelper helper = com.google.common.base.MoreObjects.toStringHelper(self) .addValue(getName()) .addValue(getType()) @@ -60,11 +85,13 @@ protected MoreObjects.ToStringHelper toStringHelper(Object self) { } @Override - public String toString() { - return toStringHelper(this).toString(); + public String toString() + { + return toStringHelper(this).toString(); } - public enum Kind { + public enum Kind + { TIME, TAG, FIELD, diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumnHandle.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumnHandle.java index 6fac484db5d9..7fab78cbd6b0 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumnHandle.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumnHandle.java @@ -1,45 +1,65 @@ +/* + * 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.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import io.prestosql.spi.connector.ColumnHandle; -import io.prestosql.spi.connector.SchemaTableName; - -public class InfluxColumnHandle extends InfluxColumn implements ColumnHandle { +public class InfluxColumnHandle + extends InfluxColumn + implements ColumnHandle +{ private final String retentionPolicy; private final String measurement; @JsonCreator public InfluxColumnHandle(@JsonProperty("retentionPolicy") String retentionPolicy, - @JsonProperty("measurement") String measurement, - @JsonProperty("influxName") String influxName, - @JsonProperty("influxType") String influxType, - @JsonProperty("kind") Kind kind) { + @JsonProperty("measurement") String measurement, + @JsonProperty("influxName") String influxName, + @JsonProperty("influxType") String influxType, + @JsonProperty("kind") Kind kind) + { super(influxName, influxType, kind); this.retentionPolicy = retentionPolicy; this.measurement = measurement; } public InfluxColumnHandle(String retentionPolicy, - String measurement, - InfluxColumn column) { + String measurement, + InfluxColumn column) + { this(retentionPolicy, measurement, column.getInfluxName(), column.getInfluxType(), column.getKind()); } @JsonProperty - public String getRetentionPolicy() { + public String getRetentionPolicy() + { return retentionPolicy; } @JsonProperty - public String getMeasurement() { + public String getMeasurement() + { return measurement; } @Override - public String toString() { + public String toString() + { return toStringHelper(this) .addValue(getRetentionPolicy()) .addValue(getMeasurement()) diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConfig.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConfig.java index b8cbdf470507..95b6cdd59e57 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConfig.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConfig.java @@ -1,11 +1,25 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.prestosql.plugin.influx; import io.airlift.configuration.Config; import javax.validation.constraints.NotNull; -import java.net.URI; -public class InfluxConfig { +public class InfluxConfig +{ private long cacheMetaDataMillis = 10000; private String host = "localhost"; @@ -15,12 +29,14 @@ public class InfluxConfig { private String password; @NotNull - public long getCacheMetaDataMillis() { + public long getCacheMetaDataMillis() + { return cacheMetaDataMillis; } @Config("cache-meta-data-millis") - public InfluxConfig setCacheMetaDataMillis(long cacheMetaDataMillis) { + public InfluxConfig setCacheMetaDataMillis(long cacheMetaDataMillis) + { this.cacheMetaDataMillis = cacheMetaDataMillis; return this; } @@ -32,50 +48,59 @@ public String getHost() } @Config("host") - public InfluxConfig setHost(String host) { + public InfluxConfig setHost(String host) + { this.host = host; return this; } - public int getPort() { + public int getPort() + { return port; } @Config("port") - public InfluxConfig setPort(int port) { + public InfluxConfig setPort(int port) + { this.port = port; return this; } @NotNull - public String getDatabase() { + public String getDatabase() + { return database; } @Config("database") - public InfluxConfig setDatabase(String database) { + public InfluxConfig setDatabase(String database) + { this.database = database; return this; } @NotNull - public String getUserName() { + public String getUserName() + { return userName; } @Config("user") - public InfluxConfig setUserName(String userName) { + public InfluxConfig setUserName(String userName) + { this.userName = userName; return this; } @NotNull - public String getPassword() { + public String getPassword() + { return password; } @Config("password") - public InfluxConfig setPassword(String password) { + public InfluxConfig setPassword(String password) + { this.password = password; return this; } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConnector.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConnector.java index 9f4e57dcdc2b..929bf9e0f3e4 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConnector.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConnector.java @@ -1,7 +1,25 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.prestosql.plugin.influx; import io.airlift.bootstrap.LifeCycleManager; -import io.prestosql.spi.connector.*; +import io.prestosql.spi.connector.Connector; +import io.prestosql.spi.connector.ConnectorMetadata; +import io.prestosql.spi.connector.ConnectorRecordSetProvider; +import io.prestosql.spi.connector.ConnectorSplitManager; +import io.prestosql.spi.connector.ConnectorTransactionHandle; import io.prestosql.spi.transaction.IsolationLevel; import javax.inject.Inject; @@ -9,9 +27,10 @@ import static io.prestosql.plugin.influx.InfluxTransactionHandle.INSTANCE; import static io.prestosql.spi.transaction.IsolationLevel.READ_COMMITTED; import static io.prestosql.spi.transaction.IsolationLevel.checkConnectorSupports; -import static java.util.Objects.requireNonNull; -public class InfluxConnector implements Connector { +public class InfluxConnector + implements Connector +{ private final LifeCycleManager lifeCycleManager; private final InfluxMetadata metadata; @@ -20,9 +39,10 @@ public class InfluxConnector implements Connector { @Inject public InfluxConnector(LifeCycleManager lifeCycleManager, - InfluxMetadata metadata, - InfluxSplitManager splitManager, - InfluxRecordSetProvider recordSetProvider) { + InfluxMetadata metadata, + InfluxSplitManager splitManager, + InfluxRecordSetProvider recordSetProvider) + { this.lifeCycleManager = lifeCycleManager; this.metadata = metadata; this.splitManager = splitManager; @@ -30,7 +50,8 @@ public InfluxConnector(LifeCycleManager lifeCycleManager, } @Override - public ConnectorTransactionHandle beginTransaction(IsolationLevel isolationLevel, boolean readOnly) { + public ConnectorTransactionHandle beginTransaction(IsolationLevel isolationLevel, boolean readOnly) + { checkConnectorSupports(READ_COMMITTED, isolationLevel); return INSTANCE; } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConnectorFactory.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConnectorFactory.java index a341ed13de66..0f5e00c1a6cf 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConnectorFactory.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConnectorFactory.java @@ -1,3 +1,17 @@ +/* + * 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.google.inject.Injector; @@ -12,20 +26,25 @@ import static java.util.Objects.requireNonNull; -public class InfluxConnectorFactory implements ConnectorFactory { +public class InfluxConnectorFactory + implements ConnectorFactory +{ @Override - public String getName() { + public String getName() + { return "influx"; } @Override - public ConnectorHandleResolver getHandleResolver() { + public ConnectorHandleResolver getHandleResolver() + { return new InfluxHandleResolver(); } @Override - public Connector create(String catalogName, Map config, ConnectorContext context) { + public Connector create(String catalogName, Map config, ConnectorContext context) + { requireNonNull(config, "config is null"); Bootstrap app = new Bootstrap( diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxError.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxError.java index 0117ad607e7b..a8f1108d01eb 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxError.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxError.java @@ -1,3 +1,17 @@ +/* + * 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 io.prestosql.spi.ErrorCode; @@ -7,49 +21,59 @@ import static com.google.common.base.MoreObjects.toStringHelper; -public enum InfluxError implements ErrorCodeSupplier { +public enum InfluxError + implements ErrorCodeSupplier +{ - GENERAL (ErrorType.INTERNAL_ERROR), - EXTERNAL (ErrorType.EXTERNAL), - IDENTIFIER_CASE_SENSITIVITY (ErrorType.EXTERNAL), - BAD_VALUE (ErrorType.USER_ERROR); + GENERAL(ErrorType.INTERNAL_ERROR), + EXTERNAL(ErrorType.EXTERNAL), + IDENTIFIER_CASE_SENSITIVITY(ErrorType.EXTERNAL), + BAD_VALUE(ErrorType.USER_ERROR); private static final int ERROR_BASE = 0; // FIXME needs allocating private final ErrorCode errorCode; - InfluxError(ErrorType type) { + InfluxError(ErrorType type) + { this.errorCode = new ErrorCode(ERROR_BASE + ordinal(), name(), type); } - public void check(boolean condition, String message, String context) { + public void check(boolean condition, String message, String context) + { if (!condition) { fail(message, context); } } - public void check(boolean condition, String message) { + public void check(boolean condition, String message) + { check(condition, message, null); } - public void fail(String message, String context) { - throw new PrestoException(this, message + (context != null && !context.isEmpty()? " " + context: "")); + public void fail(String message, String context) + { + throw new PrestoException(this, message + (context != null && !context.isEmpty() ? " " + context : "")); } - public void fail(String message) { + public void fail(String message) + { fail(message, null); } - public void fail(Throwable t) { + public void fail(Throwable t) + { throw new PrestoException(this, t); } @Override - public ErrorCode toErrorCode() { + public ErrorCode toErrorCode() + { return errorCode; } @Override - public String toString() { + public String toString() + { return toStringHelper(this) .add("code", errorCode) .toString(); diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxHandleResolver.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxHandleResolver.java index 91c40b093589..eea80146d3e6 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxHandleResolver.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxHandleResolver.java @@ -1,3 +1,17 @@ +/* + * 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 io.prestosql.spi.connector.ColumnHandle; diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java index 294d9a0a6ca6..77ab461ed71a 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java @@ -1,49 +1,91 @@ +/* + * 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.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import io.prestosql.spi.connector.*; -import io.prestosql.spi.predicate.*; +import io.prestosql.spi.connector.ColumnHandle; +import io.prestosql.spi.connector.ColumnMetadata; +import io.prestosql.spi.connector.ConnectorMetadata; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.connector.ConnectorTableHandle; +import io.prestosql.spi.connector.ConnectorTableMetadata; +import io.prestosql.spi.connector.ConnectorTableProperties; +import io.prestosql.spi.connector.Constraint; +import io.prestosql.spi.connector.ConstraintApplicationResult; +import io.prestosql.spi.connector.LimitApplicationResult; +import io.prestosql.spi.connector.SchemaTableName; +import io.prestosql.spi.connector.SchemaTablePrefix; +import io.prestosql.spi.predicate.Domain; +import io.prestosql.spi.predicate.EquatableValueSet; +import io.prestosql.spi.predicate.Range; +import io.prestosql.spi.predicate.SortedRangeSet; +import io.prestosql.spi.predicate.TupleDomain; +import io.prestosql.spi.predicate.ValueSet; import io.prestosql.spi.type.DateTimeEncoding; import javax.inject.Inject; + import java.time.Instant; -import java.util.*; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; import static java.util.Objects.requireNonNull; -public class InfluxMetadata implements ConnectorMetadata { +public class InfluxMetadata + implements ConnectorMetadata +{ private final InfluxClient client; @Inject - public InfluxMetadata(InfluxClient client) { + public InfluxMetadata(InfluxClient client) + { this.client = requireNonNull(client, "client is null"); } @Override - public List listSchemaNames(ConnectorSession session) { + public List listSchemaNames(ConnectorSession session) + { return ImmutableList.copyOf(client.getSchemaNames()); } @Override - public List listTables(ConnectorSession session, Optional schemaName) { + public List listTables(ConnectorSession session, Optional schemaName) + { Collection schemaNames; if (schemaName.isPresent()) { if (client.getSchemaNames().contains(schemaName.get())) { schemaNames = Collections.singletonList(schemaName.get()); - } else { + } + else { return Collections.emptyList(); } - } else { + } + else { schemaNames = client.getSchemaNames(); } // in Influx, all measurements can exist in all retention policies, // (and all tickets asking for a way to know which measurements are actually // used in which retention policy are closed as wont-fix) ImmutableList.Builder builder = ImmutableList.builder(); - for (String tableName: client.getTableNames()) { - for (String matchingSchemaName: schemaNames) { + for (String tableName : client.getTableNames()) { + for (String matchingSchemaName : schemaNames) { if (client.tableExistsInSchema(matchingSchemaName, tableName)) { builder.add(new SchemaTableName(matchingSchemaName, tableName)); } @@ -64,13 +106,14 @@ public InfluxTableHandle getTableHandle(ConnectorSession session, SchemaTableNam } @Override - public Map> listTableColumns(ConnectorSession session, SchemaTablePrefix prefix) { + public Map> listTableColumns(ConnectorSession session, SchemaTablePrefix prefix) + { requireNonNull(prefix, "prefix is null"); ImmutableMap.Builder> result = ImmutableMap.builder(); Collection schemaNames = client.getSchemaNames(); Collection tableNames = client.getTableNames(); - for (String schemaName: schemaNames) { - for (String tableName: tableNames) { + for (String schemaName : schemaNames) { + for (String tableName : tableNames) { SchemaTableName schemaTableName = new SchemaTableName(schemaName, tableName); if (prefix.matches(schemaTableName)) { List columns = client.getColumns(schemaName, tableName); @@ -84,20 +127,22 @@ public Map> listTableColumns(ConnectorSess } @Override - public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle table) { - InfluxTableHandle influxTable = (InfluxTableHandle)table; + public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle table) + { + InfluxTableHandle influxTable = (InfluxTableHandle) table; ImmutableList.Builder columns = new ImmutableList.Builder<>(); - for (InfluxColumn column: client.getColumns(influxTable.getSchemaName(), influxTable.getTableName())) { + for (InfluxColumn column : client.getColumns(influxTable.getSchemaName(), influxTable.getTableName())) { columns.add(new InfluxColumnHandle(influxTable.getRetentionPolicy(), influxTable.getMeasurement(), column)); } return new ConnectorTableMetadata(influxTable, columns.build()); } @Override - public Map getColumnHandles(ConnectorSession session, ConnectorTableHandle tableHandle) { + public Map getColumnHandles(ConnectorSession session, ConnectorTableHandle tableHandle) + { ImmutableMap.Builder handles = new ImmutableMap.Builder<>(); - for (ColumnMetadata column: getTableMetadata(session, tableHandle).getColumns()) { - handles.put(column.getName(), (InfluxColumnHandle)column); + for (ColumnMetadata column : getTableMetadata(session, tableHandle).getColumns()) { + handles.put(column.getName(), (InfluxColumnHandle) column); } return handles.build(); } @@ -109,7 +154,8 @@ public ColumnMetadata getColumnMetadata(ConnectorSession session, ConnectorTable } @Override - public boolean usesLegacyTableLayouts() { + public boolean usesLegacyTableLayouts() + { return false; } @@ -120,7 +166,8 @@ public ConnectorTableProperties getTableProperties(ConnectorSession session, Con } @Override - public Optional> applyLimit(ConnectorSession session, ConnectorTableHandle handle, long limit) { + public Optional> applyLimit(ConnectorSession session, ConnectorTableHandle handle, long limit) + { InfluxTableHandle table = (InfluxTableHandle) handle; return Optional.of(new LimitApplicationResult<>(new InfluxTableHandle( table.getRetentionPolicy(), @@ -130,10 +177,11 @@ public Optional> applyLimit(Connect } @Override - public Optional> applyFilter(ConnectorSession session, ConnectorTableHandle handle, Constraint constraint) { + public Optional> applyFilter(ConnectorSession session, ConnectorTableHandle handle, Constraint constraint) + { boolean all = true; InfluxQL where = new InfluxQL(); - for (Map.Entry predicate: constraint.getSummary().getDomains().orElse(Collections.emptyMap()).entrySet()) { + for (Map.Entry predicate : constraint.getSummary().getDomains().orElse(Collections.emptyMap()).entrySet()) { InfluxColumnHandle column = (InfluxColumnHandle) predicate.getKey(); ValueSet values = predicate.getValue().getValues(); if (values instanceof SortedRangeSet) { @@ -148,7 +196,8 @@ public Optional> applyFilter(C where.append(first ? where.isEmpty() ? "WHERE ((" : " AND ((" : ") OR ("); if (range.isSingleValue()) { where.add(column).append(" = ").add(range.getSingleValue()); - } else { + } + else { boolean hasLow = false; if (range.getLow().getValueBlock().isPresent()) { final String low; @@ -165,7 +214,7 @@ public Optional> applyFilter(C } Object value = range.getLow().getValue(); if (column.getKind() == InfluxColumn.Kind.TIME && value instanceof Long) { - value = Instant.ofEpochMilli(DateTimeEncoding.unpackMillisUtc((Long)value)).toString(); + value = Instant.ofEpochMilli(DateTimeEncoding.unpackMillisUtc((Long) value)).toString(); } where.add(column).append(low).add(value); hasLow = true; @@ -188,7 +237,7 @@ public Optional> applyFilter(C } Object value = range.getHigh().getValue(); if (column.getKind() == InfluxColumn.Kind.TIME && value instanceof Long) { - value = Instant.ofEpochMilli(DateTimeEncoding.unpackMillisUtc((Long)value)).toString(); + value = Instant.ofEpochMilli(DateTimeEncoding.unpackMillisUtc((Long) value)).toString(); } where.add(column).append(high).add(value); } @@ -198,23 +247,27 @@ public Optional> applyFilter(C if (first) { client.logger.warn("unhandled SortedRangeSet " + column + ":" + values.getClass().getName() + "=" + values.toString(session)); all = false; - } else { + } + else { where.append("))"); } - } else if (values instanceof EquatableValueSet) { + } + else if (values instanceof EquatableValueSet) { boolean first = true; - for (Object value: values.getDiscreteValues().getValues()) { - where.append(first? where.isEmpty()? "WHERE (": " AND (": " OR ") + for (Object value : values.getDiscreteValues().getValues()) { + where.append(first ? where.isEmpty() ? "WHERE (" : " AND (" : " OR ") .add(column).append(" = ").add(value); first = false; } if (first) { client.logger.warn("unhandled EquatableValueSet " + column + ":" + values.getClass().getName() + "=" + values.toString(session)); all = false; - } else { + } + else { where.append(')'); } - } else { + } + else { client.logger.warn("unhandled predicate " + column + ":" + values.getClass().getName() + "=" + values.toString(session)); all = false; } @@ -225,6 +278,6 @@ public Optional> applyFilter(C table.getRetentionPolicy(), table.getMeasurement(), where, - table.getLimit()), all? TupleDomain.all(): constraint.getSummary())); + table.getLimit()), all ? TupleDomain.all() : constraint.getSummary())); } } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxModule.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxModule.java index 4f2fd8b52e29..44f1ac1d4735 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxModule.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxModule.java @@ -1,3 +1,17 @@ +/* + * 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.google.inject.Binder; @@ -6,10 +20,13 @@ import static io.airlift.configuration.ConfigBinder.configBinder; -public class InfluxModule implements Module { +public class InfluxModule + implements Module +{ @Override - public void configure(Binder binder) { + public void configure(Binder binder) + { configBinder(binder).bindConfig(InfluxConfig.class); binder.bind(InfluxClient.class).in(Scopes.SINGLETON); binder.bind(InfluxConnector.class).in(Scopes.SINGLETON); diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxPlugin.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxPlugin.java index 8d4b8321cef8..e00e16a9a6f9 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxPlugin.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxPlugin.java @@ -1,10 +1,26 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.prestosql.plugin.influx; import com.google.common.collect.ImmutableList; import io.prestosql.spi.Plugin; import io.prestosql.spi.connector.ConnectorFactory; -public class InfluxPlugin implements Plugin { +public class InfluxPlugin + implements Plugin +{ @Override public Iterable getConnectorFactories() diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java index 7796b8d5047e..e5d83b94e77c 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java @@ -1,67 +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.prestosql.plugin.influx; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import io.airlift.slice.Slice; -public class InfluxQL { +public class InfluxQL +{ private final StringBuilder influxQL; - public InfluxQL() { + public InfluxQL() + { influxQL = new StringBuilder(); } @JsonCreator - public InfluxQL(@JsonProperty("q") String prefix) { + public InfluxQL(@JsonProperty("q") String prefix) + { influxQL = new StringBuilder(prefix); } - public InfluxQL append(InfluxQL fragment) { + public InfluxQL append(InfluxQL fragment) + { influxQL.append(fragment); return this; } - public InfluxQL append(String s) { + public InfluxQL append(String s) + { influxQL.append(s); return this; } - public InfluxQL append(char ch) { + public InfluxQL append(char ch) + { influxQL.append(ch); return this; } - public InfluxQL append(long l) { + public InfluxQL append(long l) + { influxQL.append(l); return this; } - public InfluxQL append(int i) { + public InfluxQL append(int i) + { influxQL.append(i); return this; } - public InfluxQL add(InfluxColumn column) { + public InfluxQL add(InfluxColumn column) + { addIdentifier(column.getInfluxName()); return this; } - public InfluxQL add(Object value) { - assert !(value instanceof InfluxColumn): value; + public InfluxQL add(Object value) + { + assert !(value instanceof InfluxColumn) : value; if (value == null) { influxQL.append("null"); - } else if (value instanceof Slice) { + } + else if (value instanceof Slice) { quote(((Slice) value).toStringUtf8(), '\''); - } else if (value instanceof Number || value instanceof Boolean) { + } + else if (value instanceof Number || value instanceof Boolean) { influxQL.append(value); - } else { + } + else { quote(value.toString(), '\''); } return this; } - public InfluxQL addIdentifier(String identifier) { + public InfluxQL addIdentifier(String identifier) + { boolean safe = true; for (int i = 0; i < identifier.length() && safe; i++) { char ch = identifier.charAt(i); @@ -69,13 +97,15 @@ public InfluxQL addIdentifier(String identifier) { } if (safe) { influxQL.append(identifier); - } else { + } + else { quote(identifier, '"'); } return this; } - public void quote(String value, char delimiter) { + public void quote(String value, char delimiter) + { append(delimiter); for (int i = 0; i < value.length(); i++) { char ch = value.charAt(i); @@ -90,14 +120,15 @@ public void quote(String value, char delimiter) { append(delimiter); } - public boolean isEmpty() { + public boolean isEmpty() + { return influxQL.length() == 0; } @JsonProperty("q") @Override - public String toString() { + public String toString() + { return influxQL.toString(); } - } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordCursor.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordCursor.java index f54f12def27d..39fd4a209100 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordCursor.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordCursor.java @@ -1,53 +1,71 @@ +/* + * 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 io.airlift.slice.Slice; import io.airlift.slice.Slices; import io.prestosql.spi.connector.RecordCursor; -import io.prestosql.spi.type.DateTimeEncoding; -import io.prestosql.spi.type.TimeZoneKey; import io.prestosql.spi.type.Type; -import java.time.Instant; import java.util.List; import java.util.Objects; - -public class InfluxRecordCursor implements RecordCursor { +public class InfluxRecordCursor + implements RecordCursor +{ private final List columns; private final List rows; private Object[] row; private int rowId; - public InfluxRecordCursor(List columns, List rows) { + public InfluxRecordCursor(List columns, List rows) + { this.columns = columns; this.rows = rows; this.rowId = -1; } @Override - public long getCompletedBytes() { + public long getCompletedBytes() + { return 0; } @Override - public long getReadTimeNanos() { + public long getReadTimeNanos() + { return 0; } @Override - public Type getType(int field) { + public Type getType(int field) + { return columns.get(field).getType(); } @Override - public boolean advanceNextPosition() { - row = ++rowId < rows.size()? rows.get(rowId): null; + public boolean advanceNextPosition() + { + row = ++rowId < rows.size() ? rows.get(rowId) : null; return row != null; } @Override - public boolean getBoolean(int field) { + public boolean getBoolean(int field) + { Object value = getObject(field); if (value instanceof Boolean) { return (Boolean) value; @@ -62,7 +80,8 @@ public boolean getBoolean(int field) { } @Override - public long getLong(int field) { + public long getLong(int field) + { Object value = getObject(field); if (value instanceof Number) { return ((Number) value).longValue(); @@ -74,7 +93,8 @@ public long getLong(int field) { } @Override - public double getDouble(int field) { + public double getDouble(int field) + { Object value = getObject(field); if (value instanceof Number) { return ((Number) value).doubleValue(); @@ -86,23 +106,27 @@ public double getDouble(int field) { } @Override - public Slice getSlice(int field) { + public Slice getSlice(int field) + { String value = Objects.toString(getObject(field), null); - return value != null? Slices.utf8Slice(value): null; + return value != null ? Slices.utf8Slice(value) : null; } @Override - public Object getObject(int field) { + public Object getObject(int field) + { return row[field]; } @Override - public boolean isNull(int field) { + public boolean isNull(int field) + { return row[field] == null; } @Override - public void close() { + public void close() + { rowId = rows.size(); row = null; } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java index f81a84946461..c6b88d0da04f 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java @@ -1,3 +1,17 @@ +/* + * 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; @@ -9,26 +23,32 @@ import io.prestosql.spi.type.Type; import java.time.Instant; -import java.util.*; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; -public class InfluxRecordSet implements RecordSet { +public class InfluxRecordSet + implements RecordSet +{ private final List columns; private final List columnTypes; private final List rows; - public InfluxRecordSet(List columns, JsonNode results) { + public InfluxRecordSet(List columns, JsonNode results) + { this.columns = columns; ImmutableList.Builder columnTypes = new ImmutableList.Builder<>(); Map mapping = new HashMap<>(); - for (InfluxColumn column: columns) { + for (InfluxColumn column : columns) { columnTypes.add(column.getType()); mapping.put(column.getInfluxName(), mapping.size()); } this.columnTypes = columnTypes.build(); this.rows = new ArrayList<>(); final int IGNORE = -1; - for (JsonNode series: results) { + for (JsonNode series : results) { if (!series.has("values")) { continue; } @@ -38,7 +58,7 @@ public InfluxRecordSet(List columns, JsonNode results) { for (int i = 0; i < fields.length; i++) { fields[i] = mapping.getOrDefault(header.get(i).textValue(), IGNORE); } - for (JsonNode values: series.get("values")) { + for (JsonNode values : series.get("values")) { Object[] row = new Object[columns.size()]; for (int i = 0; i < fields.length; i++) { int slot = fields[i]; @@ -47,7 +67,8 @@ public InfluxRecordSet(List columns, JsonNode results) { JsonNode node = values.get(i); if (node.isNull()) { value = null; - } else { + } + else { switch (columns.get(slot).getInfluxType()) { case "string": value = node.textValue(); @@ -79,12 +100,14 @@ public InfluxRecordSet(List columns, JsonNode results) { } @Override - public List getColumnTypes() { + public List getColumnTypes() + { return columnTypes; } @Override - public RecordCursor cursor() { + public RecordCursor cursor() + { return new InfluxRecordCursor(columns, rows); } } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSetProvider.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSetProvider.java index d156619d62fe..5b0c298a84fd 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSetProvider.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSetProvider.java @@ -1,29 +1,54 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.prestosql.plugin.influx; import com.fasterxml.jackson.databind.JsonNode; import com.google.common.collect.ImmutableList; -import io.prestosql.spi.connector.*; +import io.prestosql.spi.connector.ColumnHandle; +import io.prestosql.spi.connector.ConnectorRecordSetProvider; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.connector.ConnectorSplit; +import io.prestosql.spi.connector.ConnectorTableHandle; +import io.prestosql.spi.connector.ConnectorTransactionHandle; +import io.prestosql.spi.connector.RecordSet; import javax.inject.Inject; + import java.util.List; import static java.util.Objects.requireNonNull; -public class InfluxRecordSetProvider implements ConnectorRecordSetProvider { +public class InfluxRecordSetProvider + implements ConnectorRecordSetProvider +{ private final InfluxClient client; @Inject - public InfluxRecordSetProvider(InfluxClient client) { + public InfluxRecordSetProvider(InfluxClient client) + { this.client = requireNonNull(client, "client is null"); } @Override - public RecordSet getRecordSet(ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorSplit split, ConnectorTableHandle tableHandle, List columns) { + public RecordSet getRecordSet(ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorSplit split, ConnectorTableHandle tableHandle, List columns) + { InfluxTableHandle table = (InfluxTableHandle) tableHandle; client.logger.info("getRecordSet(" + split + ", " + table + ", " + columns + ")"); ImmutableList.Builder handles = ImmutableList.builder(); - for (ColumnHandle handle: columns) { + for (ColumnHandle handle : columns) { InfluxColumnHandle column = (InfluxColumnHandle) handle; InfluxError.GENERAL.check(column.getMeasurement().equals(table.getMeasurement()), "bad measurement for " + column + " in " + table); InfluxError.GENERAL.check(column.getRetentionPolicy().equals(table.getRetentionPolicy()), "bad retention-policy for " + column + " in " + table); diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxSplit.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxSplit.java index f93c47c961a7..89bddc6c11df 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxSplit.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxSplit.java @@ -1,3 +1,17 @@ +/* + * 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.annotation.JsonCreator; @@ -11,32 +25,37 @@ import static com.google.common.base.MoreObjects.toStringHelper; import static java.util.Objects.requireNonNull; -public class InfluxSplit implements ConnectorSplit { +public class InfluxSplit implements ConnectorSplit +{ private final HostAddress address; @JsonCreator - public InfluxSplit(@JsonProperty("host") String host, @JsonProperty("port") int port) { + public InfluxSplit(@JsonProperty("host") String host, @JsonProperty("port") int port) + { this.address = HostAddress.fromParts(requireNonNull(host, "host is null"), port); } - public InfluxSplit(HostAddress address) { + public InfluxSplit(HostAddress address) + { this.address = requireNonNull(address, "address is null"); } @JsonProperty - public String getHost() { + public String getHost() + { return address.getHostText(); } @JsonProperty - public int getPort() { + public int getPort() + { return address.getPort(); } - @Override - public boolean isRemotelyAccessible() { + public boolean isRemotelyAccessible() + { return true; } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxSplitManager.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxSplitManager.java index 87431051091d..35ce15a37af2 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxSplitManager.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxSplitManager.java @@ -1,11 +1,33 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package io.prestosql.plugin.influx; -import io.prestosql.spi.connector.*; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.connector.ConnectorSplitManager; +import io.prestosql.spi.connector.ConnectorSplitSource; +import io.prestosql.spi.connector.ConnectorTableHandle; +import io.prestosql.spi.connector.ConnectorTransactionHandle; +import io.prestosql.spi.connector.FixedSplitSource; import javax.inject.Inject; + import java.util.Collections; -public class InfluxSplitManager implements ConnectorSplitManager { +public class InfluxSplitManager + implements ConnectorSplitManager +{ private final InfluxClient client; diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTableHandle.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTableHandle.java index e0be0b700b69..9ea93ecef2da 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTableHandle.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTableHandle.java @@ -1,3 +1,17 @@ +/* + * 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.annotation.JsonCreator; @@ -7,7 +21,10 @@ import static java.util.Objects.requireNonNull; -public class InfluxTableHandle extends SchemaTableName implements ConnectorTableHandle { +public class InfluxTableHandle + extends SchemaTableName + implements ConnectorTableHandle +{ private final String retentionPolicy; private final String measurement; @@ -16,9 +33,10 @@ public class InfluxTableHandle extends SchemaTableName implements ConnectorTable @JsonCreator public InfluxTableHandle(@JsonProperty("retentionPolicy") String retentionPolicy, - @JsonProperty("measurement") String measurement, - @JsonProperty("where") InfluxQL where, - @JsonProperty("limit") Long limit) { + @JsonProperty("measurement") String measurement, + @JsonProperty("where") InfluxQL where, + @JsonProperty("limit") Long limit) + { super(retentionPolicy, measurement); this.retentionPolicy = requireNonNull(retentionPolicy, "retentionPolicy is null"); this.measurement = requireNonNull(measurement, "measurement is null"); @@ -26,31 +44,37 @@ public InfluxTableHandle(@JsonProperty("retentionPolicy") String retentionPolicy this.limit = limit; } - public InfluxTableHandle(String retentionPolicy, String measurement) { + public InfluxTableHandle(String retentionPolicy, String measurement) + { this(retentionPolicy, measurement, new InfluxQL(), null); } @JsonProperty - public String getRetentionPolicy() { + public String getRetentionPolicy() + { return retentionPolicy; } @JsonProperty - public String getMeasurement() { + public String getMeasurement() + { return measurement; } @JsonProperty - public InfluxQL getWhere() { + public InfluxQL getWhere() + { return where; } @JsonProperty - public Long getLimit() { + public Long getLimit() + { return limit; } - public InfluxQL getFromWhere() { + public InfluxQL getFromWhere() + { InfluxQL from = new InfluxQL("FROM ") .addIdentifier(getRetentionPolicy()).append('.').addIdentifier(getMeasurement()); if (!getWhere().isEmpty()) { @@ -63,7 +87,8 @@ public InfluxQL getFromWhere() { } @Override - public String toString() { + public String toString() + { return getFromWhere().toString(); } } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTransactionHandle.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTransactionHandle.java index 4f43562d877f..d1edcd638edd 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTransactionHandle.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTransactionHandle.java @@ -1,8 +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.prestosql.plugin.influx; import io.prestosql.spi.connector.ConnectorTransactionHandle; -public enum InfluxTransactionHandle implements ConnectorTransactionHandle +public enum InfluxTransactionHandle + implements ConnectorTransactionHandle { INSTANCE } From 7ed872054e77e204ad9b766e5b4fffc5029feb84 Mon Sep 17 00:00:00 2001 From: Will Edwards Date: Fri, 20 Dec 2019 21:10:55 +0100 Subject: [PATCH 07/19] Fighting the style checkers --- presto-influx/pom.xml | 14 +++ .../prestosql/plugin/influx/InfluxClient.java | 90 +++++++++---------- .../prestosql/plugin/influx/InfluxColumn.java | 42 +++++---- .../plugin/influx/InfluxColumnHandle.java | 23 +++-- .../prestosql/plugin/influx/InfluxConfig.java | 1 - .../plugin/influx/InfluxConnector.java | 9 +- .../plugin/influx/InfluxConnectorFactory.java | 15 ++-- .../prestosql/plugin/influx/InfluxError.java | 33 ++++--- .../plugin/influx/InfluxHandleResolver.java | 2 +- .../plugin/influx/InfluxMetadata.java | 21 +++-- .../prestosql/plugin/influx/InfluxModule.java | 3 +- .../prestosql/plugin/influx/InfluxPlugin.java | 3 +- .../io/prestosql/plugin/influx/InfluxQL.java | 3 +- .../plugin/influx/InfluxRecordCursor.java | 3 +- .../plugin/influx/InfluxRecordSet.java | 9 +- .../influx/InfluxRecordSetProvider.java | 3 +- .../prestosql/plugin/influx/InfluxSplit.java | 8 +- .../plugin/influx/InfluxSplitManager.java | 3 +- .../plugin/influx/InfluxTableHandle.java | 13 ++- .../influx/InfluxTransactionHandle.java | 2 +- 20 files changed, 152 insertions(+), 148 deletions(-) diff --git a/presto-influx/pom.xml b/presto-influx/pom.xml index b6f4bfa46e89..f82084250abe 100644 --- a/presto-influx/pom.xml +++ b/presto-influx/pom.xml @@ -14,6 +14,10 @@ Presto - Influx Connector presto-plugin + + ${project.parent.basedir} + + io.airlift @@ -30,6 +34,11 @@ json + + io.airlift + log + + com.google.guava guava @@ -45,6 +54,11 @@ javax.inject + + javax.validation + validation-api + + com.fasterxml.jackson.core jackson-databind diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java index e14b10cf7333..a15a045deaf1 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java @@ -31,6 +31,7 @@ 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; @@ -39,7 +40,6 @@ public class InfluxClient { - final Logger logger; private final InfluxConfig config; // the various metadata are cached for a configurable number of milliseconds so we don't hammer the server @@ -82,55 +82,54 @@ public String getMeasurement(String tableName) private Map getTags(String tableName) { return tagKeys.computeIfAbsent(tableName, - 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 tags = new ImmutableMap.Builder<>(); - for (Map.Entry name : showNames(query).entrySet()) { - tags.put(name.getKey(), new InfluxColumn(name.getValue(), "string", InfluxColumn.Kind.TAG)); - } - return tags.build(); - } - )) - .get(); + 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 tags = new ImmutableMap.Builder<>(); + for (Map.Entry name : showNames(query).entrySet()) { + tags.put(name.getKey(), new InfluxColumn(name.getValue(), "string", InfluxColumn.Kind.TAG)); + } + return tags.build(); + })) + .get(); } private Map 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 fields = new HashMap<>(); - for (JsonNode series : execute(query)) { - 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(), new InfluxColumn(name, influxType, InfluxColumn.Kind.FIELD)); - if (collision != null) { - InfluxError.IDENTIFIER_CASE_SENSITIVITY.fail("identifier " + name + " collides with " + collision.getInfluxName(), query); + 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 fields = new HashMap<>(); + for (JsonNode series : execute(query)) { + 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(); + return ImmutableMap.copyOf(fields); + })) + .get(); } public boolean tableExistsInSchema(String schemaName, String tableName) @@ -161,7 +160,7 @@ private Map showNames(String query) if (series.has("values")) { for (JsonNode row : series.get("values")) { String name = row.get(0).textValue(); - String collision = names.put(name.toLowerCase(), name); + String collision = names.put(name.toLowerCase(Locale.ENGLISH), name); if (collision != null) { InfluxError.IDENTIFIER_CASE_SENSITIVITY.fail("identifier " + name + " collides with " + collision, query); } @@ -176,7 +175,7 @@ JsonNode execute(String query) final JsonNode response; try { URL url = new URL("http://" + config.getUserName() + ":" + config.getPassword() + "@" + config.getHost() + ":" + config.getPort() + - "/query?db=" + config.getDatabase() + "&q=" + URLEncoder.encode(query, StandardCharsets.UTF_8.toString())); + "/query?db=" + config.getDatabase() + "&q=" + URLEncoder.encode(query, StandardCharsets.UTF_8.toString())); response = new ObjectMapper().readTree(url); } catch (Throwable t) { @@ -199,7 +198,6 @@ public HostAddress getHostAddress() private class CachedMetaData { - private final Supplier loader; private T value; private long lastLoaded; diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumn.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumn.java index 4b55a7a129a9..ce2fa927a498 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumn.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumn.java @@ -16,7 +16,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.MoreObjects; import com.google.common.collect.ImmutableMap; import io.prestosql.spi.connector.ColumnMetadata; import io.prestosql.spi.type.BigintType; @@ -26,29 +25,36 @@ import io.prestosql.spi.type.Type; import io.prestosql.spi.type.VarcharType; +import java.util.Locale; + +import static com.google.common.base.MoreObjects.ToStringHelper; + public class InfluxColumn - extends ColumnMetadata + extends ColumnMetadata { - public static final InfluxColumn TIME = new InfluxColumn("time", "time", Kind.TIME); // map InfluxDB types to Presto types - private final static ImmutableMap TYPES_MAPPING = new ImmutableMap.Builder() - .put("string", VarcharType.VARCHAR) - .put("boolean", BooleanType.BOOLEAN) - .put("integer", BigintType.BIGINT) - .put("float", DoubleType.DOUBLE) - .put("time", TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE) - .build(); + private static final ImmutableMap TYPES_MAPPING = new ImmutableMap.Builder() + .put("string", VarcharType.VARCHAR) + .put("boolean", BooleanType.BOOLEAN) + .put("integer", BigintType.BIGINT) + .put("float", DoubleType.DOUBLE) + .put("time", TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE) + .build(); private final String influxName; private final String influxType; private final Kind kind; @JsonCreator public InfluxColumn(@JsonProperty("influxName") String influxName, - @JsonProperty("influxType") String influxType, - @JsonProperty("kind") Kind kind) + @JsonProperty("influxType") String influxType, + @JsonProperty("kind") Kind kind) { - super(influxName.toLowerCase(), TYPES_MAPPING.get(influxType), null, kind.name().toLowerCase(), false); + super(influxName.toLowerCase(Locale.ENGLISH), + TYPES_MAPPING.get(influxType), + null, + kind.name().toLowerCase(Locale.ENGLISH), + false); this.influxName = influxName; this.influxType = influxType; this.kind = kind; @@ -72,12 +78,12 @@ public Kind getKind() return kind; } - protected MoreObjects.ToStringHelper toStringHelper(Object self) + protected ToStringHelper toStringHelper(Object self) { - MoreObjects.ToStringHelper helper = com.google.common.base.MoreObjects.toStringHelper(self) - .addValue(getName()) - .addValue(getType()) - .addValue(kind); + ToStringHelper helper = com.google.common.base.MoreObjects.toStringHelper(self) + .addValue(getName()) + .addValue(getType()) + .addValue(kind); if (!getName().equals(getInfluxName())) { helper.add("influx-name", getInfluxName()); } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumnHandle.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumnHandle.java index 7fab78cbd6b0..677ce46ae88c 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumnHandle.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumnHandle.java @@ -19,19 +19,18 @@ import io.prestosql.spi.connector.ColumnHandle; public class InfluxColumnHandle - extends InfluxColumn - implements ColumnHandle + extends InfluxColumn + implements ColumnHandle { - private final String retentionPolicy; private final String measurement; @JsonCreator public InfluxColumnHandle(@JsonProperty("retentionPolicy") String retentionPolicy, - @JsonProperty("measurement") String measurement, - @JsonProperty("influxName") String influxName, - @JsonProperty("influxType") String influxType, - @JsonProperty("kind") Kind kind) + @JsonProperty("measurement") String measurement, + @JsonProperty("influxName") String influxName, + @JsonProperty("influxType") String influxType, + @JsonProperty("kind") Kind kind) { super(influxName, influxType, kind); this.retentionPolicy = retentionPolicy; @@ -39,8 +38,8 @@ public InfluxColumnHandle(@JsonProperty("retentionPolicy") String retentionPolic } public InfluxColumnHandle(String retentionPolicy, - String measurement, - InfluxColumn column) + String measurement, + InfluxColumn column) { this(retentionPolicy, measurement, column.getInfluxName(), column.getInfluxType(), column.getKind()); } @@ -61,8 +60,8 @@ public String getMeasurement() public String toString() { return toStringHelper(this) - .addValue(getRetentionPolicy()) - .addValue(getMeasurement()) - .toString(); + .addValue(getRetentionPolicy()) + .addValue(getMeasurement()) + .toString(); } } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConfig.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConfig.java index 95b6cdd59e57..20befb4860ea 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConfig.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConfig.java @@ -20,7 +20,6 @@ public class InfluxConfig { - private long cacheMetaDataMillis = 10000; private String host = "localhost"; private int port = 8086; diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConnector.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConnector.java index 929bf9e0f3e4..581a65c2ed26 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConnector.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConnector.java @@ -29,9 +29,8 @@ import static io.prestosql.spi.transaction.IsolationLevel.checkConnectorSupports; public class InfluxConnector - implements Connector + implements Connector { - private final LifeCycleManager lifeCycleManager; private final InfluxMetadata metadata; private final InfluxSplitManager splitManager; @@ -39,9 +38,9 @@ public class InfluxConnector @Inject public InfluxConnector(LifeCycleManager lifeCycleManager, - InfluxMetadata metadata, - InfluxSplitManager splitManager, - InfluxRecordSetProvider recordSetProvider) + InfluxMetadata metadata, + InfluxSplitManager splitManager, + InfluxRecordSetProvider recordSetProvider) { this.lifeCycleManager = lifeCycleManager; this.metadata = metadata; diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConnectorFactory.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConnectorFactory.java index 0f5e00c1a6cf..4229a0b9114e 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConnectorFactory.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConnectorFactory.java @@ -27,9 +27,8 @@ import static java.util.Objects.requireNonNull; public class InfluxConnectorFactory - implements ConnectorFactory + implements ConnectorFactory { - @Override public String getName() { @@ -48,14 +47,14 @@ public Connector create(String catalogName, Map config, Connecto requireNonNull(config, "config is null"); Bootstrap app = new Bootstrap( - new JsonModule(), - new InfluxModule()); + new JsonModule(), + new InfluxModule()); Injector injector = app - .strictConfig() - .doNotInitializeLogging() - .setRequiredConfigurationProperties(config) - .initialize(); + .strictConfig() + .doNotInitializeLogging() + .setRequiredConfigurationProperties(config) + .initialize(); return injector.getInstance(InfluxConnector.class); } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxError.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxError.java index a8f1108d01eb..fe85bb7401ef 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxError.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxError.java @@ -19,12 +19,13 @@ import io.prestosql.spi.ErrorType; import io.prestosql.spi.PrestoException; +import java.util.Arrays; + import static com.google.common.base.MoreObjects.toStringHelper; public enum InfluxError - implements ErrorCodeSupplier + implements ErrorCodeSupplier { - GENERAL(ErrorType.INTERNAL_ERROR), EXTERNAL(ErrorType.EXTERNAL), IDENTIFIER_CASE_SENSITIVITY(ErrorType.EXTERNAL), @@ -38,31 +39,29 @@ public enum InfluxError this.errorCode = new ErrorCode(ERROR_BASE + ordinal(), name(), type); } - public void check(boolean condition, String message, String context) + public void check(boolean condition, String message, Object... context) { if (!condition) { - fail(message, context); + fail(message, getContextString(context)); } } - public void check(boolean condition, String message) - { - check(condition, message, null); - } - - public void fail(String message, String context) + public void fail(String message, Object... context) { - throw new PrestoException(this, message + (context != null && !context.isEmpty() ? " " + context : "")); + throw new PrestoException(this, message + getContextString(context)); } - public void fail(String message) + public void fail(Throwable t) { - fail(message, null); + throw new PrestoException(this, t); } - public void fail(Throwable t) + private static String getContextString(Object[] context) { - throw new PrestoException(this, t); + if (context == null || context.length == 0) { + return ""; + } + return " " + String.join(" ", Arrays.stream(context).toString()); } @Override @@ -75,7 +74,7 @@ public ErrorCode toErrorCode() public String toString() { return toStringHelper(this) - .add("code", errorCode) - .toString(); + .add("code", errorCode) + .toString(); } } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxHandleResolver.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxHandleResolver.java index eea80146d3e6..3df42815b07b 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxHandleResolver.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxHandleResolver.java @@ -21,7 +21,7 @@ import io.prestosql.spi.connector.ConnectorTransactionHandle; public class InfluxHandleResolver - implements ConnectorHandleResolver + implements ConnectorHandleResolver { @Override public Class getTableHandleClass() diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java index 77ab461ed71a..ca4ddb3af14d 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java @@ -48,9 +48,8 @@ import static java.util.Objects.requireNonNull; public class InfluxMetadata - implements ConnectorMetadata + implements ConnectorMetadata { - private final InfluxClient client; @Inject @@ -170,10 +169,10 @@ public Optional> applyLimit(Connect { InfluxTableHandle table = (InfluxTableHandle) handle; return Optional.of(new LimitApplicationResult<>(new InfluxTableHandle( - table.getRetentionPolicy(), - table.getMeasurement(), - table.getWhere(), - limit), true)); + table.getRetentionPolicy(), + table.getMeasurement(), + table.getWhere(), + limit), true)); } @Override @@ -256,7 +255,7 @@ else if (values instanceof EquatableValueSet) { boolean first = true; for (Object value : values.getDiscreteValues().getValues()) { where.append(first ? where.isEmpty() ? "WHERE (" : " AND (" : " OR ") - .add(column).append(" = ").add(value); + .add(column).append(" = ").add(value); first = false; } if (first) { @@ -275,9 +274,9 @@ else if (values instanceof EquatableValueSet) { client.logger.debug("applyFilter(" + handle + ", " + constraint.getSummary().toString(session) + ") = " + all + ", " + where); InfluxTableHandle table = (InfluxTableHandle) handle; return Optional.of(new ConstraintApplicationResult<>(new InfluxTableHandle( - table.getRetentionPolicy(), - table.getMeasurement(), - where, - table.getLimit()), all ? TupleDomain.all() : constraint.getSummary())); + table.getRetentionPolicy(), + table.getMeasurement(), + where, + table.getLimit()), all ? TupleDomain.all() : constraint.getSummary())); } } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxModule.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxModule.java index 44f1ac1d4735..dcd99fee6a39 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxModule.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxModule.java @@ -21,9 +21,8 @@ import static io.airlift.configuration.ConfigBinder.configBinder; public class InfluxModule - implements Module + implements Module { - @Override public void configure(Binder binder) { diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxPlugin.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxPlugin.java index e00e16a9a6f9..5da9d8182476 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxPlugin.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxPlugin.java @@ -19,9 +19,8 @@ import io.prestosql.spi.connector.ConnectorFactory; public class InfluxPlugin - implements Plugin + implements Plugin { - @Override public Iterable getConnectorFactories() { diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java index e5d83b94e77c..6525e75a7d20 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java @@ -20,7 +20,6 @@ public class InfluxQL { - private final StringBuilder influxQL; public InfluxQL() @@ -72,7 +71,7 @@ public InfluxQL add(InfluxColumn column) public InfluxQL add(Object value) { - assert !(value instanceof InfluxColumn) : value; + InfluxError.GENERAL.check(!(value instanceof InfluxColumn), "value cannot be a column", value); if (value == null) { influxQL.append("null"); } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordCursor.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordCursor.java index 39fd4a209100..64c043a62980 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordCursor.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordCursor.java @@ -23,9 +23,8 @@ import java.util.Objects; public class InfluxRecordCursor - implements RecordCursor + implements RecordCursor { - private final List columns; private final List rows; private Object[] row; diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java index c6b88d0da04f..dc63d4fae50c 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java @@ -29,9 +29,8 @@ import java.util.Map; public class InfluxRecordSet - implements RecordSet + implements RecordSet { - private final List columns; private final List columnTypes; private final List rows; @@ -47,7 +46,7 @@ public InfluxRecordSet(List columns, JsonNode results) } this.columnTypes = columnTypes.build(); this.rows = new ArrayList<>(); - final int IGNORE = -1; + final int ignore = -1; for (JsonNode series : results) { if (!series.has("values")) { continue; @@ -56,13 +55,13 @@ public InfluxRecordSet(List columns, JsonNode results) JsonNode header = series.get("columns"); int[] fields = new int[header.size()]; for (int i = 0; i < fields.length; i++) { - fields[i] = mapping.getOrDefault(header.get(i).textValue(), IGNORE); + fields[i] = mapping.getOrDefault(header.get(i).textValue(), ignore); } for (JsonNode values : series.get("values")) { Object[] row = new Object[columns.size()]; for (int i = 0; i < fields.length; i++) { int slot = fields[i]; - if (slot != IGNORE) { + if (slot != ignore) { final Object value; JsonNode node = values.get(i); if (node.isNull()) { diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSetProvider.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSetProvider.java index 5b0c298a84fd..0ab07197e9f5 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSetProvider.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSetProvider.java @@ -31,9 +31,8 @@ import static java.util.Objects.requireNonNull; public class InfluxRecordSetProvider - implements ConnectorRecordSetProvider + implements ConnectorRecordSetProvider { - private final InfluxClient client; @Inject diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxSplit.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxSplit.java index 89bddc6c11df..52f2d03fdac4 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxSplit.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxSplit.java @@ -25,9 +25,9 @@ import static com.google.common.base.MoreObjects.toStringHelper; import static java.util.Objects.requireNonNull; -public class InfluxSplit implements ConnectorSplit +public class InfluxSplit + implements ConnectorSplit { - private final HostAddress address; @JsonCreator @@ -75,7 +75,7 @@ public Object getInfo() public String toString() { return toStringHelper(this) - .add("host", address) - .toString(); + .add("host", address) + .toString(); } } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxSplitManager.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxSplitManager.java index 35ce15a37af2..66378e86b014 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxSplitManager.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxSplitManager.java @@ -26,9 +26,8 @@ import java.util.Collections; public class InfluxSplitManager - implements ConnectorSplitManager + implements ConnectorSplitManager { - private final InfluxClient client; @Inject diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTableHandle.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTableHandle.java index 9ea93ecef2da..88cf41a411f1 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTableHandle.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTableHandle.java @@ -22,10 +22,9 @@ import static java.util.Objects.requireNonNull; public class InfluxTableHandle - extends SchemaTableName - implements ConnectorTableHandle + extends SchemaTableName + implements ConnectorTableHandle { - private final String retentionPolicy; private final String measurement; private final InfluxQL where; @@ -33,9 +32,9 @@ public class InfluxTableHandle @JsonCreator public InfluxTableHandle(@JsonProperty("retentionPolicy") String retentionPolicy, - @JsonProperty("measurement") String measurement, - @JsonProperty("where") InfluxQL where, - @JsonProperty("limit") Long limit) + @JsonProperty("measurement") String measurement, + @JsonProperty("where") InfluxQL where, + @JsonProperty("limit") Long limit) { super(retentionPolicy, measurement); this.retentionPolicy = requireNonNull(retentionPolicy, "retentionPolicy is null"); @@ -76,7 +75,7 @@ public Long getLimit() public InfluxQL getFromWhere() { InfluxQL from = new InfluxQL("FROM ") - .addIdentifier(getRetentionPolicy()).append('.').addIdentifier(getMeasurement()); + .addIdentifier(getRetentionPolicy()).append('.').addIdentifier(getMeasurement()); if (!getWhere().isEmpty()) { from.append(' ').append(getWhere().toString()); } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTransactionHandle.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTransactionHandle.java index d1edcd638edd..5e408b9cf5ca 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTransactionHandle.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTransactionHandle.java @@ -17,7 +17,7 @@ import io.prestosql.spi.connector.ConnectorTransactionHandle; public enum InfluxTransactionHandle - implements ConnectorTransactionHandle + implements ConnectorTransactionHandle { INSTANCE } From b4a89c8882a653e1019e5bd4ed753d5ac26ef174 Mon Sep 17 00:00:00 2001 From: Will Edwards Date: Mon, 30 Dec 2019 07:17:40 +0100 Subject: [PATCH 08/19] If a column filter cannot be handled completely, don't handle it at all --- .../plugin/influx/InfluxMetadata.java | 28 +++++++++++++------ .../io/prestosql/plugin/influx/InfluxQL.java | 11 ++++++++ 2 files changed, 30 insertions(+), 9 deletions(-) diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java index ca4ddb3af14d..9e481392dc7e 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java @@ -181,6 +181,8 @@ public Optional> applyFilter(C boolean all = true; InfluxQL where = new InfluxQL(); for (Map.Entry predicate : constraint.getSummary().getDomains().orElse(Collections.emptyMap()).entrySet()) { + int startPos = where.getPos(); + boolean ok = true; // can we handle this column? InfluxColumnHandle column = (InfluxColumnHandle) predicate.getKey(); ValueSet values = predicate.getValue().getValues(); if (values instanceof SortedRangeSet) { @@ -188,8 +190,8 @@ public Optional> applyFilter(C for (Range range : values.getRanges().getOrderedRanges()) { if (!range.isSingleValue() && !range.getLow().getValueBlock().isPresent() && !range.getHigh().getValueBlock().isPresent()) { // can't do an IS NULL - client.logger.debug("can't tackle range " + column + ": " + range.toString(session)); - all = false; + client.logger.warn("unhandled range " + column + ": " + range.toString(session)); + ok = false; continue; } where.append(first ? where.isEmpty() ? "WHERE ((" : " AND ((" : ") OR ("); @@ -208,7 +210,8 @@ public Optional> applyFilter(C low = " > "; break; default: - InfluxError.GENERAL.fail("bad low bound", range.toString(session)); + client.logger.warn("unhandled low bound " + column + ": " + range.toString(session)); + ok = false; continue; } Object value = range.getLow().getValue(); @@ -228,7 +231,8 @@ public Optional> applyFilter(C high = " < "; break; default: - InfluxError.GENERAL.fail("bad high bound", range.toString(session)); + client.logger.warn("unhandled high bound " + column + ": " + range.toString(session)); + ok = false; continue; } if (hasLow) { @@ -244,8 +248,8 @@ public Optional> applyFilter(C first = false; } if (first) { - client.logger.warn("unhandled SortedRangeSet " + column + ":" + values.getClass().getName() + "=" + values.toString(session)); - all = false; + client.logger.warn("unhandled SortedRangeSet " + column + ": " + values.getClass().getName() + "=" + values.toString(session)); + ok = false; } else { where.append("))"); @@ -259,15 +263,21 @@ else if (values instanceof EquatableValueSet) { first = false; } if (first) { - client.logger.warn("unhandled EquatableValueSet " + column + ":" + values.getClass().getName() + "=" + values.toString(session)); - all = false; + client.logger.warn("unhandled EquatableValueSet " + column + ": " + values.getClass().getName() + "=" + values.toString(session)); + ok = false; } else { where.append(')'); } } else { - client.logger.warn("unhandled predicate " + column + ":" + values.getClass().getName() + "=" + values.toString(session)); + client.logger.warn("unhandled predicate " + column + ": " + values.getClass().getName() + "=" + values.toString(session)); + ok = false; + } + if (!ok) { + // undo everything we did add to the where-clause + where.truncate(startPos); + // and tell Presto we couldn't handle all the filtering all = false; } } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java index 6525e75a7d20..e8c8f365afa2 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java @@ -63,6 +63,17 @@ public InfluxQL append(int i) return this; } + public int getPos() + { + return influxQL.length(); + } + + public void truncate(int pos) + { + InfluxError.GENERAL.check(influxQL.length() >= pos, "bad truncation (" + pos + " > " + influxQL.length() + ")", influxQL); + influxQL.setLength(pos); + } + public InfluxQL add(InfluxColumn column) { addIdentifier(column.getInfluxName()); From 4edc8d4d3d2bda76f3f5d9fdb3a212e9cc013d76 Mon Sep 17 00:00:00 2001 From: Will Edwards Date: Fri, 3 Jan 2020 00:54:00 +0100 Subject: [PATCH 09/19] Some comments; don't generate comparisons for strings --- .gitignore | 1 + .../src/main/sphinx/connector/influx.rst | 43 ++++++++++ .../prestosql/plugin/influx/InfluxClient.java | 45 ++++++---- .../prestosql/plugin/influx/InfluxColumn.java | 3 +- .../prestosql/plugin/influx/InfluxConfig.java | 25 ++++-- .../prestosql/plugin/influx/InfluxError.java | 4 +- .../plugin/influx/InfluxMetadata.java | 77 ++++++++++++----- .../io/prestosql/plugin/influx/InfluxQL.java | 2 + .../plugin/influx/InfluxRecordSet.java | 86 ++++++++++--------- .../influx/InfluxRecordSetProvider.java | 4 +- 10 files changed, 198 insertions(+), 92 deletions(-) create mode 100644 presto-docs/src/main/sphinx/connector/influx.rst diff --git a/.gitignore b/.gitignore index f9b1a4b5901c..bdd19a780340 100644 --- a/.gitignore +++ b/.gitignore @@ -18,6 +18,7 @@ temp-testng-customsuite.xml test-output .externalToolBuilders *~ +\#* benchmark_outputs *.pyc *.class diff --git a/presto-docs/src/main/sphinx/connector/influx.rst b/presto-docs/src/main/sphinx/connector/influx.rst new file mode 100644 index 000000000000..802943a60055 --- /dev/null +++ b/presto-docs/src/main/sphinx/connector/influx.rst @@ -0,0 +1,43 @@ +================ +Influx Connector +================ + +.. contents:: + :local: + :backlinks: none + :depth: 1 + +Overview +-------- + +The Influx connector allows querying data-points stored in an +`InfluxDB `_ +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 \ No newline at end of file diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java index a15a045deaf1..f9edf599407d 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java @@ -22,6 +22,7 @@ import io.airlift.log.Logger; import io.prestosql.spi.HostAddress; +import javax.annotation.CheckForNull; import javax.inject.Inject; import java.net.URL; @@ -43,10 +44,10 @@ public class InfluxClient final Logger logger; private final InfluxConfig config; // the various metadata are cached for a configurable number of milliseconds so we don't hammer the server - private final CachedMetaData> retentionPolicies; - private final CachedMetaData> measurements; - private final Map>> tagKeys; - private final Map>>> fields; + private final CachedMetaData> retentionPolicies; // schema name (lower-case) -> retention policy (case-sensitive) + private final CachedMetaData> measurements; // table name (lower-case) -> measurement (case-sensitive) + private final Map>> tagKeys; // column name (lower-case) -> tags + private final Map>>> fields; // column name (lower-case) -> measurement -> fields @Inject public InfluxClient(InfluxConfig config) @@ -79,6 +80,7 @@ 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 getTags(String tableName) { return tagKeys.computeIfAbsent(tableName, @@ -115,14 +117,17 @@ private Map getFields(String schemaName, String tableName) .addIdentifier(measurement) .toString(); Map fields = new HashMap<>(); - for (JsonNode series : execute(query)) { - 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); + 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); + } } } } @@ -154,8 +159,8 @@ private Map showNames(String query) { Map names = new HashMap<>(); JsonNode series = execute(query); - InfluxError.GENERAL.check(series.getNodeType().equals(JsonNodeType.ARRAY), "expecting an array, not " + series, query); - InfluxError.GENERAL.check(series.size() == 1, "expecting one element, not " + series, 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")) { @@ -169,13 +174,17 @@ private Map showNames(String query) return ImmutableMap.copyOf(names); } + /* Using raw HTTP because the influx java library has dependency conflicts and puts the burden of quoting identifiers on the caller */ + @CheckForNull JsonNode execute(String query) { - logger.debug("executing: " + query); final JsonNode response; try { - URL url = new URL("http://" + config.getUserName() + ":" + config.getPassword() + "@" + config.getHost() + ":" + config.getPort() + - "/query?db=" + config.getDatabase() + "&q=" + URLEncoder.encode(query, StandardCharsets.UTF_8.toString())); + URL url = new URL((config.isUseHttps() ? "https://" : "http://") + + config.getUserName() + ":" + config.getPassword() + "@" + + config.getHost() + ":" + config.getPort() + + "/query?db=" + config.getDatabase() + + "&q=" + URLEncoder.encode(query, StandardCharsets.UTF_8.toString())); response = new ObjectMapper().readTree(url); } catch (Throwable t) { @@ -183,7 +192,7 @@ JsonNode execute(String query) return null; } JsonNode results = response.get("results"); - InfluxError.GENERAL.check(results.size() == 1, "expecting one result", query); + InfluxError.GENERAL.check(results != null && results.size() == 1, "expecting one result", query); JsonNode result = results.get(0); if (result.has("error")) { InfluxError.GENERAL.fail(result.get("error").asText(), query); diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumn.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumn.java index ce2fa927a498..263d3ff1a3b1 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumn.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumn.java @@ -32,7 +32,6 @@ public class InfluxColumn extends ColumnMetadata { - public static final InfluxColumn TIME = new InfluxColumn("time", "time", Kind.TIME); // map InfluxDB types to Presto types private static final ImmutableMap TYPES_MAPPING = new ImmutableMap.Builder() .put("string", VarcharType.VARCHAR) @@ -41,6 +40,8 @@ public class InfluxColumn .put("float", DoubleType.DOUBLE) .put("time", TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE) .build(); + public static final InfluxColumn TIME = new InfluxColumn("time", "time", Kind.TIME); + private final String influxName; private final String influxType; private final Kind kind; diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConfig.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConfig.java index 20befb4860ea..8fafe3a8754e 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConfig.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConfig.java @@ -26,6 +26,7 @@ public class InfluxConfig private String database; private String userName; private String password; + private boolean useHttps; @NotNull public long getCacheMetaDataMillis() @@ -33,7 +34,7 @@ public long getCacheMetaDataMillis() return cacheMetaDataMillis; } - @Config("cache-meta-data-millis") + @Config("influx.cache-meta-data-millis") public InfluxConfig setCacheMetaDataMillis(long cacheMetaDataMillis) { this.cacheMetaDataMillis = cacheMetaDataMillis; @@ -46,7 +47,7 @@ public String getHost() return host; } - @Config("host") + @Config("influx.host") public InfluxConfig setHost(String host) { this.host = host; @@ -58,7 +59,7 @@ public int getPort() return port; } - @Config("port") + @Config("influx.port") public InfluxConfig setPort(int port) { this.port = port; @@ -71,7 +72,7 @@ public String getDatabase() return database; } - @Config("database") + @Config("influx.database") public InfluxConfig setDatabase(String database) { this.database = database; @@ -84,7 +85,7 @@ public String getUserName() return userName; } - @Config("user") + @Config("influx.username") public InfluxConfig setUserName(String userName) { this.userName = userName; @@ -97,10 +98,22 @@ public String getPassword() return password; } - @Config("password") + @Config("influx.password") public InfluxConfig setPassword(String password) { this.password = password; return this; } + + public boolean isUseHttps() + { + return useHttps; + } + + @Config("influx.use-https") + public InfluxConfig setUseHttps(boolean useHttps) + { + this.useHttps = useHttps; + return this; + } } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxError.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxError.java index fe85bb7401ef..7fca8f3ad1a1 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxError.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxError.java @@ -20,6 +20,8 @@ import io.prestosql.spi.PrestoException; import java.util.Arrays; +import java.util.Objects; +import java.util.stream.Collectors; import static com.google.common.base.MoreObjects.toStringHelper; @@ -61,7 +63,7 @@ private static String getContextString(Object[] context) if (context == null || context.length == 0) { return ""; } - return " " + String.join(" ", Arrays.stream(context).toString()); + return " " + Arrays.stream(context).map(Objects::toString).collect(Collectors.joining(" ")); } @Override diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java index 9e481392dc7e..9fde7e897eef 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java @@ -44,6 +44,8 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Consumer; import static java.util.Objects.requireNonNull; @@ -182,21 +184,35 @@ public Optional> applyFilter(C InfluxQL where = new InfluxQL(); for (Map.Entry predicate : constraint.getSummary().getDomains().orElse(Collections.emptyMap()).entrySet()) { int startPos = where.getPos(); - boolean ok = true; // can we handle this column? InfluxColumnHandle column = (InfluxColumnHandle) predicate.getKey(); ValueSet values = predicate.getValue().getValues(); + AtomicBoolean ok = new AtomicBoolean(true); // can we handle this column? + Consumer fail = error -> { + client.logger.debug("unhandled " + error + " " + column + ": " + values.toString(session)); + ok.set(false); + }; if (values instanceof SortedRangeSet) { boolean first = true; + ranges: for (Range range : values.getRanges().getOrderedRanges()) { if (!range.isSingleValue() && !range.getLow().getValueBlock().isPresent() && !range.getHigh().getValueBlock().isPresent()) { // can't do an IS NULL - client.logger.warn("unhandled range " + column + ": " + range.toString(session)); - ok = false; - continue; + fail.accept("range"); + break; } where.append(first ? where.isEmpty() ? "WHERE ((" : " AND ((" : ") OR ("); if (range.isSingleValue()) { - where.add(column).append(" = ").add(range.getSingleValue()); + Object value = range.getSingleValue(); + if (column.getKind() == InfluxColumn.Kind.TIME) { + if (value instanceof Long) { + value = Instant.ofEpochMilli(DateTimeEncoding.unpackMillisUtc((Long) value)).toString(); + } + else { + fail.accept("time"); + break; + } + } + where.add(column).append(" = ").add(value); } else { boolean hasLow = false; @@ -210,13 +226,22 @@ public Optional> applyFilter(C low = " > "; break; default: - client.logger.warn("unhandled low bound " + column + ": " + range.toString(session)); - ok = false; - continue; + fail.accept("low bound"); + break ranges; } Object value = range.getLow().getValue(); - if (column.getKind() == InfluxColumn.Kind.TIME && value instanceof Long) { - value = Instant.ofEpochMilli(DateTimeEncoding.unpackMillisUtc((Long) value)).toString(); + if (column.getKind() == InfluxColumn.Kind.TIME) { + if (value instanceof Long) { + value = Instant.ofEpochMilli(DateTimeEncoding.unpackMillisUtc((Long) value)).toString(); + } + else { + fail.accept("time low bound"); + break; + } + } + else if (!(value instanceof Number)) { + fail.accept("tag comparision low bound"); + break; } where.add(column).append(low).add(value); hasLow = true; @@ -231,25 +256,33 @@ public Optional> applyFilter(C high = " < "; break; default: - client.logger.warn("unhandled high bound " + column + ": " + range.toString(session)); - ok = false; - continue; + fail.accept("high bound"); + break ranges; } if (hasLow) { where.append(" AND "); } Object value = range.getHigh().getValue(); - if (column.getKind() == InfluxColumn.Kind.TIME && value instanceof Long) { - value = Instant.ofEpochMilli(DateTimeEncoding.unpackMillisUtc((Long) value)).toString(); + if (column.getKind() == InfluxColumn.Kind.TIME) { + if (value instanceof Long) { + value = Instant.ofEpochMilli(DateTimeEncoding.unpackMillisUtc((Long) value)).toString(); + } + else { + fail.accept("time high bound"); + break; + } + } + else if (!(value instanceof Number)) { + fail.accept("tag comparison high bound"); + break; } where.add(column).append(high).add(value); } } first = false; } - if (first) { - client.logger.warn("unhandled SortedRangeSet " + column + ": " + values.getClass().getName() + "=" + values.toString(session)); - ok = false; + if (ok.get() && first) { + fail.accept("SortedRangeSet"); } else { where.append("))"); @@ -263,18 +296,16 @@ else if (values instanceof EquatableValueSet) { first = false; } if (first) { - client.logger.warn("unhandled EquatableValueSet " + column + ": " + values.getClass().getName() + "=" + values.toString(session)); - ok = false; + fail.accept("EquatableValueSet"); } else { where.append(')'); } } else { - client.logger.warn("unhandled predicate " + column + ": " + values.getClass().getName() + "=" + values.toString(session)); - ok = false; + fail.accept("predicate"); } - if (!ok) { + if (!ok.get()) { // undo everything we did add to the where-clause where.truncate(startPos); // and tell Presto we couldn't handle all the filtering diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java index e8c8f365afa2..143c56e0e618 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java @@ -18,6 +18,8 @@ import com.fasterxml.jackson.annotation.JsonProperty; import io.airlift.slice.Slice; +/* A query builder that takes care correctly quoting identifiers and string values */ + public class InfluxQL { private final StringBuilder influxQL; diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java index dc63d4fae50c..12f287e34bd0 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java @@ -47,53 +47,55 @@ public InfluxRecordSet(List columns, JsonNode results) this.columnTypes = columnTypes.build(); this.rows = new ArrayList<>(); final int ignore = -1; - for (JsonNode series : results) { - if (!series.has("values")) { - continue; - } - // we can't push down group-bys so we have no tags to consider - JsonNode header = series.get("columns"); - int[] fields = new int[header.size()]; - for (int i = 0; i < fields.length; i++) { - fields[i] = mapping.getOrDefault(header.get(i).textValue(), ignore); - } - for (JsonNode values : series.get("values")) { - Object[] row = new Object[columns.size()]; + if (results != null) { + for (JsonNode series : results) { + if (!series.has("values")) { + continue; + } + // we can't push down group-bys so we have no tags to consider + JsonNode header = series.get("columns"); + int[] fields = new int[header.size()]; for (int i = 0; i < fields.length; i++) { - int slot = fields[i]; - if (slot != ignore) { - final Object value; - JsonNode node = values.get(i); - if (node.isNull()) { - value = null; - } - else { - switch (columns.get(slot).getInfluxType()) { - case "string": - value = node.textValue(); - break; - case "boolean": - value = node.booleanValue(); - break; - case "integer": - value = node.longValue(); - break; - case "float": - value = node.doubleValue(); - break; - case "time": - Instant timestamp = Instant.parse(node.textValue()); - value = DateTimeEncoding.packDateTimeWithZone(timestamp.toEpochMilli(), TimeZoneKey.UTC_KEY); - break; - default: - InfluxError.GENERAL.fail("cannot map " + node + " to " + columns.get(slot)); - value = null; + fields[i] = mapping.getOrDefault(header.get(i).textValue(), ignore); + } + for (JsonNode values : series.get("values")) { + Object[] row = new Object[columns.size()]; + for (int i = 0; i < fields.length; i++) { + int slot = fields[i]; + if (slot != ignore) { + final Object value; + JsonNode node = values.get(i); + if (node.isNull()) { + value = null; + } + else { + switch (columns.get(slot).getInfluxType()) { + case "string": + value = node.textValue(); + break; + case "boolean": + value = node.booleanValue(); + break; + case "integer": + value = node.longValue(); + break; + case "float": + value = node.doubleValue(); + break; + case "time": + Instant timestamp = Instant.parse(node.textValue()); + value = DateTimeEncoding.packDateTimeWithZone(timestamp.toEpochMilli(), TimeZoneKey.UTC_KEY); + break; + default: + InfluxError.GENERAL.fail("cannot map " + node + " to " + columns.get(slot)); + value = null; + } } + row[slot] = value; } - row[slot] = value; } + rows.add(row); } - rows.add(row); } } } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSetProvider.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSetProvider.java index 0ab07197e9f5..817ed771021c 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSetProvider.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSetProvider.java @@ -45,7 +45,6 @@ public InfluxRecordSetProvider(InfluxClient client) public RecordSet getRecordSet(ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorSplit split, ConnectorTableHandle tableHandle, List columns) { InfluxTableHandle table = (InfluxTableHandle) tableHandle; - client.logger.info("getRecordSet(" + split + ", " + table + ", " + columns + ")"); ImmutableList.Builder handles = ImmutableList.builder(); for (ColumnHandle handle : columns) { InfluxColumnHandle column = (InfluxColumnHandle) handle; @@ -53,6 +52,9 @@ public RecordSet getRecordSet(ConnectorTransactionHandle transaction, ConnectorS InfluxError.GENERAL.check(column.getRetentionPolicy().equals(table.getRetentionPolicy()), "bad retention-policy for " + column + " in " + table); handles.add(column); } + // we SELECT * as this returns tags, and returns nulls for unpopulated fields; + // if we ask for specific columns, e.g. SELECT a WHERE b = 1, we would be asking for SELECT a WHERE b = 1 AND a IS NOT NULL instead + // and there's no way to ask for tags without grouping String query = new InfluxQL("SELECT * ").append(table.getFromWhere()).toString(); JsonNode results = client.execute(query); // actually run the query against our Influx server return new InfluxRecordSet(handles.build(), results); From d761c8b308f8b47c75a26c278404a296fb5ad35b Mon Sep 17 00:00:00 2001 From: Will Edwards Date: Fri, 3 Jan 2020 01:40:11 +0100 Subject: [PATCH 10/19] Added some tests --- .../src/main/sphinx/connector/influx.rst | 2 +- .../prestosql/plugin/influx/InfluxClient.java | 9 +- .../prestosql/plugin/influx/InfluxConfig.java | 2 - .../plugin/influx/InfluxMetadata.java | 205 +++++++++--------- .../io/prestosql/plugin/influx/InfluxQL.java | 6 +- .../plugin/influx/InfluxPluginTest.java | 45 ++++ .../prestosql/plugin/influx/InfluxQLTest.java | 33 +++ 7 files changed, 190 insertions(+), 112 deletions(-) create mode 100644 presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxPluginTest.java create mode 100644 presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxQLTest.java diff --git a/presto-docs/src/main/sphinx/connector/influx.rst b/presto-docs/src/main/sphinx/connector/influx.rst index 802943a60055..8d893a8b158d 100644 --- a/presto-docs/src/main/sphinx/connector/influx.rst +++ b/presto-docs/src/main/sphinx/connector/influx.rst @@ -40,4 +40,4 @@ Limitations * 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 \ No newline at end of file +* LDAP on InfluxDB Enterprise editions is not supported diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java index f9edf599407d..4a2a3a3c457c 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java @@ -22,7 +22,6 @@ import io.airlift.log.Logger; import io.prestosql.spi.HostAddress; -import javax.annotation.CheckForNull; import javax.inject.Inject; import java.net.URL; @@ -175,13 +174,17 @@ private Map showNames(String query) } /* Using raw HTTP because the influx java library has dependency conflicts and puts the burden of quoting identifiers on the caller */ - @CheckForNull JsonNode execute(String query) { final JsonNode response; try { + String authentication = config.getUserName() != null ? + config.getUserName() + (config.getPassword() != null ? + ":" + config.getPassword() : + "") + "@" : + ""; URL url = new URL((config.isUseHttps() ? "https://" : "http://") - + config.getUserName() + ":" + config.getPassword() + "@" + + authentication + config.getHost() + ":" + config.getPort() + "/query?db=" + config.getDatabase() + "&q=" + URLEncoder.encode(query, StandardCharsets.UTF_8.toString())); diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConfig.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConfig.java index 8fafe3a8754e..93bf0a809497 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConfig.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConfig.java @@ -79,7 +79,6 @@ public InfluxConfig setDatabase(String database) return this; } - @NotNull public String getUserName() { return userName; @@ -92,7 +91,6 @@ public InfluxConfig setUserName(String userName) return this; } - @NotNull public String getPassword() { return password; diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java index 9fde7e897eef..47d8c7dd9a47 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java @@ -44,8 +44,6 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.function.Consumer; import static java.util.Objects.requireNonNull; @@ -186,126 +184,120 @@ public Optional> applyFilter(C int startPos = where.getPos(); InfluxColumnHandle column = (InfluxColumnHandle) predicate.getKey(); ValueSet values = predicate.getValue().getValues(); - AtomicBoolean ok = new AtomicBoolean(true); // can we handle this column? - Consumer fail = error -> { - client.logger.debug("unhandled " + error + " " + column + ": " + values.toString(session)); - ok.set(false); - }; - if (values instanceof SortedRangeSet) { - boolean first = true; - ranges: - for (Range range : values.getRanges().getOrderedRanges()) { - if (!range.isSingleValue() && !range.getLow().getValueBlock().isPresent() && !range.getHigh().getValueBlock().isPresent()) { - // can't do an IS NULL - fail.accept("range"); - break; - } - where.append(first ? where.isEmpty() ? "WHERE ((" : " AND ((" : ") OR ("); - if (range.isSingleValue()) { - Object value = range.getSingleValue(); - if (column.getKind() == InfluxColumn.Kind.TIME) { - if (value instanceof Long) { - value = Instant.ofEpochMilli(DateTimeEncoding.unpackMillisUtc((Long) value)).toString(); - } - else { - fail.accept("time"); - break; - } + try { + if (values instanceof SortedRangeSet) { + boolean first = true; + for (Range range : values.getRanges().getOrderedRanges()) { + if (!range.isSingleValue() && !range.getLow().getValueBlock().isPresent() && !range.getHigh().getValueBlock().isPresent()) { + throw new UnhandledFilterException("range"); } - where.add(column).append(" = ").add(value); - } - else { - boolean hasLow = false; - if (range.getLow().getValueBlock().isPresent()) { - final String low; - switch (range.getLow().getBound()) { - case EXACTLY: - low = " >= "; - break; - case ABOVE: - low = " > "; - break; - default: - fail.accept("low bound"); - break ranges; - } - Object value = range.getLow().getValue(); + where.append(first ? where.isEmpty() ? "WHERE ((" : " AND ((" : ") OR ("); + if (range.isSingleValue()) { + Object value = range.getSingleValue(); if (column.getKind() == InfluxColumn.Kind.TIME) { if (value instanceof Long) { value = Instant.ofEpochMilli(DateTimeEncoding.unpackMillisUtc((Long) value)).toString(); } else { - fail.accept("time low bound"); - break; + throw new UnhandledFilterException("time"); } } - else if (!(value instanceof Number)) { - fail.accept("tag comparision low bound"); - break; - } - where.add(column).append(low).add(value); - hasLow = true; + where.add(column).append(" = ").add(value); } - if (range.getHigh().getValueBlock().isPresent()) { - final String high; - switch (range.getHigh().getBound()) { - case EXACTLY: - high = " <= "; - break; - case BELOW: - high = " < "; - break; - default: - fail.accept("high bound"); - break ranges; - } - if (hasLow) { - where.append(" AND "); - } - Object value = range.getHigh().getValue(); - if (column.getKind() == InfluxColumn.Kind.TIME) { - if (value instanceof Long) { - value = Instant.ofEpochMilli(DateTimeEncoding.unpackMillisUtc((Long) value)).toString(); + else { + boolean hasLow = false; + if (range.getLow().getValueBlock().isPresent()) { + final String low; + switch (range.getLow().getBound()) { + case EXACTLY: + low = " >= "; + break; + case ABOVE: + low = " > "; + break; + default: + throw new UnhandledFilterException("low bound"); } - else { - fail.accept("time high bound"); - break; + Object value = range.getLow().getValue(); + if (column.getKind() == InfluxColumn.Kind.TIME) { + if (value instanceof Long) { + value = Instant.ofEpochMilli(DateTimeEncoding.unpackMillisUtc((Long) value)).toString(); + } + else { + throw new UnhandledFilterException("time low bound"); + } } + else if (!(value instanceof Number)) { + throw new UnhandledFilterException("tag comparision low bound"); + } + where.add(column).append(low).add(value); + hasLow = true; } - else if (!(value instanceof Number)) { - fail.accept("tag comparison high bound"); - break; + if (range.getHigh().getValueBlock().isPresent()) { + final String high; + switch (range.getHigh().getBound()) { + case EXACTLY: + high = " <= "; + break; + case BELOW: + high = " < "; + break; + default: + throw new UnhandledFilterException("high bound"); + } + if (hasLow) { + where.append(" AND "); + } + Object value = range.getHigh().getValue(); + if (column.getKind() == InfluxColumn.Kind.TIME) { + if (value instanceof Long) { + value = Instant.ofEpochMilli(DateTimeEncoding.unpackMillisUtc((Long) value)).toString(); + } + else { + throw new UnhandledFilterException("time high bound"); + } + } + else if (!(value instanceof Number)) { + throw new UnhandledFilterException("tag comparison high bound"); + } + where.add(column).append(high).add(value); } - where.add(column).append(high).add(value); } + first = false; + } + if (first) { + if (column.getKind() == InfluxColumn.Kind.TAG) { + // WHERE column IS NULL + where.append(where.isEmpty() ? "WHERE (" : " AND (").add(column).append(" = '')"); + } + else { + throw new UnhandledFilterException("SortedRangeSet"); + } + } + else { + where.append("))"); } - first = false; - } - if (ok.get() && first) { - fail.accept("SortedRangeSet"); - } - else { - where.append("))"); - } - } - else if (values instanceof EquatableValueSet) { - boolean first = true; - for (Object value : values.getDiscreteValues().getValues()) { - where.append(first ? where.isEmpty() ? "WHERE (" : " AND (" : " OR ") - .add(column).append(" = ").add(value); - first = false; } - if (first) { - fail.accept("EquatableValueSet"); + else if (values instanceof EquatableValueSet) { + boolean first = true; + for (Object value : values.getDiscreteValues().getValues()) { + where.append(first ? where.isEmpty() ? "WHERE (" : " AND (" : " OR ") + .add(column).append(" = ").add(value); + first = false; + } + if (first) { + throw new UnhandledFilterException("EquatableValueSet"); + } + else { + where.append(')'); + } } else { - where.append(')'); + throw new UnhandledFilterException("predicate"); } } - else { - fail.accept("predicate"); - } - if (!ok.get()) { + catch (UnhandledFilterException e) { + client.logger.debug("unhandled " + e + " " + column + ": " + values.toString(session)); // undo everything we did add to the where-clause where.truncate(startPos); // and tell Presto we couldn't handle all the filtering @@ -320,4 +312,13 @@ else if (values instanceof EquatableValueSet) { where, table.getLimit()), all ? TupleDomain.all() : constraint.getSummary())); } + + private static class UnhandledFilterException + extends Exception + { + private UnhandledFilterException(String message) + { + super(message); + } + } } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java index 143c56e0e618..bece49c19ce3 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java @@ -121,10 +121,8 @@ public void quote(String value, char delimiter) append(delimiter); for (int i = 0; i < value.length(); i++) { char ch = value.charAt(i); - if (ch < ' ' || ch > 127) { - InfluxError.BAD_VALUE.fail("illegal value", value); - } - if (ch == delimiter || ch == '\\') { + InfluxError.BAD_VALUE.check(ch >= ' ', "illegal value", value); + if (ch == '\\' || ch == delimiter) { append('\\'); } append(ch); diff --git a/presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxPluginTest.java b/presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxPluginTest.java new file mode 100644 index 000000000000..f45cf67cf6d6 --- /dev/null +++ b/presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxPluginTest.java @@ -0,0 +1,45 @@ +/* + * 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.google.common.collect.ImmutableMap; +import io.prestosql.spi.connector.Connector; +import io.prestosql.spi.connector.ConnectorFactory; +import io.prestosql.testing.TestingConnectorContext; +import org.testng.annotations.Test; + +import static com.google.common.collect.Iterables.getOnlyElement; +import static io.airlift.testing.Assertions.assertInstanceOf; +import static org.testng.Assert.assertNotNull; + +public class InfluxPluginTest +{ + @Test + public void testSpinUp() + { + InfluxPlugin plugin = new InfluxPlugin(); + + ConnectorFactory factory = getOnlyElement(plugin.getConnectorFactories()); + assertInstanceOf(factory, InfluxConnectorFactory.class); + + Connector c = factory.create( + "test-connector", + ImmutableMap.builder() + .put("influx.database", "test") + .build(), + new TestingConnectorContext()); + assertNotNull(c); + } +} diff --git a/presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxQLTest.java b/presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxQLTest.java new file mode 100644 index 000000000000..303c7d5c659f --- /dev/null +++ b/presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxQLTest.java @@ -0,0 +1,33 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.prestosql.plugin.influx; + +import org.testng.annotations.Test; + +import static org.testng.Assert.assertEquals; + +public class InfluxQLTest +{ + @Test + public void test() + { + InfluxQL test = new InfluxQL(); + test.addIdentifier("hello").append(" = ").add("world"); + assertEquals("hello = 'world'", test.toString()); + test.truncate(0); + test.addIdentifier("åäö").append(" = ").add("\"åäö'"); + assertEquals("\"åäö\" = '\"åäö\\''", test.toString()); + } +} From 0c879fa4db7cb8eb53d2441d4ae84cca43f89a21 Mon Sep 17 00:00:00 2001 From: Will Edwards Date: Fri, 3 Jan 2020 10:00:06 +0100 Subject: [PATCH 11/19] Influx SQL tests with QueryRunner --- presto-influx/pom.xml | 12 + .../plugin/influx/InfluxQueryTest.java | 257 ++++++++++++++++++ .../src/test/resources/canned_http.json | 90 ++++++ 3 files changed, 359 insertions(+) create mode 100644 presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxQueryTest.java create mode 100644 presto-influx/src/test/resources/canned_http.json diff --git a/presto-influx/pom.xml b/presto-influx/pom.xml index f82084250abe..ecbc2ecfdbf7 100644 --- a/presto-influx/pom.xml +++ b/presto-influx/pom.xml @@ -102,6 +102,18 @@ test + + org.assertj + assertj-core + test + + + + io.prestosql + presto-testing + test + + io.airlift testing diff --git a/presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxQueryTest.java b/presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxQueryTest.java new file mode 100644 index 000000000000..e1f32e85ac2c --- /dev/null +++ b/presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxQueryTest.java @@ -0,0 +1,257 @@ +/* + * 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.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import com.sun.net.httpserver.HttpServer; +import io.prestosql.Session; +import io.prestosql.spi.security.Identity; +import io.prestosql.spi.security.SelectedRole; +import io.prestosql.testing.DistributedQueryRunner; +import io.prestosql.testing.MaterializedResult; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import java.io.InputStream; +import java.net.HttpURLConnection; +import java.net.InetSocketAddress; +import java.net.URLDecoder; +import java.nio.charset.StandardCharsets; +import java.time.Instant; +import java.time.ZoneId; +import java.time.ZonedDateTime; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicReference; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import static io.prestosql.spi.security.SelectedRole.Type.ROLE; +import static io.prestosql.spi.type.DoubleType.DOUBLE; +import static io.prestosql.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; +import static io.prestosql.spi.type.VarcharType.VARCHAR; +import static io.prestosql.testing.TestingSession.testSessionBuilder; +import static io.prestosql.testing.assertions.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; + +public class InfluxQueryTest +{ + private HttpServer httpServer; + private DistributedQueryRunner queryRunner; + private Session session; + private Map cannedHttp; + private final AtomicReference lastQuery = new AtomicReference<>(); + private static ZonedDateTime t1 = getTimestamp("2019-12-10T21:00:04.446Z"); + private static ZonedDateTime t2 = getTimestamp("2019-12-10T21:00:20.446Z"); + private static ZonedDateTime t3 = getTimestamp("2019-12-10T22:00:04.446Z"); + + @BeforeClass + public void beforeClass() + throws Exception + { + // load canned HTTP responses + cannedHttp = new HashMap<>(); + try (InputStream in = getClass().getClassLoader().getResourceAsStream("canned_http.json")) { + assertNotNull(in); + new ObjectMapper() + .readTree(in) + .get("queries") + .fields() + .forEachRemaining(node -> cannedHttp.put(node.getKey(), node.getValue().toString())); + } + + // create a real local http server to test against + Pattern queryExtractor = Pattern.compile("q=([^&]+)"); + httpServer = HttpServer.create(new InetSocketAddress(0), 0); + httpServer.createContext("/query", exchange -> { + try { + String params = exchange.getRequestURI().getQuery(); + assertNotNull(params); + Matcher queryMatcher = queryExtractor.matcher(params); + assertTrue(queryMatcher.find()); + String queryString = queryMatcher.group(1); + queryString = URLDecoder.decode(queryString, StandardCharsets.UTF_8.name()); + lastQuery.set(queryString); + assertFalse(queryMatcher.find()); + String results = cannedHttp.get(queryString); + assertNotNull(results, queryString); + byte[] response = results.getBytes(StandardCharsets.UTF_8); + exchange.sendResponseHeaders(HttpURLConnection.HTTP_OK, response.length); + exchange.getResponseBody().write(response); + } + catch (Throwable t) { + exchange.sendResponseHeaders(HttpURLConnection.HTTP_INTERNAL_ERROR, 0); + exchange.getResponseBody().write(new byte[] {}); + } + finally { + exchange.close(); + } + }); + httpServer.start(); + + // create a query runner with a real influx connector + queryRunner = DistributedQueryRunner.builder(testSessionBuilder() + .setIdentity(Identity.forUser("test") + .withRole("catalog", new SelectedRole(ROLE, Optional.of("admin"))) + .build()) + .build()).build(); + queryRunner.installPlugin(new InfluxPlugin()); + queryRunner.createCatalog("catalog", "influx", + new ImmutableMap.Builder() + .put("influx.port", Integer.toString(httpServer.getAddress().getPort())) + .put("influx.database", "test") + .build()); + session = queryRunner.getDefaultSession(); + } + + @Test + public synchronized void testShow() + { + MaterializedResult result = queryRunner.getClient().execute(session, "SHOW TABLES").getResult(); + assertEquals(result.getOnlyColumn().collect(Collectors.toList()), Collections.singletonList("data")); + + result = queryRunner.getClient().execute(session, "DESC data") + .getResult() + .toTestTypes(); + MaterializedResult expectedColumns = MaterializedResult + .resultBuilder(session, VARCHAR, VARCHAR, VARCHAR, VARCHAR) + .row("time", "timestamp with time zone", "time", "") + .row("k1", "varchar", "tag", "") + .row("k2", "varchar", "tag", "") + .row("v1", "double", "field", "") + .row("v2", "double", "field", "") + .build(); + assertEquals(result, expectedColumns); + } + + @Test + public synchronized void testSelectAll() + { + MaterializedResult result = queryRunner.getClient().execute(session, "SELECT * FROM datA") + .getResult() + .toTestTypes(); + assertEquals(lastQuery.get(), "SELECT * FROM SCHEMA.Data"); + MaterializedResult expectedColumns = MaterializedResult + .resultBuilder(session, TIMESTAMP_WITH_TIME_ZONE, VARCHAR, VARCHAR, DOUBLE, DOUBLE) + .row(t1, "a", "b", 1., 2.) + .row(t2, "a", "c", 3., 4.) + .row(t3, "d", "b", 5., 6.) + .build(); + assertEquals(result, expectedColumns); + } + + @Test + public synchronized void testSelectTagEqual() + { + MaterializedResult result = queryRunner.getClient().execute(session, "SELECT * FROM data WHERE k1 = 'a'") + .getResult() + .toTestTypes(); + assertEquals(lastQuery.get(), "SELECT * FROM SCHEMA.Data WHERE ((k1 = 'a'))"); + MaterializedResult expectedColumns = MaterializedResult + .resultBuilder(session, TIMESTAMP_WITH_TIME_ZONE, VARCHAR, VARCHAR, DOUBLE, DOUBLE) + .row(t1, "a", "b", 1., 2.) + .row(t2, "a", "c", 3., 4.) + .build(); + assertEquals(result, expectedColumns); + } + + @Test + public synchronized void testSelectTagNotEqual() + { + MaterializedResult result = queryRunner.getClient().execute(session, "SELECT * FROM data WHERE k1 != 'd'") + .getResult() + .toTestTypes(); + assertEquals(lastQuery.get(), "SELECT * FROM SCHEMA.Data"); // can't be pushed down + MaterializedResult expectedColumns = MaterializedResult + .resultBuilder(session, TIMESTAMP_WITH_TIME_ZONE, VARCHAR, VARCHAR, DOUBLE, DOUBLE) + .row(t1, "a", "b", 1., 2.) + .row(t2, "a", "c", 3., 4.) + .build(); + assertEquals(result, expectedColumns); + } + + @Test + public synchronized void testSelectTagNull() + { + MaterializedResult result = queryRunner.getClient().execute(session, "SELECT * FROM data WHERE k1 IS NULL") + .getResult() + .toTestTypes(); + assertEquals(lastQuery.get(), "SELECT * FROM SCHEMA.Data WHERE (k1 = '')"); + MaterializedResult expectedColumns = MaterializedResult + .resultBuilder(session, TIMESTAMP_WITH_TIME_ZONE, VARCHAR, VARCHAR, DOUBLE, DOUBLE) + .build(); + assertEquals(result, expectedColumns); + } + + @Test + public synchronized void testSelectFieldNull() + { + MaterializedResult result = queryRunner.getClient().execute(session, "SELECT * FROM data WHERE v1 IS NULL") + .getResult() + .toTestTypes(); + assertEquals(lastQuery.get(), "SELECT * FROM SCHEMA.Data"); // can't be pushed down + MaterializedResult expectedColumns = MaterializedResult + .resultBuilder(session, TIMESTAMP_WITH_TIME_ZONE, VARCHAR, VARCHAR, DOUBLE, DOUBLE) + .build(); + assertEquals(result, expectedColumns); + } + + @Test + public synchronized void testSelectFieldEqual() + { + MaterializedResult result = queryRunner.getClient().execute(session, "SELECT * FROM data WHERE v1 = 1") + .getResult() + .toTestTypes(); + assertEquals(lastQuery.get(), "SELECT * FROM SCHEMA.Data WHERE ((v1 = 1.0))"); + MaterializedResult expectedColumns = MaterializedResult + .resultBuilder(session, TIMESTAMP_WITH_TIME_ZONE, VARCHAR, VARCHAR, DOUBLE, DOUBLE) + .row(t1, "a", "b", 1., 2.) + .build(); + assertEquals(result, expectedColumns); + } + + @Test + public synchronized void testSelectFieldBetween() + { + MaterializedResult result = queryRunner.getClient().execute(session, "SELECT * FROM data WHERE v1 BETWEEN 1 AND 3") + .getResult() + .toTestTypes(); + assertEquals(lastQuery.get(), "SELECT * FROM SCHEMA.Data WHERE ((v1 >= 1.0 AND v1 <= 3.0))"); + MaterializedResult expectedColumns = MaterializedResult + .resultBuilder(session, TIMESTAMP_WITH_TIME_ZONE, VARCHAR, VARCHAR, DOUBLE, DOUBLE) + .row(t1, "a", "b", 1., 2.) + .row(t2, "a", "c", 3., 4.) + .build(); + assertEquals(result, expectedColumns); + } + + private static ZonedDateTime getTimestamp(String timestamp) + { + return ZonedDateTime.ofInstant(Instant.parse(timestamp), ZoneId.of("UTC")); + } + + @AfterClass + public void afterClass() + { + queryRunner.close(); + httpServer.stop(0); + } +} diff --git a/presto-influx/src/test/resources/canned_http.json b/presto-influx/src/test/resources/canned_http.json new file mode 100644 index 000000000000..1099ebafed97 --- /dev/null +++ b/presto-influx/src/test/resources/canned_http.json @@ -0,0 +1,90 @@ +{ + "queries": { + "SHOW RETENTION POLICIES": { + "results": [{ + "series": [{ + "columns": ["name", "duration", "shardGroupDuration", "replicaN", "default"], + "values": [["autogen", "0s", "168h0m0s", 1, true], ["SCHEMA", "0s", "168h0m0s", 3, false]] + }] + }] + }, + "SHOW MEASUREMENTS": { + "results": [{ + "series": [{ + "name": "measurements", + "columns": ["name"], + "values": [["Data"]] + }] + }] + }, + "SHOW FIELD KEYS FROM SCHEMA.Data": { + "results": [{ + "series": [{ + "name": "Data", + "columns": ["fieldKey", "fieldType"], + "values": [["v1", "float"], ["v2","float"]] + }] + }] + }, + "SHOW TAG KEYS FROM Data": { + "results": [{ + "series": [{ + "name": "Data", + "columns": ["tagKey"], + "values": [["k1"],["k2"]] + }] + }] + }, + "SELECT * FROM SCHEMA.Data": { + "results": [{ + "series": [{ + "name": "Data", + "columns": ["time","v1","v2","k1","k2"], + "values": [ + ["2019-12-10T21:00:04.446017Z", 1, 2, "a", "b"], + ["2019-12-10T21:00:20.446017Z", 3, 4, "a", "c"], + ["2019-12-10T22:00:04.446017Z", 5, 6, "d", "b"] + ] + }] + }] + }, + "SELECT * FROM SCHEMA.Data WHERE ((k1 = 'a'))": { + "results": [{ + "series": [{ + "name": "Data", + "columns": ["time","v1","v2","k1","k2"], + "values": [ + ["2019-12-10T21:00:04.446017Z", 1, 2, "a", "b"], + ["2019-12-10T21:00:20.446017Z", 3, 4, "a", "c"] + ] + }] + }] + }, + "SELECT * FROM SCHEMA.Data WHERE (k1 = '')": { + "results":[{}] + }, + "SELECT * FROM SCHEMA.Data WHERE ((v1 = 1.0))": { + "results": [{ + "series": [{ + "name": "Data", + "columns": ["time","v1","v2","k1","k2"], + "values": [ + ["2019-12-10T21:00:04.446017Z", 1, 2, "a", "b"] + ] + }] + }] + }, + "SELECT * FROM SCHEMA.Data WHERE ((v1 >= 1.0 AND v1 <= 3.0))": { + "results": [{ + "series": [{ + "name": "Data", + "columns": ["time","v1","v2","k1","k2"], + "values": [ + ["2019-12-10T21:00:04.446017Z", 1, 2, "a", "b"], + ["2019-12-10T21:00:20.446017Z", 3, 4, "a", "c"] + ] + }] + }] + } + } +} \ No newline at end of file From f83a5a056a5f5ad62291fba60bb3b8abce004640 Mon Sep 17 00:00:00 2001 From: Will Edwards Date: Fri, 3 Jan 2020 10:14:59 +0100 Subject: [PATCH 12/19] Tidied up influx query tests --- .../plugin/influx/InfluxQueryTest.java | 106 ++++++++++-------- 1 file changed, 61 insertions(+), 45 deletions(-) diff --git a/presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxQueryTest.java b/presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxQueryTest.java index e1f32e85ac2c..c22ed291a783 100644 --- a/presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxQueryTest.java +++ b/presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxQueryTest.java @@ -59,9 +59,6 @@ public class InfluxQueryTest private Session session; private Map cannedHttp; private final AtomicReference lastQuery = new AtomicReference<>(); - private static ZonedDateTime t1 = getTimestamp("2019-12-10T21:00:04.446Z"); - private static ZonedDateTime t2 = getTimestamp("2019-12-10T21:00:20.446Z"); - private static ZonedDateTime t3 = getTimestamp("2019-12-10T22:00:04.446Z"); @BeforeClass public void beforeClass() @@ -123,14 +120,12 @@ public void beforeClass() } @Test - public synchronized void testShow() + public void testShow() { - MaterializedResult result = queryRunner.getClient().execute(session, "SHOW TABLES").getResult(); + MaterializedResult result = execute("SHOW TABLES"); assertEquals(result.getOnlyColumn().collect(Collectors.toList()), Collections.singletonList("data")); - result = queryRunner.getClient().execute(session, "DESC data") - .getResult() - .toTestTypes(); + result = execute("DESC data"); MaterializedResult expectedColumns = MaterializedResult .resultBuilder(session, VARCHAR, VARCHAR, VARCHAR, VARCHAR) .row("time", "timestamp with time zone", "time", "") @@ -143,12 +138,11 @@ public synchronized void testShow() } @Test - public synchronized void testSelectAll() + public void testSelectAll() { - MaterializedResult result = queryRunner.getClient().execute(session, "SELECT * FROM datA") - .getResult() - .toTestTypes(); - assertEquals(lastQuery.get(), "SELECT * FROM SCHEMA.Data"); + MaterializedResult result = execute( + "SELECT * FROM datA", + "SELECT * FROM SCHEMA.Data"); MaterializedResult expectedColumns = MaterializedResult .resultBuilder(session, TIMESTAMP_WITH_TIME_ZONE, VARCHAR, VARCHAR, DOUBLE, DOUBLE) .row(t1, "a", "b", 1., 2.) @@ -159,12 +153,11 @@ public synchronized void testSelectAll() } @Test - public synchronized void testSelectTagEqual() + public void testSelectTagEqual() { - MaterializedResult result = queryRunner.getClient().execute(session, "SELECT * FROM data WHERE k1 = 'a'") - .getResult() - .toTestTypes(); - assertEquals(lastQuery.get(), "SELECT * FROM SCHEMA.Data WHERE ((k1 = 'a'))"); + MaterializedResult result = execute( + "SELECT * FROM data WHERE k1 = 'a'", + "SELECT * FROM SCHEMA.Data WHERE ((k1 = 'a'))"); MaterializedResult expectedColumns = MaterializedResult .resultBuilder(session, TIMESTAMP_WITH_TIME_ZONE, VARCHAR, VARCHAR, DOUBLE, DOUBLE) .row(t1, "a", "b", 1., 2.) @@ -174,12 +167,11 @@ public synchronized void testSelectTagEqual() } @Test - public synchronized void testSelectTagNotEqual() + public void testSelectTagNotEqual() { - MaterializedResult result = queryRunner.getClient().execute(session, "SELECT * FROM data WHERE k1 != 'd'") - .getResult() - .toTestTypes(); - assertEquals(lastQuery.get(), "SELECT * FROM SCHEMA.Data"); // can't be pushed down + MaterializedResult result = execute( + "SELECT * FROM data WHERE k1 != 'd'", + "SELECT * FROM SCHEMA.Data"); // can't be pushed down MaterializedResult expectedColumns = MaterializedResult .resultBuilder(session, TIMESTAMP_WITH_TIME_ZONE, VARCHAR, VARCHAR, DOUBLE, DOUBLE) .row(t1, "a", "b", 1., 2.) @@ -189,12 +181,25 @@ public synchronized void testSelectTagNotEqual() } @Test - public synchronized void testSelectTagNull() + public void testSelectField() { - MaterializedResult result = queryRunner.getClient().execute(session, "SELECT * FROM data WHERE k1 IS NULL") - .getResult() - .toTestTypes(); - assertEquals(lastQuery.get(), "SELECT * FROM SCHEMA.Data WHERE (k1 = '')"); + MaterializedResult result = execute( + "SELECT v1 FROM data WHERE k1 != 'd'", + "SELECT * FROM SCHEMA.Data"); // always SELECT *, and k1 clause can't be pushed down + MaterializedResult expectedColumns = MaterializedResult + .resultBuilder(session, DOUBLE) + .row(1.) + .row(3.) + .build(); + assertEquals(result, expectedColumns); + } + + @Test + public void testSelectTagNull() + { + MaterializedResult result = execute( + "SELECT * FROM data WHERE k1 IS NULL", + "SELECT * FROM SCHEMA.Data WHERE (k1 = '')"); MaterializedResult expectedColumns = MaterializedResult .resultBuilder(session, TIMESTAMP_WITH_TIME_ZONE, VARCHAR, VARCHAR, DOUBLE, DOUBLE) .build(); @@ -202,12 +207,11 @@ public synchronized void testSelectTagNull() } @Test - public synchronized void testSelectFieldNull() + public void testSelectFieldNull() { - MaterializedResult result = queryRunner.getClient().execute(session, "SELECT * FROM data WHERE v1 IS NULL") - .getResult() - .toTestTypes(); - assertEquals(lastQuery.get(), "SELECT * FROM SCHEMA.Data"); // can't be pushed down + MaterializedResult result = execute( + "SELECT * FROM data WHERE v1 IS NULL", + "SELECT * FROM SCHEMA.Data"); // can't be pushed down MaterializedResult expectedColumns = MaterializedResult .resultBuilder(session, TIMESTAMP_WITH_TIME_ZONE, VARCHAR, VARCHAR, DOUBLE, DOUBLE) .build(); @@ -215,12 +219,11 @@ public synchronized void testSelectFieldNull() } @Test - public synchronized void testSelectFieldEqual() + public void testSelectFieldEqual() { - MaterializedResult result = queryRunner.getClient().execute(session, "SELECT * FROM data WHERE v1 = 1") - .getResult() - .toTestTypes(); - assertEquals(lastQuery.get(), "SELECT * FROM SCHEMA.Data WHERE ((v1 = 1.0))"); + MaterializedResult result = execute( + "SELECT * FROM data WHERE v1 = 1", + "SELECT * FROM SCHEMA.Data WHERE ((v1 = 1.0))"); MaterializedResult expectedColumns = MaterializedResult .resultBuilder(session, TIMESTAMP_WITH_TIME_ZONE, VARCHAR, VARCHAR, DOUBLE, DOUBLE) .row(t1, "a", "b", 1., 2.) @@ -229,12 +232,11 @@ public synchronized void testSelectFieldEqual() } @Test - public synchronized void testSelectFieldBetween() + public void testSelectFieldBetween() { - MaterializedResult result = queryRunner.getClient().execute(session, "SELECT * FROM data WHERE v1 BETWEEN 1 AND 3") - .getResult() - .toTestTypes(); - assertEquals(lastQuery.get(), "SELECT * FROM SCHEMA.Data WHERE ((v1 >= 1.0 AND v1 <= 3.0))"); + MaterializedResult result = execute( + "SELECT * FROM data WHERE v1 BETWEEN 1 AND 3", + "SELECT * FROM SCHEMA.Data WHERE ((v1 >= 1.0 AND v1 <= 3.0))"); MaterializedResult expectedColumns = MaterializedResult .resultBuilder(session, TIMESTAMP_WITH_TIME_ZONE, VARCHAR, VARCHAR, DOUBLE, DOUBLE) .row(t1, "a", "b", 1., 2.) @@ -243,9 +245,18 @@ public synchronized void testSelectFieldBetween() assertEquals(result, expectedColumns); } - private static ZonedDateTime getTimestamp(String timestamp) + private synchronized MaterializedResult execute(String sql) { - return ZonedDateTime.ofInstant(Instant.parse(timestamp), ZoneId.of("UTC")); + return queryRunner.getClient().execute(session, sql) + .getResult() + .toTestTypes(); + } + + private synchronized MaterializedResult execute(String sql, String influxQL) + { + MaterializedResult result = execute(sql); + assertEquals(lastQuery.get(), influxQL); + return result; } @AfterClass @@ -254,4 +265,9 @@ public void afterClass() queryRunner.close(); httpServer.stop(0); } + + // test data + private static ZonedDateTime t1 = ZonedDateTime.ofInstant(Instant.parse("2019-12-10T21:00:04.446Z"), ZoneId.of("UTC")); + private static ZonedDateTime t2 = ZonedDateTime.ofInstant(Instant.parse("2019-12-10T21:00:20.446Z"), ZoneId.of("UTC")); + private static ZonedDateTime t3 = ZonedDateTime.ofInstant(Instant.parse("2019-12-10T22:00:04.446Z"), ZoneId.of("UTC")); } From 18bce36a696308de36aa5eb7b0ce6abb9dd38aa5 Mon Sep 17 00:00:00 2001 From: Will Edwards Date: Sat, 4 Jan 2020 22:55:41 +0100 Subject: [PATCH 13/19] Use influxdb in a test-container rather than canned responses --- presto-influx/pom.xml | 6 + .../prestosql/plugin/influx/InfluxClient.java | 33 +-- .../prestosql/plugin/influx/InfluxHttp.java | 152 ++++++++++ .../plugin/influx/InfluxQueryTest.java | 273 ------------------ ...xPluginTest.java => TestInfluxPlugin.java} | 2 +- .../{InfluxQLTest.java => TestInfluxQL.java} | 2 +- .../plugin/influx/TestInfluxSelect.java | 171 +++++++++++ .../plugin/influx/TestingInfluxServer.java | 76 +++++ 8 files changed, 410 insertions(+), 305 deletions(-) create mode 100644 presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxHttp.java delete mode 100644 presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxQueryTest.java rename presto-influx/src/test/java/io/prestosql/plugin/influx/{InfluxPluginTest.java => TestInfluxPlugin.java} (98%) rename presto-influx/src/test/java/io/prestosql/plugin/influx/{InfluxQLTest.java => TestInfluxQL.java} (97%) create mode 100644 presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxSelect.java create mode 100644 presto-influx/src/test/java/io/prestosql/plugin/influx/TestingInfluxServer.java diff --git a/presto-influx/pom.xml b/presto-influx/pom.xml index ecbc2ecfdbf7..8dcfcf49b8ee 100644 --- a/presto-influx/pom.xml +++ b/presto-influx/pom.xml @@ -119,6 +119,12 @@ testing test + + + org.testcontainers + influxdb + test + diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java index 4a2a3a3c457c..eb171873c0ab 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java @@ -15,7 +15,6 @@ package io.prestosql.plugin.influx; import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.JsonNodeType; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -24,9 +23,6 @@ import javax.inject.Inject; -import java.net.URL; -import java.net.URLEncoder; -import java.nio.charset.StandardCharsets; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -42,6 +38,7 @@ public class InfluxClient { final Logger logger; 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> retentionPolicies; // schema name (lower-case) -> retention policy (case-sensitive) private final CachedMetaData> measurements; // table name (lower-case) -> measurement (case-sensitive) @@ -53,6 +50,7 @@ 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<>(); @@ -173,34 +171,9 @@ private Map showNames(String query) return ImmutableMap.copyOf(names); } - /* Using raw HTTP because the influx java library has dependency conflicts and puts the burden of quoting identifiers on the caller */ JsonNode execute(String query) { - final JsonNode response; - try { - String authentication = config.getUserName() != null ? - config.getUserName() + (config.getPassword() != null ? - ":" + config.getPassword() : - "") + "@" : - ""; - URL url = new URL((config.isUseHttps() ? "https://" : "http://") - + authentication - + config.getHost() + ":" + config.getPort() + - "/query?db=" + config.getDatabase() + - "&q=" + URLEncoder.encode(query, StandardCharsets.UTF_8.toString())); - response = new ObjectMapper().readTree(url); - } - catch (Throwable t) { - InfluxError.EXTERNAL.fail(t); - return null; - } - JsonNode results = response.get("results"); - InfluxError.GENERAL.check(results != null && results.size() == 1, "expecting one result", query); - JsonNode result = results.get(0); - if (result.has("error")) { - InfluxError.GENERAL.fail(result.get("error").asText(), query); - } - return result.get("series"); + return http.query(query); } public HostAddress getHostAddress() diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxHttp.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxHttp.java new file mode 100644 index 000000000000..567d2749d988 --- /dev/null +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxHttp.java @@ -0,0 +1,152 @@ +/* + * 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.ObjectMapper; +import com.google.common.collect.ImmutableMap; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.UnsupportedEncodingException; +import java.net.HttpURLConnection; +import java.net.MalformedURLException; +import java.net.URL; +import java.net.URLEncoder; +import java.nio.charset.StandardCharsets; +import java.util.Map; +import java.util.Scanner; + +/* Using raw HTTP because the influx java library has dependency conflicts and puts the burden of quoting identifiers on the caller */ +public class InfluxHttp +{ + private final String host; + private final int port; + private final boolean useHttps; + private final String database; + private final String username; + private final String password; + + public InfluxHttp(String host, int port, boolean useHttps, String database, String username, String password) + { + this.host = host; + this.port = port; + this.useHttps = useHttps; + this.database = database; + this.username = username; + this.password = password; + } + + public void execute(String ddl) + { + try { + URL url = getUrl("query", + new ImmutableMap.Builder() + .put("q", ddl) + .build()); + checkSuccess((HttpURLConnection) url.openConnection()); + } + catch (Throwable t) { + InfluxError.EXTERNAL.fail(t); + } + } + + public JsonNode query(String query) + { + final JsonNode response; + try { + response = new ObjectMapper().readTree(getUrl("query", + new ImmutableMap.Builder() + .put("db", database) + .put("q", query) + .build())); + } + catch (Throwable t) { + InfluxError.EXTERNAL.fail(t); + return null; + } + JsonNode results = response.get("results"); + InfluxError.GENERAL.check(results != null && results.size() == 1, "expecting one result", query); + JsonNode result = results.get(0); + if (result.has("error")) { + InfluxError.GENERAL.fail(result.get("error").asText(), query); + } + return result.get("series"); + } + + public void write(String retentionPolicy, String... lines) + { + byte[] body = String.join("\n", lines).getBytes(StandardCharsets.UTF_8); + try { + URL url = getUrl("write", + new ImmutableMap.Builder() + .put("db", database) + .put("rp", retentionPolicy) + .build()); + HttpURLConnection conn = (HttpURLConnection) url.openConnection(); + conn.setRequestMethod("POST"); + conn.setDoOutput(true); + conn.setFixedLengthStreamingMode(body.length); + conn.connect(); + try (OutputStream out = conn.getOutputStream()) { + out.write(body); + } + checkSuccess(conn); + } + catch (Throwable t) { + InfluxError.EXTERNAL.fail(t); + } + } + + private URL getUrl(String path, Map parameters) + throws UnsupportedEncodingException, MalformedURLException + { + String authentication = username != null ? + username + (password != null ? + ":" + password : + "") + "@" : + ""; + StringBuilder query = new StringBuilder(); + for (Map.Entry parameter : parameters.entrySet()) { + query + .append(query.length() == 0 ? '?' : '&') + .append(parameter.getKey()) + .append('=') + .append(URLEncoder.encode(parameter.getValue(), StandardCharsets.UTF_8.toString())); + } + return new URL((useHttps ? "https://" : "http://") + + authentication + + host + ":" + port + + "/" + path + + query); + } + + private void checkSuccess(HttpURLConnection conn) + throws IOException + { + int responseCode = conn.getResponseCode(); + if (responseCode < 200 || responseCode >= 300) { + String errorMessage = ""; + InputStream err = conn.getErrorStream(); + if (err != null) { + try (Scanner scanner = new Scanner(err)) { + scanner.useDelimiter("\\Z"); + errorMessage += scanner.next(); + } + } + InfluxError.EXTERNAL.fail("error writing to influx (" + responseCode + ")", errorMessage); + } + } +} diff --git a/presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxQueryTest.java b/presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxQueryTest.java deleted file mode 100644 index c22ed291a783..000000000000 --- a/presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxQueryTest.java +++ /dev/null @@ -1,273 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.prestosql.plugin.influx; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableMap; -import com.sun.net.httpserver.HttpServer; -import io.prestosql.Session; -import io.prestosql.spi.security.Identity; -import io.prestosql.spi.security.SelectedRole; -import io.prestosql.testing.DistributedQueryRunner; -import io.prestosql.testing.MaterializedResult; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.Test; - -import java.io.InputStream; -import java.net.HttpURLConnection; -import java.net.InetSocketAddress; -import java.net.URLDecoder; -import java.nio.charset.StandardCharsets; -import java.time.Instant; -import java.time.ZoneId; -import java.time.ZonedDateTime; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.Optional; -import java.util.concurrent.atomic.AtomicReference; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import java.util.stream.Collectors; - -import static io.prestosql.spi.security.SelectedRole.Type.ROLE; -import static io.prestosql.spi.type.DoubleType.DOUBLE; -import static io.prestosql.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; -import static io.prestosql.spi.type.VarcharType.VARCHAR; -import static io.prestosql.testing.TestingSession.testSessionBuilder; -import static io.prestosql.testing.assertions.Assert.assertEquals; -import static org.testng.Assert.assertFalse; -import static org.testng.Assert.assertNotNull; -import static org.testng.Assert.assertTrue; - -public class InfluxQueryTest -{ - private HttpServer httpServer; - private DistributedQueryRunner queryRunner; - private Session session; - private Map cannedHttp; - private final AtomicReference lastQuery = new AtomicReference<>(); - - @BeforeClass - public void beforeClass() - throws Exception - { - // load canned HTTP responses - cannedHttp = new HashMap<>(); - try (InputStream in = getClass().getClassLoader().getResourceAsStream("canned_http.json")) { - assertNotNull(in); - new ObjectMapper() - .readTree(in) - .get("queries") - .fields() - .forEachRemaining(node -> cannedHttp.put(node.getKey(), node.getValue().toString())); - } - - // create a real local http server to test against - Pattern queryExtractor = Pattern.compile("q=([^&]+)"); - httpServer = HttpServer.create(new InetSocketAddress(0), 0); - httpServer.createContext("/query", exchange -> { - try { - String params = exchange.getRequestURI().getQuery(); - assertNotNull(params); - Matcher queryMatcher = queryExtractor.matcher(params); - assertTrue(queryMatcher.find()); - String queryString = queryMatcher.group(1); - queryString = URLDecoder.decode(queryString, StandardCharsets.UTF_8.name()); - lastQuery.set(queryString); - assertFalse(queryMatcher.find()); - String results = cannedHttp.get(queryString); - assertNotNull(results, queryString); - byte[] response = results.getBytes(StandardCharsets.UTF_8); - exchange.sendResponseHeaders(HttpURLConnection.HTTP_OK, response.length); - exchange.getResponseBody().write(response); - } - catch (Throwable t) { - exchange.sendResponseHeaders(HttpURLConnection.HTTP_INTERNAL_ERROR, 0); - exchange.getResponseBody().write(new byte[] {}); - } - finally { - exchange.close(); - } - }); - httpServer.start(); - - // create a query runner with a real influx connector - queryRunner = DistributedQueryRunner.builder(testSessionBuilder() - .setIdentity(Identity.forUser("test") - .withRole("catalog", new SelectedRole(ROLE, Optional.of("admin"))) - .build()) - .build()).build(); - queryRunner.installPlugin(new InfluxPlugin()); - queryRunner.createCatalog("catalog", "influx", - new ImmutableMap.Builder() - .put("influx.port", Integer.toString(httpServer.getAddress().getPort())) - .put("influx.database", "test") - .build()); - session = queryRunner.getDefaultSession(); - } - - @Test - public void testShow() - { - MaterializedResult result = execute("SHOW TABLES"); - assertEquals(result.getOnlyColumn().collect(Collectors.toList()), Collections.singletonList("data")); - - result = execute("DESC data"); - MaterializedResult expectedColumns = MaterializedResult - .resultBuilder(session, VARCHAR, VARCHAR, VARCHAR, VARCHAR) - .row("time", "timestamp with time zone", "time", "") - .row("k1", "varchar", "tag", "") - .row("k2", "varchar", "tag", "") - .row("v1", "double", "field", "") - .row("v2", "double", "field", "") - .build(); - assertEquals(result, expectedColumns); - } - - @Test - public void testSelectAll() - { - MaterializedResult result = execute( - "SELECT * FROM datA", - "SELECT * FROM SCHEMA.Data"); - MaterializedResult expectedColumns = MaterializedResult - .resultBuilder(session, TIMESTAMP_WITH_TIME_ZONE, VARCHAR, VARCHAR, DOUBLE, DOUBLE) - .row(t1, "a", "b", 1., 2.) - .row(t2, "a", "c", 3., 4.) - .row(t3, "d", "b", 5., 6.) - .build(); - assertEquals(result, expectedColumns); - } - - @Test - public void testSelectTagEqual() - { - MaterializedResult result = execute( - "SELECT * FROM data WHERE k1 = 'a'", - "SELECT * FROM SCHEMA.Data WHERE ((k1 = 'a'))"); - MaterializedResult expectedColumns = MaterializedResult - .resultBuilder(session, TIMESTAMP_WITH_TIME_ZONE, VARCHAR, VARCHAR, DOUBLE, DOUBLE) - .row(t1, "a", "b", 1., 2.) - .row(t2, "a", "c", 3., 4.) - .build(); - assertEquals(result, expectedColumns); - } - - @Test - public void testSelectTagNotEqual() - { - MaterializedResult result = execute( - "SELECT * FROM data WHERE k1 != 'd'", - "SELECT * FROM SCHEMA.Data"); // can't be pushed down - MaterializedResult expectedColumns = MaterializedResult - .resultBuilder(session, TIMESTAMP_WITH_TIME_ZONE, VARCHAR, VARCHAR, DOUBLE, DOUBLE) - .row(t1, "a", "b", 1., 2.) - .row(t2, "a", "c", 3., 4.) - .build(); - assertEquals(result, expectedColumns); - } - - @Test - public void testSelectField() - { - MaterializedResult result = execute( - "SELECT v1 FROM data WHERE k1 != 'd'", - "SELECT * FROM SCHEMA.Data"); // always SELECT *, and k1 clause can't be pushed down - MaterializedResult expectedColumns = MaterializedResult - .resultBuilder(session, DOUBLE) - .row(1.) - .row(3.) - .build(); - assertEquals(result, expectedColumns); - } - - @Test - public void testSelectTagNull() - { - MaterializedResult result = execute( - "SELECT * FROM data WHERE k1 IS NULL", - "SELECT * FROM SCHEMA.Data WHERE (k1 = '')"); - MaterializedResult expectedColumns = MaterializedResult - .resultBuilder(session, TIMESTAMP_WITH_TIME_ZONE, VARCHAR, VARCHAR, DOUBLE, DOUBLE) - .build(); - assertEquals(result, expectedColumns); - } - - @Test - public void testSelectFieldNull() - { - MaterializedResult result = execute( - "SELECT * FROM data WHERE v1 IS NULL", - "SELECT * FROM SCHEMA.Data"); // can't be pushed down - MaterializedResult expectedColumns = MaterializedResult - .resultBuilder(session, TIMESTAMP_WITH_TIME_ZONE, VARCHAR, VARCHAR, DOUBLE, DOUBLE) - .build(); - assertEquals(result, expectedColumns); - } - - @Test - public void testSelectFieldEqual() - { - MaterializedResult result = execute( - "SELECT * FROM data WHERE v1 = 1", - "SELECT * FROM SCHEMA.Data WHERE ((v1 = 1.0))"); - MaterializedResult expectedColumns = MaterializedResult - .resultBuilder(session, TIMESTAMP_WITH_TIME_ZONE, VARCHAR, VARCHAR, DOUBLE, DOUBLE) - .row(t1, "a", "b", 1., 2.) - .build(); - assertEquals(result, expectedColumns); - } - - @Test - public void testSelectFieldBetween() - { - MaterializedResult result = execute( - "SELECT * FROM data WHERE v1 BETWEEN 1 AND 3", - "SELECT * FROM SCHEMA.Data WHERE ((v1 >= 1.0 AND v1 <= 3.0))"); - MaterializedResult expectedColumns = MaterializedResult - .resultBuilder(session, TIMESTAMP_WITH_TIME_ZONE, VARCHAR, VARCHAR, DOUBLE, DOUBLE) - .row(t1, "a", "b", 1., 2.) - .row(t2, "a", "c", 3., 4.) - .build(); - assertEquals(result, expectedColumns); - } - - private synchronized MaterializedResult execute(String sql) - { - return queryRunner.getClient().execute(session, sql) - .getResult() - .toTestTypes(); - } - - private synchronized MaterializedResult execute(String sql, String influxQL) - { - MaterializedResult result = execute(sql); - assertEquals(lastQuery.get(), influxQL); - return result; - } - - @AfterClass - public void afterClass() - { - queryRunner.close(); - httpServer.stop(0); - } - - // test data - private static ZonedDateTime t1 = ZonedDateTime.ofInstant(Instant.parse("2019-12-10T21:00:04.446Z"), ZoneId.of("UTC")); - private static ZonedDateTime t2 = ZonedDateTime.ofInstant(Instant.parse("2019-12-10T21:00:20.446Z"), ZoneId.of("UTC")); - private static ZonedDateTime t3 = ZonedDateTime.ofInstant(Instant.parse("2019-12-10T22:00:04.446Z"), ZoneId.of("UTC")); -} diff --git a/presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxPluginTest.java b/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxPlugin.java similarity index 98% rename from presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxPluginTest.java rename to presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxPlugin.java index f45cf67cf6d6..045476aa5651 100644 --- a/presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxPluginTest.java +++ b/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxPlugin.java @@ -24,7 +24,7 @@ import static io.airlift.testing.Assertions.assertInstanceOf; import static org.testng.Assert.assertNotNull; -public class InfluxPluginTest +public class TestInfluxPlugin { @Test public void testSpinUp() diff --git a/presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxQLTest.java b/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxQL.java similarity index 97% rename from presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxQLTest.java rename to presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxQL.java index 303c7d5c659f..fac3f6ed9973 100644 --- a/presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxQLTest.java +++ b/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxQL.java @@ -18,7 +18,7 @@ import static org.testng.Assert.assertEquals; -public class InfluxQLTest +public class TestInfluxQL { @Test public void test() diff --git a/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxSelect.java b/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxSelect.java new file mode 100644 index 000000000000..9169c0b9e57f --- /dev/null +++ b/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxSelect.java @@ -0,0 +1,171 @@ +/* + * 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.google.common.collect.ImmutableMap; +import io.prestosql.Session; +import io.prestosql.spi.security.Identity; +import io.prestosql.spi.security.SelectedRole; +import io.prestosql.testing.DistributedQueryRunner; +import io.prestosql.testing.MaterializedResult; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import java.io.IOException; +import java.util.Collections; +import java.util.Optional; +import java.util.stream.Collectors; + +import static io.prestosql.spi.security.SelectedRole.Type.ROLE; +import static io.prestosql.spi.type.DoubleType.DOUBLE; +import static io.prestosql.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; +import static io.prestosql.spi.type.VarcharType.VARCHAR; +import static io.prestosql.testing.TestingSession.testSessionBuilder; +import static io.prestosql.testing.assertions.Assert.assertEquals; + +public class TestInfluxSelect +{ + private TestingInfluxServer influxServer; + private DistributedQueryRunner queryRunner; + private Session session; + + @BeforeClass + public void beforeClass() + throws Exception + { + influxServer = new TestingInfluxServer(); + + // create a query runner with a real influx connector + queryRunner = DistributedQueryRunner.builder(testSessionBuilder() + .setIdentity(Identity.forUser("test") + .withRole("catalog", new SelectedRole(ROLE, Optional.of("admin"))) + .build()) + .build()).build(); + queryRunner.installPlugin(new InfluxPlugin()); + queryRunner.createCatalog("catalog", "influx", + new ImmutableMap.Builder() + .put("influx.host", influxServer.getHost()) + .put("influx.port", Integer.toString(influxServer.getPort())) + .put("influx.database", TestingInfluxServer.DATABASE) + .build()); + session = queryRunner.getDefaultSession(); + } + + @Test + public void testShow() + { + MaterializedResult result = execute("SHOW TABLES"); + assertEquals(result.getOnlyColumn().collect(Collectors.toList()), Collections.singletonList("data")); + + result = execute("DESC data"); + MaterializedResult expectedColumns = MaterializedResult + .resultBuilder(session, VARCHAR, VARCHAR, VARCHAR, VARCHAR) + .row("time", "timestamp with time zone", "time", "") + .row("k1", "varchar", "tag", "") + .row("k2", "varchar", "tag", "") + .row("v1", "double", "field", "") + .row("v2", "double", "field", "") + .build(); + assertEquals(result, expectedColumns); + } + + @Test + public void testSelectAll() + { + MaterializedResult result = execute("SELECT * FROM datA"); + assertEquals(result, expect(0, 1, 2)); + } + + @Test + public void testSelectTagEqual() + { + MaterializedResult result = execute("SELECT * FROM data WHERE k1 = 'a'"); + assertEquals(result, expect(0)); + } + + @Test + public void testSelectTagNotEqual() + { + // can't be pushed down + MaterializedResult result = execute("SELECT * FROM data WHERE k1 != 'd'"); + assertEquals(result, expect(0, 1)); + } + + @Test + public void testSelectField() + { + MaterializedResult result = execute("SELECT v1 FROM data WHERE k1 != 'd'"); + MaterializedResult expectedColumns = MaterializedResult + .resultBuilder(session, DOUBLE) + .row(1.) + .row(3.) + .build(); + assertEquals(result, expectedColumns); + } + + @Test + public void testSelectTagNull() + { + MaterializedResult result = execute("SELECT * FROM data WHERE k1 IS NULL"); + assertEquals(result, expect()); + } + + @Test + public void testSelectFieldNull() + { + // can't be pushed down + MaterializedResult result = execute("SELECT * FROM data WHERE v1 IS NULL"); + assertEquals(result, expect()); + } + + @Test + public void testSelectFieldEqual() + { + MaterializedResult result = execute("SELECT * FROM data WHERE v1 = 1"); + assertEquals(result, expect(0)); + } + + @Test + public void testSelectFieldBetween() + { + MaterializedResult result = execute("SELECT * FROM data WHERE v1 BETWEEN 1 AND 3"); + assertEquals(result, expect(0, 1)); + } + + private synchronized MaterializedResult execute(String sql) + { + return queryRunner.getClient().execute(session, sql) + .getResult() + .toTestTypes(); + } + + private MaterializedResult expect(int... rows) + { + MaterializedResult.Builder expected = MaterializedResult + .resultBuilder(session, TIMESTAMP_WITH_TIME_ZONE, VARCHAR, VARCHAR, DOUBLE, DOUBLE); + for (int row : rows) { + expected.row(TestingInfluxServer.getColumns(row)); + } + return expected.build(); + } + + @AfterClass + public void afterClass() + throws IOException + { + queryRunner.close(); + influxServer.close(); + } +} diff --git a/presto-influx/src/test/java/io/prestosql/plugin/influx/TestingInfluxServer.java b/presto-influx/src/test/java/io/prestosql/plugin/influx/TestingInfluxServer.java new file mode 100644 index 000000000000..704ed268e938 --- /dev/null +++ b/presto-influx/src/test/java/io/prestosql/plugin/influx/TestingInfluxServer.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.prestosql.plugin.influx; + +import org.testcontainers.containers.InfluxDBContainer; + +import java.time.Instant; +import java.time.ZoneId; +import java.time.ZonedDateTime; + +public class TestingInfluxServer +{ + public static final String DATABASE = "Test"; + public static final String RETENTION_POLICY = "Schema"; + public static final String MEASUREMENT = "Data"; + private static final Instant[] T = new Instant[] { + Instant.parse("2019-12-10T21:00:04.446Z"), + Instant.parse("2019-12-10T21:00:20.446Z"), + Instant.parse("2019-12-10T22:00:04.446Z"), + }; + private static final String[] K1 = new String[] {"a", "b", "d"}; + private static final String[] K2 = new String[] {"b", "c", "b"}; + private static final double[] V1 = new double[] {1, 3, 5}; + private static final double[] V2 = new double[] {2, 4, 6}; + + private final InfluxDBContainer dockerContainer; + + public TestingInfluxServer() + { + dockerContainer = new InfluxDBContainer() + .withDatabase(DATABASE) + .withAuthEnabled(false); + dockerContainer.start(); + + InfluxHttp http = new InfluxHttp(getHost(), getPort(), false, DATABASE, null, null); + //http.executeDDL("CREATE DATABASE " + DATABASE); + http.execute("CREATE RETENTION POLICY " + RETENTION_POLICY + " ON " + DATABASE + " DURATION INF REPLICATION 1"); + http.write(RETENTION_POLICY, getWrite(0), getWrite(1), getWrite(2)); + } + + public String getHost() + { + return dockerContainer.getContainerIpAddress(); + } + + public int getPort() + { + return dockerContainer.getMappedPort(InfluxDBContainer.INFLUXDB_PORT); + } + + public static String getWrite(int row) + { + return MEASUREMENT + ",k1=" + K1[row] + ",k2=" + K2[row] + " v1=" + V1[row] + ",v2=" + V2[row] + " " + T[row].toEpochMilli() + "000000"; + } + + public static Object[] getColumns(int row) + { + return new Object[] {ZonedDateTime.ofInstant(T[row], ZoneId.of("UTC")), K1[row], K2[row], V1[row], V2[row]}; + } + + public void close() + { + dockerContainer.close(); + } +} From 7069d3cf705cfc370c2f838e0d034bbb90239e02 Mon Sep 17 00:00:00 2001 From: Will Edwards Date: Sun, 5 Jan 2020 19:51:24 +0100 Subject: [PATCH 14/19] Tidied up tests re review --- .gitignore | 1 - .../prestosql/plugin/influx/InfluxHttp.java | 20 ++--- .../prestosql/plugin/influx/TestInfluxQL.java | 7 +- .../plugin/influx/TestInfluxSelect.java | 56 +++++++++--- .../plugin/influx/TestingInfluxServer.java | 41 +++------ .../src/test/resources/canned_http.json | 90 ------------------- 6 files changed, 71 insertions(+), 144 deletions(-) delete mode 100644 presto-influx/src/test/resources/canned_http.json diff --git a/.gitignore b/.gitignore index bdd19a780340..f9b1a4b5901c 100644 --- a/.gitignore +++ b/.gitignore @@ -18,7 +18,6 @@ temp-testng-customsuite.xml test-output .externalToolBuilders *~ -\#* benchmark_outputs *.pyc *.class diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxHttp.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxHttp.java index 567d2749d988..c4f03456bd3e 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxHttp.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxHttp.java @@ -95,15 +95,15 @@ public void write(String retentionPolicy, String... lines) .put("db", database) .put("rp", retentionPolicy) .build()); - HttpURLConnection conn = (HttpURLConnection) url.openConnection(); - conn.setRequestMethod("POST"); - conn.setDoOutput(true); - conn.setFixedLengthStreamingMode(body.length); - conn.connect(); - try (OutputStream out = conn.getOutputStream()) { + HttpURLConnection connection = (HttpURLConnection) url.openConnection(); + connection.setRequestMethod("POST"); + connection.setDoOutput(true); + connection.setFixedLengthStreamingMode((long) body.length); + connection.connect(); + try (OutputStream out = connection.getOutputStream()) { out.write(body); } - checkSuccess(conn); + checkSuccess(connection); } catch (Throwable t) { InfluxError.EXTERNAL.fail(t); @@ -133,13 +133,13 @@ private URL getUrl(String path, Map parameters) query); } - private void checkSuccess(HttpURLConnection conn) + private void checkSuccess(HttpURLConnection connection) throws IOException { - int responseCode = conn.getResponseCode(); + int responseCode = connection.getResponseCode(); if (responseCode < 200 || responseCode >= 300) { String errorMessage = ""; - InputStream err = conn.getErrorStream(); + InputStream err = connection.getErrorStream(); if (err != null) { try (Scanner scanner = new Scanner(err)) { scanner.useDelimiter("\\Z"); diff --git a/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxQL.java b/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxQL.java index fac3f6ed9973..e4e9f3b5ecd3 100644 --- a/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxQL.java +++ b/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxQL.java @@ -24,10 +24,13 @@ public class TestInfluxQL public void test() { InfluxQL test = new InfluxQL(); + test.addIdentifier("hello").append(" = ").add("world"); - assertEquals("hello = 'world'", test.toString()); + assertEquals(test.toString(), "hello = 'world'"); + test.truncate(0); + test.addIdentifier("åäö").append(" = ").add("\"åäö'"); - assertEquals("\"åäö\" = '\"åäö\\''", test.toString()); + assertEquals(test.toString(), "\"åäö\" = '\"åäö\\''"); } } diff --git a/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxSelect.java b/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxSelect.java index 9169c0b9e57f..a392f5851b79 100644 --- a/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxSelect.java +++ b/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxSelect.java @@ -23,7 +23,9 @@ import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; -import java.io.IOException; +import java.time.Instant; +import java.time.ZoneId; +import java.time.ZonedDateTime; import java.util.Collections; import java.util.Optional; import java.util.stream.Collectors; @@ -46,6 +48,7 @@ public void beforeClass() throws Exception { influxServer = new TestingInfluxServer(); + TestData.initServer(influxServer); // create a query runner with a real influx connector queryRunner = DistributedQueryRunner.builder(testSessionBuilder() @@ -73,10 +76,10 @@ public void testShow() MaterializedResult expectedColumns = MaterializedResult .resultBuilder(session, VARCHAR, VARCHAR, VARCHAR, VARCHAR) .row("time", "timestamp with time zone", "time", "") - .row("k1", "varchar", "tag", "") - .row("k2", "varchar", "tag", "") - .row("v1", "double", "field", "") - .row("v2", "double", "field", "") + .row("tag1", "varchar", "tag", "") + .row("tag2", "varchar", "tag", "") + .row("field1", "double", "field", "") + .row("field2", "double", "field", "") .build(); assertEquals(result, expectedColumns); } @@ -91,7 +94,7 @@ public void testSelectAll() @Test public void testSelectTagEqual() { - MaterializedResult result = execute("SELECT * FROM data WHERE k1 = 'a'"); + MaterializedResult result = execute("SELECT * FROM data WHERE tAG1 = 'a'"); assertEquals(result, expect(0)); } @@ -99,14 +102,14 @@ public void testSelectTagEqual() public void testSelectTagNotEqual() { // can't be pushed down - MaterializedResult result = execute("SELECT * FROM data WHERE k1 != 'd'"); + MaterializedResult result = execute("SELECT * FROM data WHERE tAG1 != 'd'"); assertEquals(result, expect(0, 1)); } @Test public void testSelectField() { - MaterializedResult result = execute("SELECT v1 FROM data WHERE k1 != 'd'"); + MaterializedResult result = execute("SELECT fIELD1 FROM data WHERE tAG1 != 'd'"); MaterializedResult expectedColumns = MaterializedResult .resultBuilder(session, DOUBLE) .row(1.) @@ -118,7 +121,7 @@ public void testSelectField() @Test public void testSelectTagNull() { - MaterializedResult result = execute("SELECT * FROM data WHERE k1 IS NULL"); + MaterializedResult result = execute("SELECT * FROM data WHERE tAG1 IS NULL"); assertEquals(result, expect()); } @@ -126,21 +129,21 @@ public void testSelectTagNull() public void testSelectFieldNull() { // can't be pushed down - MaterializedResult result = execute("SELECT * FROM data WHERE v1 IS NULL"); + MaterializedResult result = execute("SELECT * FROM data WHERE fIELD1 IS NULL"); assertEquals(result, expect()); } @Test public void testSelectFieldEqual() { - MaterializedResult result = execute("SELECT * FROM data WHERE v1 = 1"); + MaterializedResult result = execute("SELECT * FROM data WHERE fIELD1 = 1"); assertEquals(result, expect(0)); } @Test public void testSelectFieldBetween() { - MaterializedResult result = execute("SELECT * FROM data WHERE v1 BETWEEN 1 AND 3"); + MaterializedResult result = execute("SELECT * FROM data WHERE fIELD1 BETWEEN 1 AND 3"); assertEquals(result, expect(0, 1)); } @@ -156,16 +159,41 @@ private MaterializedResult expect(int... rows) MaterializedResult.Builder expected = MaterializedResult .resultBuilder(session, TIMESTAMP_WITH_TIME_ZONE, VARCHAR, VARCHAR, DOUBLE, DOUBLE); for (int row : rows) { - expected.row(TestingInfluxServer.getColumns(row)); + expected.row(TestData.getColumns(row)); } return expected.build(); } @AfterClass public void afterClass() - throws IOException { queryRunner.close(); influxServer.close(); } + + private static class TestData + { + private static final Instant[] T = new Instant[] { + Instant.parse("2019-12-10T21:00:04.446Z"), + Instant.parse("2019-12-10T21:00:20.446Z"), + Instant.parse("2019-12-10T22:00:04.446Z"), + }; + private static final String[] TAG1 = new String[] {"a", "b", "d"}; + private static final String[] TAG2 = new String[] {"b", "c", "b"}; + private static final double[] FIELD1 = new double[] {1, 3, 5}; + private static final double[] FIELD2 = new double[] {2, 4, 6}; + + private static void initServer(TestingInfluxServer server) + { + for (int row = 0; row < 3; row++) { + String line = String.format("%s,tag1=%s,tag2=%s field1=%f,field2=%f %d000000", TestingInfluxServer.MEASUREMENT, TAG1[row], TAG2[row], FIELD1[row], FIELD2[row], T[row].toEpochMilli()); + server.getInfluxClient().write(TestingInfluxServer.RETENTION_POLICY, line); + } + } + + private static Object[] getColumns(int row) + { + return new Object[] {ZonedDateTime.ofInstant(T[row], ZoneId.of("UTC")), TAG1[row], TAG2[row], FIELD1[row], FIELD2[row]}; + } + } } diff --git a/presto-influx/src/test/java/io/prestosql/plugin/influx/TestingInfluxServer.java b/presto-influx/src/test/java/io/prestosql/plugin/influx/TestingInfluxServer.java index 704ed268e938..8c35c522109e 100644 --- a/presto-influx/src/test/java/io/prestosql/plugin/influx/TestingInfluxServer.java +++ b/presto-influx/src/test/java/io/prestosql/plugin/influx/TestingInfluxServer.java @@ -15,26 +15,19 @@ import org.testcontainers.containers.InfluxDBContainer; -import java.time.Instant; -import java.time.ZoneId; -import java.time.ZonedDateTime; +import java.io.Closeable; + +import static org.testcontainers.containers.InfluxDBContainer.INFLUXDB_PORT; public class TestingInfluxServer + implements Closeable { public static final String DATABASE = "Test"; public static final String RETENTION_POLICY = "Schema"; public static final String MEASUREMENT = "Data"; - private static final Instant[] T = new Instant[] { - Instant.parse("2019-12-10T21:00:04.446Z"), - Instant.parse("2019-12-10T21:00:20.446Z"), - Instant.parse("2019-12-10T22:00:04.446Z"), - }; - private static final String[] K1 = new String[] {"a", "b", "d"}; - private static final String[] K2 = new String[] {"b", "c", "b"}; - private static final double[] V1 = new double[] {1, 3, 5}; - private static final double[] V2 = new double[] {2, 4, 6}; private final InfluxDBContainer dockerContainer; + private final InfluxHttp influxClient; public TestingInfluxServer() { @@ -43,32 +36,26 @@ public TestingInfluxServer() .withAuthEnabled(false); dockerContainer.start(); - InfluxHttp http = new InfluxHttp(getHost(), getPort(), false, DATABASE, null, null); - //http.executeDDL("CREATE DATABASE " + DATABASE); - http.execute("CREATE RETENTION POLICY " + RETENTION_POLICY + " ON " + DATABASE + " DURATION INF REPLICATION 1"); - http.write(RETENTION_POLICY, getWrite(0), getWrite(1), getWrite(2)); - } - - public String getHost() - { - return dockerContainer.getContainerIpAddress(); + influxClient = new InfluxHttp(getHost(), getPort(), false, DATABASE, null, null); + influxClient.execute("CREATE RETENTION POLICY " + RETENTION_POLICY + " ON " + DATABASE + " DURATION INF REPLICATION 1"); } - public int getPort() + public InfluxHttp getInfluxClient() { - return dockerContainer.getMappedPort(InfluxDBContainer.INFLUXDB_PORT); + return influxClient; } - public static String getWrite(int row) + public String getHost() { - return MEASUREMENT + ",k1=" + K1[row] + ",k2=" + K2[row] + " v1=" + V1[row] + ",v2=" + V2[row] + " " + T[row].toEpochMilli() + "000000"; + return dockerContainer.getContainerIpAddress(); } - public static Object[] getColumns(int row) + public int getPort() { - return new Object[] {ZonedDateTime.ofInstant(T[row], ZoneId.of("UTC")), K1[row], K2[row], V1[row], V2[row]}; + return dockerContainer.getMappedPort(INFLUXDB_PORT); } + @Override public void close() { dockerContainer.close(); diff --git a/presto-influx/src/test/resources/canned_http.json b/presto-influx/src/test/resources/canned_http.json deleted file mode 100644 index 1099ebafed97..000000000000 --- a/presto-influx/src/test/resources/canned_http.json +++ /dev/null @@ -1,90 +0,0 @@ -{ - "queries": { - "SHOW RETENTION POLICIES": { - "results": [{ - "series": [{ - "columns": ["name", "duration", "shardGroupDuration", "replicaN", "default"], - "values": [["autogen", "0s", "168h0m0s", 1, true], ["SCHEMA", "0s", "168h0m0s", 3, false]] - }] - }] - }, - "SHOW MEASUREMENTS": { - "results": [{ - "series": [{ - "name": "measurements", - "columns": ["name"], - "values": [["Data"]] - }] - }] - }, - "SHOW FIELD KEYS FROM SCHEMA.Data": { - "results": [{ - "series": [{ - "name": "Data", - "columns": ["fieldKey", "fieldType"], - "values": [["v1", "float"], ["v2","float"]] - }] - }] - }, - "SHOW TAG KEYS FROM Data": { - "results": [{ - "series": [{ - "name": "Data", - "columns": ["tagKey"], - "values": [["k1"],["k2"]] - }] - }] - }, - "SELECT * FROM SCHEMA.Data": { - "results": [{ - "series": [{ - "name": "Data", - "columns": ["time","v1","v2","k1","k2"], - "values": [ - ["2019-12-10T21:00:04.446017Z", 1, 2, "a", "b"], - ["2019-12-10T21:00:20.446017Z", 3, 4, "a", "c"], - ["2019-12-10T22:00:04.446017Z", 5, 6, "d", "b"] - ] - }] - }] - }, - "SELECT * FROM SCHEMA.Data WHERE ((k1 = 'a'))": { - "results": [{ - "series": [{ - "name": "Data", - "columns": ["time","v1","v2","k1","k2"], - "values": [ - ["2019-12-10T21:00:04.446017Z", 1, 2, "a", "b"], - ["2019-12-10T21:00:20.446017Z", 3, 4, "a", "c"] - ] - }] - }] - }, - "SELECT * FROM SCHEMA.Data WHERE (k1 = '')": { - "results":[{}] - }, - "SELECT * FROM SCHEMA.Data WHERE ((v1 = 1.0))": { - "results": [{ - "series": [{ - "name": "Data", - "columns": ["time","v1","v2","k1","k2"], - "values": [ - ["2019-12-10T21:00:04.446017Z", 1, 2, "a", "b"] - ] - }] - }] - }, - "SELECT * FROM SCHEMA.Data WHERE ((v1 >= 1.0 AND v1 <= 3.0))": { - "results": [{ - "series": [{ - "name": "Data", - "columns": ["time","v1","v2","k1","k2"], - "values": [ - ["2019-12-10T21:00:04.446017Z", 1, 2, "a", "b"], - ["2019-12-10T21:00:20.446017Z", 3, 4, "a", "c"] - ] - }] - }] - } - } -} \ No newline at end of file From 1e0440b234f8e053e85e496b8626085fed2950ae Mon Sep 17 00:00:00 2001 From: Will Edwards Date: Mon, 6 Jan 2020 10:31:37 +0100 Subject: [PATCH 15/19] Influx: add integer field --- .../prestosql/plugin/influx/TestInfluxSelect.java | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxSelect.java b/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxSelect.java index a392f5851b79..ce7144edee14 100644 --- a/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxSelect.java +++ b/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxSelect.java @@ -31,6 +31,7 @@ import java.util.stream.Collectors; import static io.prestosql.spi.security.SelectedRole.Type.ROLE; +import static io.prestosql.spi.type.BigintType.BIGINT; import static io.prestosql.spi.type.DoubleType.DOUBLE; import static io.prestosql.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; import static io.prestosql.spi.type.VarcharType.VARCHAR; @@ -78,7 +79,7 @@ public void testShow() .row("time", "timestamp with time zone", "time", "") .row("tag1", "varchar", "tag", "") .row("tag2", "varchar", "tag", "") - .row("field1", "double", "field", "") + .row("field1", "bigint", "field", "") .row("field2", "double", "field", "") .build(); assertEquals(result, expectedColumns); @@ -111,9 +112,9 @@ public void testSelectField() { MaterializedResult result = execute("SELECT fIELD1 FROM data WHERE tAG1 != 'd'"); MaterializedResult expectedColumns = MaterializedResult - .resultBuilder(session, DOUBLE) - .row(1.) - .row(3.) + .resultBuilder(session, BIGINT) + .row(1L) + .row(3L) .build(); assertEquals(result, expectedColumns); } @@ -157,7 +158,7 @@ private synchronized MaterializedResult execute(String sql) private MaterializedResult expect(int... rows) { MaterializedResult.Builder expected = MaterializedResult - .resultBuilder(session, TIMESTAMP_WITH_TIME_ZONE, VARCHAR, VARCHAR, DOUBLE, DOUBLE); + .resultBuilder(session, TIMESTAMP_WITH_TIME_ZONE, VARCHAR, VARCHAR, BIGINT, DOUBLE); for (int row : rows) { expected.row(TestData.getColumns(row)); } @@ -180,13 +181,13 @@ private static class TestData }; private static final String[] TAG1 = new String[] {"a", "b", "d"}; private static final String[] TAG2 = new String[] {"b", "c", "b"}; - private static final double[] FIELD1 = new double[] {1, 3, 5}; + private static final long[] FIELD1 = new long[] {1, 3, 5}; private static final double[] FIELD2 = new double[] {2, 4, 6}; private static void initServer(TestingInfluxServer server) { for (int row = 0; row < 3; row++) { - String line = String.format("%s,tag1=%s,tag2=%s field1=%f,field2=%f %d000000", TestingInfluxServer.MEASUREMENT, TAG1[row], TAG2[row], FIELD1[row], FIELD2[row], T[row].toEpochMilli()); + String line = String.format("%s,tag1=%s,tag2=%s field1=%di,field2=%f %d000000", TestingInfluxServer.MEASUREMENT, TAG1[row], TAG2[row], FIELD1[row], FIELD2[row], T[row].toEpochMilli()); server.getInfluxClient().write(TestingInfluxServer.RETENTION_POLICY, line); } } From aad380113e96cc716faa7904197946b65ea8de55 Mon Sep 17 00:00:00 2001 From: Will Edwards Date: Tue, 7 Jan 2020 18:51:32 +0100 Subject: [PATCH 16/19] Runs TCPH tests --- presto-influx/pom.xml | 22 ++ .../prestosql/plugin/influx/InfluxClient.java | 79 +++++- .../prestosql/plugin/influx/InfluxColumn.java | 40 ++- .../plugin/influx/InfluxColumnHandle.java | 9 +- .../prestosql/plugin/influx/InfluxConfig.java | 37 ++- .../prestosql/plugin/influx/InfluxHttp.java | 152 ------------ .../plugin/influx/InfluxMetadata.java | 17 +- .../io/prestosql/plugin/influx/InfluxQL.java | 18 ++ .../plugin/influx/InfluxRecordSet.java | 5 +- .../plugin/influx/InfluxTpchTestSupport.java | 135 ++++++++++ .../plugin/influx/InfluxQueryRunner.java | 232 ++++++++++++++++++ .../plugin/influx/TestInfluxDistributed.java | 43 ++++ .../prestosql/plugin/influx/TestInfluxQL.java | 6 + .../plugin/influx/TestInfluxSelect.java | 97 ++++---- .../plugin/influx/TestingInfluxServer.java | 63 ----- 15 files changed, 661 insertions(+), 294 deletions(-) delete mode 100644 presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxHttp.java create mode 100644 presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTpchTestSupport.java create mode 100644 presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxQueryRunner.java create mode 100644 presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxDistributed.java delete mode 100644 presto-influx/src/test/java/io/prestosql/plugin/influx/TestingInfluxServer.java diff --git a/presto-influx/pom.xml b/presto-influx/pom.xml index 8dcfcf49b8ee..2757af6e9c89 100644 --- a/presto-influx/pom.xml +++ b/presto-influx/pom.xml @@ -64,6 +64,16 @@ jackson-databind + + com.squareup.okhttp3 + okhttp + + + + io.airlift + units + + io.prestosql @@ -125,6 +135,18 @@ influxdb test + + + io.airlift.tpch + tpch + test + + + + io.prestosql + presto-tpch + test + diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java index eb171873c0ab..982352945daa 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxClient.java @@ -15,14 +15,25 @@ package io.prestosql.plugin.influx; import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; 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 io.prestosql.spi.type.TimestampWithTimeZoneType; +import io.prestosql.spi.type.Type; +import okhttp3.Credentials; +import okhttp3.OkHttpClient; +import okhttp3.Request; +import okhttp3.Response; +import okhttp3.ResponseBody; import javax.inject.Inject; +import java.net.URLEncoder; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -30,15 +41,18 @@ import java.util.Locale; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; import java.util.function.Supplier; +import static io.prestosql.spi.type.VarcharType.VARCHAR; import static java.util.Objects.requireNonNull; public class InfluxClient { final Logger logger; private final InfluxConfig config; - private final InfluxHttp http; + private final OkHttpClient httpClient; + private final String baseUrl; // the various metadata are cached for a configurable number of milliseconds so we don't hammer the server private final CachedMetaData> retentionPolicies; // schema name (lower-case) -> retention policy (case-sensitive) private final CachedMetaData> measurements; // table name (lower-case) -> measurement (case-sensitive) @@ -50,11 +64,23 @@ 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<>(); + OkHttpClient.Builder httpClientBuilder = new OkHttpClient.Builder() + .connectTimeout(config.getConnectionTimeout(), TimeUnit.SECONDS) + .writeTimeout(config.getWriteTimeout(), TimeUnit.SECONDS) + .readTimeout(config.getReadTimeout(), TimeUnit.SECONDS); + if (config.getUserName() != null) { + httpClientBuilder.authenticator((route, response) -> response + .request() + .newBuilder() + .header("Authorization", Credentials.basic(config.getUserName(), config.getPassword())) + .build()); + } + this.httpClient = httpClientBuilder.build(); + this.baseUrl = (config.isUseHttps() ? "https://" : "http://") + config.getHost() + ":" + config.getPort() + "/query?db=" + config.getDatabase(); } public Collection getSchemaNames() @@ -91,7 +117,7 @@ private Map getTags(String tableName) .toString(); ImmutableMap.Builder tags = new ImmutableMap.Builder<>(); for (Map.Entry name : showNames(query).entrySet()) { - tags.put(name.getKey(), new InfluxColumn(name.getValue(), "string", InfluxColumn.Kind.TAG)); + tags.put(name.getKey(), new InfluxColumn(name.getValue(), "string", VARCHAR, InfluxColumn.Kind.TAG, false)); } return tags.build(); })) @@ -121,7 +147,8 @@ private Map getFields(String schemaName, String tableName) 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)); + Type type = InfluxColumn.TYPES_MAPPING.get(influxType); + InfluxColumn collision = fields.put(name.toLowerCase(Locale.ENGLISH), new InfluxColumn(name, influxType, type, InfluxColumn.Kind.FIELD, false)); if (collision != null) { InfluxError.IDENTIFIER_CASE_SENSITIVITY.fail("identifier " + name + " collides with " + collision.getInfluxName(), query); } @@ -141,23 +168,30 @@ public boolean tableExistsInSchema(String schemaName, String tableName) public List getColumns(String schemaName, String tableName) { + List columns = InfluxTpchTestSupport.getColumns(config.getDatabase(), schemaName, tableName); + if (columns != null) { + return columns; + } + columns = new ArrayList<>(); Collection fields = getFields(schemaName, tableName).values(); if (fields.isEmpty()) { return Collections.emptyList(); } - ImmutableList.Builder columns = new ImmutableList.Builder<>(); - columns.add(InfluxColumn.TIME); + columns.add(new InfluxColumn("time", "time", TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE, InfluxColumn.Kind.TIME, false)); columns.addAll(getTags(tableName).values()); columns.addAll(fields); - return columns.build(); + return ImmutableList.copyOf(columns); } private Map showNames(String query) { Map 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); + if (series == null) { + return Collections.emptyMap(); + } + InfluxError.GENERAL.check(series.getNodeType().equals(JsonNodeType.ARRAY), "expecting an array, not " + series, query); + InfluxError.GENERAL.check(series.size() == 1, "expecting one element, not " + series, query); series = series.get(0); if (series.has("values")) { for (JsonNode row : series.get("values")) { @@ -173,7 +207,32 @@ private Map showNames(String query) JsonNode execute(String query) { - return http.query(query); + try { + Response response = httpClient.newCall(new Request + .Builder().url(baseUrl + "&q=" + URLEncoder.encode(query, StandardCharsets.UTF_8.toString())) + .build()) + .execute(); + final String responseBody; + try (ResponseBody body = response.body()) { + responseBody = body != null ? body.string() : null; + } + if (!response.isSuccessful() || responseBody == null) { + InfluxError.EXTERNAL.fail("cannot execute query", response.code(), query, responseBody != null ? responseBody : ""); + } + JsonNode results = new ObjectMapper() + .readTree(responseBody) + .get("results"); + InfluxError.GENERAL.check(results != null && results.size() == 1, "expecting one result", query); + JsonNode result = results.get(0); + if (result.has("error")) { + InfluxError.GENERAL.fail(result.get("error").asText(), query); + } + return result.get("series"); + } + catch (Throwable t) { + InfluxError.EXTERNAL.fail(t); + return null; + } } public HostAddress getHostAddress() diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumn.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumn.java index 263d3ff1a3b1..c458825377a8 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumn.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumn.java @@ -28,37 +28,40 @@ import java.util.Locale; import static com.google.common.base.MoreObjects.ToStringHelper; +import static java.util.Objects.requireNonNull; public class InfluxColumn extends ColumnMetadata { // map InfluxDB types to Presto types - private static final ImmutableMap TYPES_MAPPING = new ImmutableMap.Builder() + public static final ImmutableMap TYPES_MAPPING = new ImmutableMap.Builder() .put("string", VarcharType.VARCHAR) .put("boolean", BooleanType.BOOLEAN) .put("integer", BigintType.BIGINT) .put("float", DoubleType.DOUBLE) .put("time", TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE) .build(); - public static final InfluxColumn TIME = new InfluxColumn("time", "time", Kind.TIME); private final String influxName; private final String influxType; private final Kind kind; @JsonCreator - public InfluxColumn(@JsonProperty("influxName") String influxName, + public InfluxColumn( + @JsonProperty("influxName") String influxName, @JsonProperty("influxType") String influxType, - @JsonProperty("kind") Kind kind) + @JsonProperty("type") Type type, + @JsonProperty("kind") Kind kind, + @JsonProperty("hidden") boolean hidden) { super(influxName.toLowerCase(Locale.ENGLISH), - TYPES_MAPPING.get(influxType), + type, null, - kind.name().toLowerCase(Locale.ENGLISH), - false); - this.influxName = influxName; - this.influxType = influxType; - this.kind = kind; + null, + hidden); + this.influxName = requireNonNull(influxName); + this.influxType = requireNonNull(influxType); + this.kind = requireNonNull(kind); } @JsonProperty @@ -73,12 +76,26 @@ public String getInfluxType() return influxType; } + @JsonProperty + @Override + public Type getType() + { + return super.getType(); + } + @JsonProperty public Kind getKind() { return kind; } + @JsonProperty + @Override + public boolean isHidden() + { + return super.isHidden(); + } + protected ToStringHelper toStringHelper(Object self) { ToStringHelper helper = com.google.common.base.MoreObjects.toStringHelper(self) @@ -88,6 +105,9 @@ protected ToStringHelper toStringHelper(Object self) if (!getName().equals(getInfluxName())) { helper.add("influx-name", getInfluxName()); } + if (isHidden()) { + helper.addValue("hidden"); + } return helper; } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumnHandle.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumnHandle.java index 677ce46ae88c..ff34665bcb18 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumnHandle.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxColumnHandle.java @@ -17,6 +17,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import io.prestosql.spi.connector.ColumnHandle; +import io.prestosql.spi.type.Type; public class InfluxColumnHandle extends InfluxColumn @@ -30,9 +31,11 @@ public InfluxColumnHandle(@JsonProperty("retentionPolicy") String retentionPolic @JsonProperty("measurement") String measurement, @JsonProperty("influxName") String influxName, @JsonProperty("influxType") String influxType, - @JsonProperty("kind") Kind kind) + @JsonProperty("type") Type type, + @JsonProperty("kind") Kind kind, + @JsonProperty("hidden") boolean hidden) { - super(influxName, influxType, kind); + super(influxName, influxType, type, kind, hidden); this.retentionPolicy = retentionPolicy; this.measurement = measurement; } @@ -41,7 +44,7 @@ public InfluxColumnHandle(String retentionPolicy, String measurement, InfluxColumn column) { - this(retentionPolicy, measurement, column.getInfluxName(), column.getInfluxType(), column.getKind()); + this(retentionPolicy, measurement, column.getInfluxName(), column.getInfluxType(), column.getType(), column.getKind(), column.isHidden()); } @JsonProperty diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConfig.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConfig.java index 93bf0a809497..93952f98ccb2 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConfig.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConfig.java @@ -27,8 +27,10 @@ public class InfluxConfig private String userName; private String password; private boolean useHttps; + private int connectionTimeout = 10; + private int writeTimeout = 10; + private int readTimeout = 10; - @NotNull public long getCacheMetaDataMillis() { return cacheMetaDataMillis; @@ -114,4 +116,37 @@ public InfluxConfig setUseHttps(boolean useHttps) this.useHttps = useHttps; return this; } + + public int getConnectionTimeout() + { + return connectionTimeout; + } + + @Config("influx.connection-timeout") + public void setConnectionTimeout(int connectionTimeout) + { + this.connectionTimeout = connectionTimeout; + } + + public int getWriteTimeout() + { + return writeTimeout; + } + + @Config("influx.write-timeout") + public void setWriteTimeout(int writeTimeout) + { + this.writeTimeout = writeTimeout; + } + + public int getReadTimeout() + { + return readTimeout; + } + + @Config("influx.read-timeout") + public void setReadTimeout(int readTimeout) + { + this.readTimeout = readTimeout; + } } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxHttp.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxHttp.java deleted file mode 100644 index c4f03456bd3e..000000000000 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxHttp.java +++ /dev/null @@ -1,152 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.prestosql.plugin.influx; - -import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableMap; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.UnsupportedEncodingException; -import java.net.HttpURLConnection; -import java.net.MalformedURLException; -import java.net.URL; -import java.net.URLEncoder; -import java.nio.charset.StandardCharsets; -import java.util.Map; -import java.util.Scanner; - -/* Using raw HTTP because the influx java library has dependency conflicts and puts the burden of quoting identifiers on the caller */ -public class InfluxHttp -{ - private final String host; - private final int port; - private final boolean useHttps; - private final String database; - private final String username; - private final String password; - - public InfluxHttp(String host, int port, boolean useHttps, String database, String username, String password) - { - this.host = host; - this.port = port; - this.useHttps = useHttps; - this.database = database; - this.username = username; - this.password = password; - } - - public void execute(String ddl) - { - try { - URL url = getUrl("query", - new ImmutableMap.Builder() - .put("q", ddl) - .build()); - checkSuccess((HttpURLConnection) url.openConnection()); - } - catch (Throwable t) { - InfluxError.EXTERNAL.fail(t); - } - } - - public JsonNode query(String query) - { - final JsonNode response; - try { - response = new ObjectMapper().readTree(getUrl("query", - new ImmutableMap.Builder() - .put("db", database) - .put("q", query) - .build())); - } - catch (Throwable t) { - InfluxError.EXTERNAL.fail(t); - return null; - } - JsonNode results = response.get("results"); - InfluxError.GENERAL.check(results != null && results.size() == 1, "expecting one result", query); - JsonNode result = results.get(0); - if (result.has("error")) { - InfluxError.GENERAL.fail(result.get("error").asText(), query); - } - return result.get("series"); - } - - public void write(String retentionPolicy, String... lines) - { - byte[] body = String.join("\n", lines).getBytes(StandardCharsets.UTF_8); - try { - URL url = getUrl("write", - new ImmutableMap.Builder() - .put("db", database) - .put("rp", retentionPolicy) - .build()); - HttpURLConnection connection = (HttpURLConnection) url.openConnection(); - connection.setRequestMethod("POST"); - connection.setDoOutput(true); - connection.setFixedLengthStreamingMode((long) body.length); - connection.connect(); - try (OutputStream out = connection.getOutputStream()) { - out.write(body); - } - checkSuccess(connection); - } - catch (Throwable t) { - InfluxError.EXTERNAL.fail(t); - } - } - - private URL getUrl(String path, Map parameters) - throws UnsupportedEncodingException, MalformedURLException - { - String authentication = username != null ? - username + (password != null ? - ":" + password : - "") + "@" : - ""; - StringBuilder query = new StringBuilder(); - for (Map.Entry parameter : parameters.entrySet()) { - query - .append(query.length() == 0 ? '?' : '&') - .append(parameter.getKey()) - .append('=') - .append(URLEncoder.encode(parameter.getValue(), StandardCharsets.UTF_8.toString())); - } - return new URL((useHttps ? "https://" : "http://") - + authentication - + host + ":" + port + - "/" + path + - query); - } - - private void checkSuccess(HttpURLConnection connection) - throws IOException - { - int responseCode = connection.getResponseCode(); - if (responseCode < 200 || responseCode >= 300) { - String errorMessage = ""; - InputStream err = connection.getErrorStream(); - if (err != null) { - try (Scanner scanner = new Scanner(err)) { - scanner.useDelimiter("\\Z"); - errorMessage += scanner.next(); - } - } - InfluxError.EXTERNAL.fail("error writing to influx (" + responseCode + ")", errorMessage); - } - } -} diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java index 47d8c7dd9a47..e1c1fbd6aea8 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxMetadata.java @@ -79,9 +79,6 @@ public List listTables(ConnectorSession session, Optional builder = ImmutableList.builder(); for (String tableName : client.getTableNames()) { for (String matchingSchemaName : schemaNames) { @@ -227,8 +224,8 @@ public Optional> applyFilter(C throw new UnhandledFilterException("time low bound"); } } - else if (!(value instanceof Number)) { - throw new UnhandledFilterException("tag comparision low bound"); + if (!(value instanceof Number)) { + throw new UnhandledFilterException("non-numeric comparision low bound"); } where.add(column).append(low).add(value); hasLow = true; @@ -245,9 +242,6 @@ else if (!(value instanceof Number)) { default: throw new UnhandledFilterException("high bound"); } - if (hasLow) { - where.append(" AND "); - } Object value = range.getHigh().getValue(); if (column.getKind() == InfluxColumn.Kind.TIME) { if (value instanceof Long) { @@ -257,8 +251,11 @@ else if (!(value instanceof Number)) { throw new UnhandledFilterException("time high bound"); } } - else if (!(value instanceof Number)) { - throw new UnhandledFilterException("tag comparison high bound"); + if (!(value instanceof Number)) { + throw new UnhandledFilterException("non-numeric comparison high bound"); + } + if (hasLow) { + where.append(" AND "); } where.add(column).append(high).add(value); } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java index bece49c19ce3..c7cad4e6aecf 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxQL.java @@ -16,12 +16,27 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableSet; import io.airlift.slice.Slice; +import java.util.Locale; +import java.util.Set; + /* A query builder that takes care correctly quoting identifiers and string values */ public class InfluxQL { + private static final Set RESERVED_WORDS = ImmutableSet.copyOf( + ("all alter any as asc begin by create continuous " + + "database databases default delete desc destinations " + + "diagnostics distinct drop duration end every explain " + + "field for from grant grants group groups in inf " + + "insert into key keys kill limit show measurement " + + "measurements name offset on order password policy " + + "policies privileges queries query read replication resample " + + "retention revoke select series set shard shards slimit " + + "soffset stats subscription subscriptions tag to user" + + "users values where with write").split(" ")); private final StringBuilder influxQL; public InfluxQL() @@ -107,6 +122,9 @@ public InfluxQL addIdentifier(String identifier) char ch = identifier.charAt(i); safe = (ch >= 'A' && ch <= 'Z') || (ch >= 'a' && ch <= 'z') || (i > 0 && ch >= '0' && ch <= '9') || ch == '_'; } + if (safe) { + safe = !RESERVED_WORDS.contains(identifier.toLowerCase(Locale.ENGLISH)); + } if (safe) { influxQL.append(identifier); } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java index 12f287e34bd0..bf62965b9961 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxRecordSet.java @@ -69,7 +69,8 @@ public InfluxRecordSet(List columns, JsonNode results) value = null; } else { - switch (columns.get(slot).getInfluxType()) { + InfluxColumn column = columns.get(slot); + switch (column.getInfluxType()) { case "string": value = node.textValue(); break; @@ -87,7 +88,7 @@ public InfluxRecordSet(List columns, JsonNode results) value = DateTimeEncoding.packDateTimeWithZone(timestamp.toEpochMilli(), TimeZoneKey.UTC_KEY); break; default: - InfluxError.GENERAL.fail("cannot map " + node + " to " + columns.get(slot)); + InfluxError.GENERAL.fail("cannot map " + node, columns.get(slot), columns.get(slot).getInfluxType()); value = null; } } diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTpchTestSupport.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTpchTestSupport.java new file mode 100644 index 000000000000..340ed020983e --- /dev/null +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxTpchTestSupport.java @@ -0,0 +1,135 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.prestosql.plugin.influx; + +import io.prestosql.spi.type.BigintType; +import io.prestosql.spi.type.DateType; +import io.prestosql.spi.type.DoubleType; +import io.prestosql.spi.type.IntegerType; +import io.prestosql.spi.type.TimestampWithTimeZoneType; +import io.prestosql.spi.type.VarcharType; + +import java.util.Arrays; +import java.util.List; + +public class InfluxTpchTestSupport +{ + public static final String TEST_DATABASE = "_presto_influx_test_db"; + public static final String TPCH_SCHEMA = "tpch"; + + private InfluxTpchTestSupport() + { + } + + public static List getColumns(String databaseName, String schemaName, String tableName) + { + if (TEST_DATABASE.equals(databaseName) && TPCH_SCHEMA.equals(schemaName)) { + switch (tableName) { + case "customer": + return Arrays.asList( + time(), + key("custkey"), + varchar("name", 25), + varchar("address", 40), + key("nationkey"), + varchar("phone", 15), + real("acctbal"), + varchar("mktsegment", 10), + varchar("comment", 117)); + case "orders": + return Arrays.asList( + time(), + key("orderkey"), + key("custkey"), + varchar("orderstatus", 1), + real("totalprice"), + date("orderdate"), + varchar("orderpriority", 15), + varchar("clerk", 15), + integer("shippriority"), + varchar("comment", 79)); + case "region": + return Arrays.asList( + time(), + key("regionkey"), + varchar("name", 25), + varchar("comment", 152)); + case "nation": + return Arrays.asList( + time(), + key("nationkey"), + varchar("name", 25), + key("regionkey"), + varchar("comment", 152)); + case "lineitem": + return Arrays.asList( + time(), + key("orderkey"), + key("partkey"), + key("suppkey"), + integer("linenumber"), + real("quantity"), + real("extendedprice"), + real("discount"), + real("tax"), + varchar("returnflag", 1), + varchar("linestatus", 1), + date("shipdate"), + date("commitdate"), + date("receiptdate"), + varchar("shipinstruct", 25), + varchar("shipmode", 10), + varchar("comment", 44)); + default: + throw new UnsupportedOperationException("table " + tableName + " not supported"); + } + } + return null; + } + + private static InfluxColumn time() + { + return new InfluxColumn("time", "time", TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE, InfluxColumn.Kind.TIME, true); + } + + private static InfluxColumn key(String name) + { + return new InfluxColumn(name, "integer", BigintType.BIGINT, InfluxColumn.Kind.FIELD, false); + } + + private static InfluxColumn varchar(String name, int length) + { + return new InfluxColumn(name, "string", VarcharType.createVarcharType(length), InfluxColumn.Kind.FIELD, false); + } + + private static InfluxColumn bigint(String name) + { + return new InfluxColumn(name, "integer", BigintType.BIGINT, InfluxColumn.Kind.FIELD, false); + } + + private static InfluxColumn integer(String name) + { + return new InfluxColumn(name, "integer", IntegerType.INTEGER, InfluxColumn.Kind.FIELD, false); + } + + private static InfluxColumn date(String name) + { + return new InfluxColumn(name, "integer", DateType.DATE, InfluxColumn.Kind.FIELD, false); + } + + private static InfluxColumn real(String name) + { + return new InfluxColumn(name, "float", DoubleType.DOUBLE, InfluxColumn.Kind.FIELD, false); + } +} diff --git a/presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxQueryRunner.java b/presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxQueryRunner.java new file mode 100644 index 000000000000..fb27d86c3448 --- /dev/null +++ b/presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxQueryRunner.java @@ -0,0 +1,232 @@ +/* + * 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.google.common.collect.ImmutableMap; +import io.airlift.log.Logger; +import io.airlift.tpch.TpchTable; +import io.prestosql.Session; +import io.prestosql.metadata.QualifiedObjectName; +import io.prestosql.plugin.tpch.TpchPlugin; +import io.prestosql.spi.security.Identity; +import io.prestosql.spi.security.SelectedRole; +import io.prestosql.testing.DistributedQueryRunner; +import io.prestosql.testing.MaterializedResult; +import io.prestosql.testing.MaterializedRow; +import io.prestosql.testing.assertions.Assert; +import okhttp3.Credentials; +import okhttp3.MediaType; +import okhttp3.OkHttpClient; +import okhttp3.Request; +import okhttp3.RequestBody; +import org.testcontainers.containers.InfluxDBContainer; + +import java.io.Closeable; +import java.net.URLEncoder; +import java.nio.charset.StandardCharsets; +import java.time.LocalDate; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.TimeUnit; + +import static io.airlift.units.Duration.nanosSince; +import static io.prestosql.plugin.tpch.TpchMetadata.TINY_SCHEMA_NAME; +import static io.prestosql.spi.security.SelectedRole.Type.ROLE; +import static io.prestosql.testing.TestingSession.testSessionBuilder; +import static java.util.Locale.ENGLISH; +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.testcontainers.containers.InfluxDBContainer.INFLUXDB_PORT; + +public class InfluxQueryRunner + implements Closeable +{ + public static final String DATABASE = InfluxTpchTestSupport.TEST_DATABASE; + public static final String USERNAME = "testuser"; + public static final String PASSWORD = "testpass"; + public static final String ADMIN_USERNAME = "admin"; + public static final String ADMIN_PASSWORD = "root"; + private static final String CATALOG = "influx_tests"; + public static final String TPCH_SCHEMA = InfluxTpchTestSupport.TPCH_SCHEMA; + private final Logger logger = Logger.get("InfluxQueryRunner"); + private final InfluxDBContainer dockerContainer; + private final DistributedQueryRunner queryRunner; + private boolean initedTpch; + + public InfluxQueryRunner() + throws Exception + { + dockerContainer = new InfluxDBContainer() + .withDatabase(DATABASE) + .withAdmin(ADMIN_USERNAME) + .withAdminPassword(ADMIN_PASSWORD) + .withUsername(USERNAME) + .withPassword(PASSWORD); + dockerContainer.start(); + queryRunner = DistributedQueryRunner.builder(testSessionBuilder() + .setCatalog(CATALOG) + .setSchema(TPCH_SCHEMA) + .setIdentity(Identity.forUser("test") + .withRole(CATALOG, new SelectedRole(ROLE, Optional.of("admin"))) + .build()) + .build()).build(); + queryRunner.installPlugin(new InfluxPlugin()); + queryRunner.createCatalog(CATALOG, "influx", + new ImmutableMap.Builder() + .put("influx.host", getHost()) + .put("influx.port", Integer.toString(getPort())) + .put("influx.database", DATABASE) + .put("influx.username", USERNAME) + .put("influx.password", PASSWORD) + .put("influx.read-timeout", "60") + .build()); + } + + public DistributedQueryRunner getQueryRunner() + { + return queryRunner; + } + + public Session createSession(String schema) + { + return testSessionBuilder() + .setCatalog(CATALOG) + .setSchema(schema) + .build(); + } + + public String getHost() + { + return dockerContainer.getContainerIpAddress(); + } + + public int getPort() + { + return dockerContainer.getMappedPort(INFLUXDB_PORT); + } + + public void executeDDL(String query) + { + try { + String url = "http://" + getHost() + ":" + getPort() + "/query?db=" + DATABASE + "&q=" + URLEncoder.encode(query, StandardCharsets.UTF_8.toString()); + Assert.assertTrue( + new OkHttpClient.Builder() + .connectTimeout(10, TimeUnit.SECONDS) + .writeTimeout(10, TimeUnit.SECONDS) + .readTimeout(30, TimeUnit.SECONDS) + .build() + .newCall(new Request.Builder() + .url(url) + .addHeader("Authorization", Credentials.basic(ADMIN_USERNAME, ADMIN_PASSWORD)) + .build()) + .execute().isSuccessful()); + } + catch (Throwable t) { + Assert.fail("could not execute ddl " + query, t); + } + } + + public void createRetentionPolicy(String retentionPolicy) + { + executeDDL("CREATE RETENTION POLICY " + retentionPolicy + " ON " + DATABASE + " DURATION INF REPLICATION 1"); + } + + public void write(String retentionPolicy, String... points) + { + try { + String url = "http://" + getHost() + ":" + getPort() + "/write?db=" + DATABASE + "&rp=" + retentionPolicy; + Assert.assertTrue(new OkHttpClient.Builder() + .connectTimeout(10, TimeUnit.SECONDS) + .writeTimeout(10, TimeUnit.SECONDS) + .readTimeout(30, TimeUnit.SECONDS) + .build() + .newCall(new Request.Builder() + .url(url) + .addHeader("Authorization", Credentials.basic(USERNAME, PASSWORD)) + .post(RequestBody.create(MediaType.parse("text/plain; charset=utf-8"), String.join("\n", points))) + .build()) + .execute() + .isSuccessful()); + } + catch (Throwable t) { + Assert.fail("could not write points", t); + } + } + + public void initTpch() + throws Exception + { + if (!initedTpch) { + logger.info("loading tpch data..."); + long startTime = System.nanoTime(); + queryRunner.installPlugin(new TpchPlugin()); + queryRunner.createCatalog("tpch", "tpch"); + createRetentionPolicy(TPCH_SCHEMA); + for (TpchTable table : TpchTable.getTables()) { + QualifiedObjectName source = new QualifiedObjectName("tpch", TINY_SCHEMA_NAME, table.getTableName().toLowerCase(ENGLISH)); + MaterializedResult desc = queryRunner.execute("DESC " + source).toTestTypes(); + StringBuilder linePattern = new StringBuilder(table.getTableName()); + int columnId = 0; + Collection dateColumns = new ArrayList<>(); + for (MaterializedRow columnDefinition : desc.getMaterializedRows()) { + String columnName = columnDefinition.getField(0).toString(); + String columnType = columnDefinition.getField(1).toString(); + linePattern.append(columnId == 0 ? ' ' : ','); + linePattern.append(columnName).append('='); + if (columnType.contains("char")) { + linePattern.append("\"%s\""); + } + else if (columnType.contains("int")) { + linePattern.append("%di"); + } + else if (columnType.equals("date")) { + linePattern.append("%di"); + dateColumns.add(columnId); + } + else if (columnType.equals("double")) { + linePattern.append("%f"); + } + else { + throw new Exception("Unhandled column: " + source + " " + columnName + " is " + columnType); + } + columnId++; + } + linePattern.append(" %d"); // the timestamp after the fields + String sql = "SELECT * FROM " + source; + MaterializedResult data = queryRunner.execute(sql).toTestTypes(); + Collection points = new ArrayList<>(); + for (MaterializedRow row : data.getMaterializedRows()) { + List columns = new ArrayList<>(row.getFields()); + for (int dateColumn : dateColumns) { + columns.set(dateColumn, ((LocalDate) columns.get(dateColumn)).toEpochDay()); + } + columns.add(startTime + points.size()); // give every row a unique and incrementing timestamp + String point = String.format(linePattern.toString(), columns.toArray()); + points.add(point); + } + write(TPCH_SCHEMA, points.toArray(new String[] {})); + } + logger.info("loaded tpch in %s", nanosSince(startTime).toString(SECONDS)); + } + initedTpch = true; + } + + @Override + public void close() + { + queryRunner.close(); + dockerContainer.close(); + } +} diff --git a/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxDistributed.java b/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxDistributed.java new file mode 100644 index 000000000000..e23caa41d222 --- /dev/null +++ b/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxDistributed.java @@ -0,0 +1,43 @@ +/* + * 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 io.prestosql.testing.AbstractTestQueries; +import io.prestosql.testing.QueryRunner; +import org.testng.annotations.AfterClass; +import org.testng.annotations.Test; + +@Test +public class TestInfluxDistributed + extends AbstractTestQueries +{ + private InfluxQueryRunner queryRunner; + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + queryRunner = new InfluxQueryRunner(); + queryRunner.initTpch(); + return queryRunner.getQueryRunner(); + } + + @AfterClass(alwaysRun = true) + @Override + public void close() + { + super.close(); + queryRunner.close(); + } +} diff --git a/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxQL.java b/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxQL.java index e4e9f3b5ecd3..0d76807e25a1 100644 --- a/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxQL.java +++ b/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxQL.java @@ -32,5 +32,11 @@ public void test() test.addIdentifier("åäö").append(" = ").add("\"åäö'"); assertEquals(test.toString(), "\"åäö\" = '\"åäö\\''"); + + test.truncate(0); + + // from is a reserved word so must be quoted + test.addIdentifier("frOm").append(" = ").add("to"); + assertEquals(test.toString(), "\"frOm\" = 'to'"); } } diff --git a/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxSelect.java b/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxSelect.java index ce7144edee14..c2ce6e7fa126 100644 --- a/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxSelect.java +++ b/presto-influx/src/test/java/io/prestosql/plugin/influx/TestInfluxSelect.java @@ -13,58 +13,43 @@ */ package io.prestosql.plugin.influx; -import com.google.common.collect.ImmutableMap; import io.prestosql.Session; -import io.prestosql.spi.security.Identity; -import io.prestosql.spi.security.SelectedRole; -import io.prestosql.testing.DistributedQueryRunner; import io.prestosql.testing.MaterializedResult; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; +import java.io.Closeable; import java.time.Instant; import java.time.ZoneId; +import java.time.ZoneOffset; import java.time.ZonedDateTime; +import java.time.format.DateTimeFormatter; import java.util.Collections; -import java.util.Optional; import java.util.stream.Collectors; -import static io.prestosql.spi.security.SelectedRole.Type.ROLE; import static io.prestosql.spi.type.BigintType.BIGINT; import static io.prestosql.spi.type.DoubleType.DOUBLE; import static io.prestosql.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; import static io.prestosql.spi.type.VarcharType.VARCHAR; -import static io.prestosql.testing.TestingSession.testSessionBuilder; import static io.prestosql.testing.assertions.Assert.assertEquals; public class TestInfluxSelect + implements Closeable { - private TestingInfluxServer influxServer; - private DistributedQueryRunner queryRunner; + private static final DateTimeFormatter SQL_TIMESTAMP_FORMAT = DateTimeFormatter + .ofPattern("yyyy-MM-dd HH:mm:ss.SSS 'UTC'") + .withZone(ZoneOffset.UTC); + private InfluxQueryRunner queryRunner; private Session session; @BeforeClass public void beforeClass() throws Exception { - influxServer = new TestingInfluxServer(); - TestData.initServer(influxServer); - - // create a query runner with a real influx connector - queryRunner = DistributedQueryRunner.builder(testSessionBuilder() - .setIdentity(Identity.forUser("test") - .withRole("catalog", new SelectedRole(ROLE, Optional.of("admin"))) - .build()) - .build()).build(); - queryRunner.installPlugin(new InfluxPlugin()); - queryRunner.createCatalog("catalog", "influx", - new ImmutableMap.Builder() - .put("influx.host", influxServer.getHost()) - .put("influx.port", Integer.toString(influxServer.getPort())) - .put("influx.database", TestingInfluxServer.DATABASE) - .build()); - session = queryRunner.getDefaultSession(); + queryRunner = new InfluxQueryRunner(); + session = queryRunner.createSession("schema"); + TestData.init(queryRunner); } @Test @@ -72,15 +57,19 @@ public void testShow() { MaterializedResult result = execute("SHOW TABLES"); assertEquals(result.getOnlyColumn().collect(Collectors.toList()), Collections.singletonList("data")); + } - result = execute("DESC data"); + @Test + public void testDesc() + { + MaterializedResult result = execute("DESC data"); MaterializedResult expectedColumns = MaterializedResult .resultBuilder(session, VARCHAR, VARCHAR, VARCHAR, VARCHAR) - .row("time", "timestamp with time zone", "time", "") - .row("tag1", "varchar", "tag", "") - .row("tag2", "varchar", "tag", "") - .row("field1", "bigint", "field", "") - .row("field2", "double", "field", "") + .row("time", "timestamp with time zone", "", "") + .row("tag1", "varchar", "", "") + .row("tag2", "varchar", "", "") + .row("field1", "bigint", "", "") + .row("field2", "double", "", "") .build(); assertEquals(result, expectedColumns); } @@ -148,9 +137,35 @@ public void testSelectFieldBetween() assertEquals(result, expect(0, 1)); } + @Test + public void testTimeBetween() + { + MaterializedResult result = execute(String.format("SELECT * FROM data where time between timestamp '%s' and timestamp '%s'", + SQL_TIMESTAMP_FORMAT.format(TestData.T[0]), SQL_TIMESTAMP_FORMAT.format(TestData.T[2]))); + assertEquals(result, expect(0, 1, 2)); + } + + @Test + public void testTimeRange() + { + MaterializedResult result = execute(String.format("SELECT * FROM data where time > timestamp '%s' and time < timestamp '%s'", + SQL_TIMESTAMP_FORMAT.format(TestData.T[0]), SQL_TIMESTAMP_FORMAT.format(TestData.T[2]))); + assertEquals(result, expect(1)); + } + + @AfterClass + @Override + public void close() + { + if (queryRunner != null) { + queryRunner.close(); + queryRunner = null; + } + } + private synchronized MaterializedResult execute(String sql) { - return queryRunner.getClient().execute(session, sql) + return queryRunner.getQueryRunner().getClient().execute(session, sql) .getResult() .toTestTypes(); } @@ -165,15 +180,10 @@ private MaterializedResult expect(int... rows) return expected.build(); } - @AfterClass - public void afterClass() - { - queryRunner.close(); - influxServer.close(); - } - private static class TestData { + private static final String RETENTION_POLICY = "Schema"; + private static final String MEASUREMENT = "Data"; private static final Instant[] T = new Instant[] { Instant.parse("2019-12-10T21:00:04.446Z"), Instant.parse("2019-12-10T21:00:20.446Z"), @@ -184,11 +194,12 @@ private static class TestData private static final long[] FIELD1 = new long[] {1, 3, 5}; private static final double[] FIELD2 = new double[] {2, 4, 6}; - private static void initServer(TestingInfluxServer server) + private static void init(InfluxQueryRunner queryRunner) { + queryRunner.createRetentionPolicy(RETENTION_POLICY); for (int row = 0; row < 3; row++) { - String line = String.format("%s,tag1=%s,tag2=%s field1=%di,field2=%f %d000000", TestingInfluxServer.MEASUREMENT, TAG1[row], TAG2[row], FIELD1[row], FIELD2[row], T[row].toEpochMilli()); - server.getInfluxClient().write(TestingInfluxServer.RETENTION_POLICY, line); + String line = String.format("%s,tag1=%s,tag2=%s field1=%di,field2=%f %d000000", MEASUREMENT, TAG1[row], TAG2[row], FIELD1[row], FIELD2[row], T[row].toEpochMilli()); + queryRunner.write(RETENTION_POLICY, line); } } diff --git a/presto-influx/src/test/java/io/prestosql/plugin/influx/TestingInfluxServer.java b/presto-influx/src/test/java/io/prestosql/plugin/influx/TestingInfluxServer.java deleted file mode 100644 index 8c35c522109e..000000000000 --- a/presto-influx/src/test/java/io/prestosql/plugin/influx/TestingInfluxServer.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.prestosql.plugin.influx; - -import org.testcontainers.containers.InfluxDBContainer; - -import java.io.Closeable; - -import static org.testcontainers.containers.InfluxDBContainer.INFLUXDB_PORT; - -public class TestingInfluxServer - implements Closeable -{ - public static final String DATABASE = "Test"; - public static final String RETENTION_POLICY = "Schema"; - public static final String MEASUREMENT = "Data"; - - private final InfluxDBContainer dockerContainer; - private final InfluxHttp influxClient; - - public TestingInfluxServer() - { - dockerContainer = new InfluxDBContainer() - .withDatabase(DATABASE) - .withAuthEnabled(false); - dockerContainer.start(); - - influxClient = new InfluxHttp(getHost(), getPort(), false, DATABASE, null, null); - influxClient.execute("CREATE RETENTION POLICY " + RETENTION_POLICY + " ON " + DATABASE + " DURATION INF REPLICATION 1"); - } - - public InfluxHttp getInfluxClient() - { - return influxClient; - } - - public String getHost() - { - return dockerContainer.getContainerIpAddress(); - } - - public int getPort() - { - return dockerContainer.getMappedPort(INFLUXDB_PORT); - } - - @Override - public void close() - { - dockerContainer.close(); - } -} From b5569f6e156ec12143cfeb95cc28ddd5127f19a5 Mon Sep 17 00:00:00 2001 From: Will Edwards Date: Wed, 8 Jan 2020 14:24:32 +0100 Subject: [PATCH 17/19] Updated Influx documentation --- .../src/main/sphinx/connector/influx.rst | 32 +++++++++++-------- .../prestosql/plugin/influx/InfluxConfig.java | 2 +- 2 files changed, 19 insertions(+), 15 deletions(-) diff --git a/presto-docs/src/main/sphinx/connector/influx.rst b/presto-docs/src/main/sphinx/connector/influx.rst index 8d893a8b158d..55b133e7d14a 100644 --- a/presto-docs/src/main/sphinx/connector/influx.rst +++ b/presto-docs/src/main/sphinx/connector/influx.rst @@ -19,25 +19,29 @@ 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 -================================================== ====================================================================== +=================================== =============== ====================================================================== +Property Name Default Comment +=================================== =============== ====================================================================== +``connector.name=`` e.g. influx +``influx.host=`` localhost +``influx.port=`` 8086 +``influx.connection-timeout=`` 10 Seconds +``influx.write-timeout=`` 10 +``influx.read-timeout=`` 180 +``influx.use-https=`` 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=`` Optional +``influx.password=`` Option +``influx.cache-meta-data-millis=`` 10000 How long to cache schema info e.g. measurement names before refreshing +=================================== =============== ====================================================================== Limitations ----------- * Only SELECT queries are supported +* Performance beware: complicated filter expressions and aggregation are not pushed down; it is likely you are retrieving + all the information from the InfluxDB server and then filtering and aggregating that large data-set inside Presto * 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 diff --git a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConfig.java b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConfig.java index 93952f98ccb2..fbc50961be2d 100644 --- a/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConfig.java +++ b/presto-influx/src/main/java/io/prestosql/plugin/influx/InfluxConfig.java @@ -29,7 +29,7 @@ public class InfluxConfig private boolean useHttps; private int connectionTimeout = 10; private int writeTimeout = 10; - private int readTimeout = 10; + private int readTimeout = 180; public long getCacheMetaDataMillis() { From f98ef7914bd9a4a95917b18c5aa78fecaaa4afbb Mon Sep 17 00:00:00 2001 From: William Edwards Date: Wed, 22 Jan 2020 07:11:22 +0100 Subject: [PATCH 18/19] Update presto-influx/pom.xml change version to 329 Co-Authored-By: Yuya Ebihara --- presto-influx/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/presto-influx/pom.xml b/presto-influx/pom.xml index 2757af6e9c89..514f5d5e75cc 100644 --- a/presto-influx/pom.xml +++ b/presto-influx/pom.xml @@ -7,7 +7,7 @@ presto-root io.prestosql - 327-SNAPSHOT + 329-SNAPSHOT presto-influx From bafcaa2db61f099981b0aef1dbdb2676389dc6fc Mon Sep 17 00:00:00 2001 From: Will Edwards Date: Fri, 24 Jan 2020 11:01:31 +0100 Subject: [PATCH 19/19] Merge presto master --- presto-docs/src/main/sphinx/connector.rst | 1 + .../src/main/sphinx/connector/influx.rst | 2 +- presto-influx/pom.xml | 18 ++++++++---------- .../plugin/influx/InfluxQueryRunner.java | 2 +- 4 files changed, 11 insertions(+), 12 deletions(-) diff --git a/presto-docs/src/main/sphinx/connector.rst b/presto-docs/src/main/sphinx/connector.rst index 29e8f4e0c5cb..409fabd91ca0 100644 --- a/presto-docs/src/main/sphinx/connector.rst +++ b/presto-docs/src/main/sphinx/connector.rst @@ -16,6 +16,7 @@ from different data sources. connector/hive connector/hive-gcs-tutorial connector/hive-security + connector/influx connector/jmx connector/kafka connector/kafka-tutorial diff --git a/presto-docs/src/main/sphinx/connector/influx.rst b/presto-docs/src/main/sphinx/connector/influx.rst index 55b133e7d14a..61c9af92dc80 100644 --- a/presto-docs/src/main/sphinx/connector/influx.rst +++ b/presto-docs/src/main/sphinx/connector/influx.rst @@ -32,7 +32,7 @@ Property Name Default Comment ``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=`` Optional -``influx.password=`` Option +``influx.password=`` Optional ``influx.cache-meta-data-millis=`` 10000 How long to cache schema info e.g. measurement names before refreshing =================================== =============== ====================================================================== diff --git a/presto-influx/pom.xml b/presto-influx/pom.xml index 514f5d5e75cc..fc2479e9425c 100644 --- a/presto-influx/pom.xml +++ b/presto-influx/pom.xml @@ -1,13 +1,11 @@ - + 4.0.0 - presto-root io.prestosql - 329-SNAPSHOT + presto-root + 330-SNAPSHOT presto-influx @@ -137,14 +135,14 @@ - io.airlift.tpch - tpch + io.prestosql + presto-tpch test - + - io.prestosql - presto-tpch + io.prestosql.tpch + tpch test diff --git a/presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxQueryRunner.java b/presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxQueryRunner.java index fb27d86c3448..7ece9a7b7aee 100644 --- a/presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxQueryRunner.java +++ b/presto-influx/src/test/java/io/prestosql/plugin/influx/InfluxQueryRunner.java @@ -15,7 +15,6 @@ import com.google.common.collect.ImmutableMap; import io.airlift.log.Logger; -import io.airlift.tpch.TpchTable; import io.prestosql.Session; import io.prestosql.metadata.QualifiedObjectName; import io.prestosql.plugin.tpch.TpchPlugin; @@ -25,6 +24,7 @@ import io.prestosql.testing.MaterializedResult; import io.prestosql.testing.MaterializedRow; import io.prestosql.testing.assertions.Assert; +import io.prestosql.tpch.TpchTable; import okhttp3.Credentials; import okhttp3.MediaType; import okhttp3.OkHttpClient;