From 1ceeac1b5eefd6ed03cd5c18160d8d3e4c5c82b0 Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Mon, 15 Mar 2021 10:44:25 -0700 Subject: [PATCH 01/32] implement AbstractTestQueries for integration tests Signed-off-by: Andrew Robertson --- .../presto/pravega/PravegaPlugin.java | 2 +- .../pravega/PravegaStreamDescription.java | 10 + .../pravega/PravegaStreamFieldGroup.java | 9 +- .../PravegaTableDescriptionSupplier.java | 333 ++--------------- .../pravega/util/PravegaStreamDescUtils.java | 346 ++++++++++++++++++ .../presto/pravega/CodecSupplier.java | 71 ++++ .../presto/pravega/DockerContainer.java | 254 +++++++++++++ .../presto/pravega/EmbeddedPravega.java | 54 +++ .../presto/pravega/KeyValueTable.java | 35 ++ .../presto/pravega/PravegaKeyValueLoader.java | 137 +++++++ .../presto/pravega/PravegaLoader.java | 228 ++++++++++++ .../presto/pravega/PravegaQueryRunner.java | 175 +++++++++ .../presto/pravega/PravegaTestUtils.java | 169 +++++++++ .../pravega/TestPravegaDistributed.java | 58 +++ .../presto/pravega/TestPravegaKeyValue.java | 86 +++++ .../presto/pravega/TestPravegaPlugin.java | 9 +- src/test/resources/kv/employee-id.avsc | 8 + src/test/resources/kv/employee-value.avsc | 9 + src/test/resources/kv/employee.json | 17 + src/test/resources/kv/employee.records | 3 + src/test/resources/tpch/customer.json | 50 +++ src/test/resources/tpch/lineitem.json | 93 +++++ src/test/resources/tpch/nation.json | 30 ++ src/test/resources/tpch/orders.json | 58 +++ src/test/resources/tpch/part.json | 55 +++ src/test/resources/tpch/partsupp.json | 35 ++ src/test/resources/tpch/region.json | 25 ++ src/test/resources/tpch/supplier.json | 45 +++ 28 files changed, 2089 insertions(+), 315 deletions(-) create mode 100644 src/main/java/com/facebook/presto/pravega/util/PravegaStreamDescUtils.java create mode 100644 src/test/java/com/facebook/presto/pravega/CodecSupplier.java create mode 100644 src/test/java/com/facebook/presto/pravega/DockerContainer.java create mode 100644 src/test/java/com/facebook/presto/pravega/EmbeddedPravega.java create mode 100644 src/test/java/com/facebook/presto/pravega/KeyValueTable.java create mode 100644 src/test/java/com/facebook/presto/pravega/PravegaKeyValueLoader.java create mode 100644 src/test/java/com/facebook/presto/pravega/PravegaLoader.java create mode 100644 src/test/java/com/facebook/presto/pravega/PravegaQueryRunner.java create mode 100644 src/test/java/com/facebook/presto/pravega/PravegaTestUtils.java create mode 100644 src/test/java/com/facebook/presto/pravega/TestPravegaDistributed.java create mode 100644 src/test/java/com/facebook/presto/pravega/TestPravegaKeyValue.java create mode 100644 src/test/resources/kv/employee-id.avsc create mode 100644 src/test/resources/kv/employee-value.avsc create mode 100644 src/test/resources/kv/employee.json create mode 100644 src/test/resources/kv/employee.records create mode 100644 src/test/resources/tpch/customer.json create mode 100644 src/test/resources/tpch/lineitem.json create mode 100644 src/test/resources/tpch/nation.json create mode 100644 src/test/resources/tpch/orders.json create mode 100644 src/test/resources/tpch/part.json create mode 100644 src/test/resources/tpch/partsupp.json create mode 100644 src/test/resources/tpch/region.json create mode 100644 src/test/resources/tpch/supplier.json diff --git a/src/main/java/com/facebook/presto/pravega/PravegaPlugin.java b/src/main/java/com/facebook/presto/pravega/PravegaPlugin.java index ec927c6..7103d1e 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaPlugin.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaPlugin.java @@ -26,7 +26,7 @@ import static java.util.Objects.requireNonNull; /** - * Presto plugin to use Apache Pravega as a data source. + * Presto plugin to use Pravega as a data source. */ public class PravegaPlugin implements Plugin diff --git a/src/main/java/com/facebook/presto/pravega/PravegaStreamDescription.java b/src/main/java/com/facebook/presto/pravega/PravegaStreamDescription.java index 503e107..1609f27 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaStreamDescription.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaStreamDescription.java @@ -57,6 +57,16 @@ public PravegaStreamDescription( this.event = requireNonNull(event, "message is null"); } + public PravegaStreamDescription(PravegaStreamDescription streamDescription, List event) + { + this.tableName = streamDescription.tableName; + this.schemaName = streamDescription.schemaName; + this.objectName = streamDescription.objectName; + this.objectType = streamDescription.objectType; + this.objectArgs = streamDescription.objectArgs; + this.event = Optional.of(event); + } + @JsonProperty public Optional getSchemaName() { diff --git a/src/main/java/com/facebook/presto/pravega/PravegaStreamFieldGroup.java b/src/main/java/com/facebook/presto/pravega/PravegaStreamFieldGroup.java index 6a6137a..d793e36 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaStreamFieldGroup.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaStreamFieldGroup.java @@ -13,7 +13,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package com.facebook.presto.pravega; import com.fasterxml.jackson.annotation.JsonCreator; @@ -48,6 +47,14 @@ public PravegaStreamFieldGroup( this.fields = fields; } + public PravegaStreamFieldGroup(PravegaStreamFieldGroup fieldGroup, String dataSchema, List fields) + { + this.dataFormat = fieldGroup.dataFormat; + this.mapping = fieldGroup.mapping; + this.dataSchema = Optional.of(dataSchema); + this.fields = Optional.of(fields); + } + @JsonProperty public String getDataFormat() { diff --git a/src/main/java/com/facebook/presto/pravega/PravegaTableDescriptionSupplier.java b/src/main/java/com/facebook/presto/pravega/PravegaTableDescriptionSupplier.java index e8fc00c..6e49cf4 100644 --- a/src/main/java/com/facebook/presto/pravega/PravegaTableDescriptionSupplier.java +++ b/src/main/java/com/facebook/presto/pravega/PravegaTableDescriptionSupplier.java @@ -13,19 +13,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package com.facebook.presto.pravega; import com.facebook.airlift.json.JsonCodec; import com.facebook.airlift.log.Logger; -import com.facebook.presto.common.type.Type; import com.facebook.presto.spi.SchemaTableName; +import com.google.common.annotations.VisibleForTesting; import com.google.common.cache.Cache; import com.google.common.cache.CacheBuilder; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Streams; -import com.google.protobuf.Descriptors; import io.pravega.client.ClientConfig; import io.pravega.client.admin.StreamManager; import io.pravega.client.stream.Stream; @@ -35,12 +33,6 @@ import io.pravega.schemaregistry.contract.data.GroupProperties; import io.pravega.schemaregistry.contract.data.SchemaWithVersion; import io.pravega.schemaregistry.contract.data.SerializationFormat; -import io.pravega.schemaregistry.serializer.json.schemas.JSONSchema; -import org.everit.json.schema.BooleanSchema; -import org.everit.json.schema.NumberSchema; -import org.everit.json.schema.ObjectSchema; -import org.everit.json.schema.Schema; -import org.everit.json.schema.StringSchema; import javax.inject.Inject; @@ -55,15 +47,9 @@ import java.util.Locale; import java.util.Optional; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; import java.util.regex.Pattern; import java.util.stream.Collectors; -import static com.facebook.presto.common.type.BigintType.BIGINT; -import static com.facebook.presto.common.type.BooleanType.BOOLEAN; -import static com.facebook.presto.common.type.DoubleType.DOUBLE; -import static com.facebook.presto.common.type.VarbinaryType.VARBINARY; -import static com.facebook.presto.common.type.VarcharType.createUnboundedVarcharType; import static com.facebook.presto.pravega.ProtobufCommon.encodeSchema; import static com.facebook.presto.pravega.util.PravegaNameUtils.groupId; import static com.facebook.presto.pravega.util.PravegaNameUtils.kvFieldMapping; @@ -71,17 +57,15 @@ import static com.facebook.presto.pravega.util.PravegaNameUtils.multiSourceStream; import static com.facebook.presto.pravega.util.PravegaNameUtils.temp_streamNameToTableName; import static com.facebook.presto.pravega.util.PravegaNameUtils.temp_tableNameToStreamName; -import static com.facebook.presto.pravega.util.PravegaSchemaUtils.AVRO; import static com.facebook.presto.pravega.util.PravegaSchemaUtils.GROUP_PROPERTIES_INLINE_KEY; import static com.facebook.presto.pravega.util.PravegaSchemaUtils.GROUP_PROPERTIES_INLINE_KV_KEY; import static com.facebook.presto.pravega.util.PravegaSchemaUtils.GROUP_PROPERTIES_INLINE_KV_VALUE; import static com.facebook.presto.pravega.util.PravegaSchemaUtils.INLINE_SUFFIX; -import static com.facebook.presto.pravega.util.PravegaSchemaUtils.NESTED_RECORD_SEPARATOR; import static com.facebook.presto.pravega.util.PravegaSchemaUtils.readSchema; +import static com.facebook.presto.pravega.util.PravegaStreamDescUtils.mapFieldsFromSchema; import static java.nio.file.Files.readAllBytes; import static java.util.Arrays.asList; import static java.util.Objects.requireNonNull; -import static org.apache.avro.Schema.Type.RECORD; // pravega scope is a namespace for streams. stream is unique within scope. // presto schema is like a database, with collection of tables. @@ -130,6 +114,16 @@ public class PravegaTableDescriptionSupplier .build(); } + @VisibleForTesting + PravegaTableDescriptionSupplier(PravegaConnectorConfig pravegaConnectorConfig, + Cache schemaCache, + Cache> tableCache) + { + this.pravegaConnectorConfig = pravegaConnectorConfig; + this.schemaCache = schemaCache; + this.tableCache = tableCache; + } + public List listSchemas() { // if any expired, retrieve list again from pravega @@ -341,6 +335,12 @@ else if (kvTable(table)) { return table; } + /** + * construct PravegaStreamFieldGroup by looking up schema in schema registry + * + * @param schemaTableName + * @return + */ private Optional> fieldGroupsFromSchemaRegistry(final SchemaTableName schemaTableName) { log.info("look up description of '%s' from pravega", schemaTableName); @@ -356,19 +356,19 @@ private Optional> fieldGroupsFromSchemaRegistry(fi GroupProperties properties = registryClient.getGroupProperties(groupName); - List schemas = - registryClient.getSchemas(groupName); + List schemas = registryClient.getSchemas(groupName); if (schemas.size() == 0 || schemas.size() > 2) { throw new IllegalStateException(schemaTableName + " has " + schemas.size() + " registered schemas. expecting either 1 or 2"); } - for (int i = 0; i < schemas.size(); i++) { - SerializationFormat format = schemas.get(i).getSchemaInfo().getSerializationFormat(); + // kv table will have > 1 schema. key+value likely different types + boolean kv = schemas.size() > 1; - // kv table will have > 1 schema. key+value likely different types + for (int i = 0; i < schemas.size(); i++) { // colPrefix used for display so can differentiate between fields from key or value - boolean kv = schemas.size() > 1; String colPrefix = kv ? kvFieldMapping(i) : ""; + + SerializationFormat format = schemas.get(i).getSchemaInfo().getSerializationFormat(); fieldGroups.add(new PravegaStreamFieldGroup( dataFormat(properties.getProperties(), format, kv, i), Optional.of(colPrefix), @@ -475,291 +475,6 @@ hopefully this can all go away (see linked issue 58 above) return finalFormat + (groupProperties.containsKey(key) ? INLINE_SUFFIX : ""); } - /** - * map protobuf java type -> presto sql type - * - * @param fieldDescriptor - * @return - */ - private static Type typeFromSchema(Descriptors.FieldDescriptor fieldDescriptor) - { - switch (fieldDescriptor.getJavaType()) { - case STRING: - return createUnboundedVarcharType(); - - case INT: - case LONG: - return BIGINT; - - case FLOAT: - case DOUBLE: - return DOUBLE; - - case BOOLEAN: - return BOOLEAN; - - case BYTE_STRING: - return VARBINARY; - - default: - throw new RuntimeException("unsupported type " + fieldDescriptor); - } - } - - /** - * map json schema type -> presto sql type - * - * @param schema - * @return - */ - private static Type typeFromSchema(Schema schema) - { - if (schema instanceof NumberSchema) { - return ((NumberSchema) schema).requiresInteger() - ? BIGINT - : DOUBLE; - } - else if (schema instanceof BooleanSchema) { - return BOOLEAN; - } - else if (schema instanceof StringSchema) { - return createUnboundedVarcharType(); - } - else { - throw new RuntimeException("unsupported schema " + schema); - } - } - - /** - * map avro schema type to presto sql type - * - * @param schema - * @return - */ - private static Type typeFromSchema(org.apache.avro.Schema schema) - { - // refer to AvroColumnDecoder#isSupportedType - - switch (schema.getType()) { - case FIXED: - case STRING: - return createUnboundedVarcharType(); - - case INT: - case LONG: - return BIGINT; - - case FLOAT: - case DOUBLE: - return DOUBLE; - - case BOOLEAN: - return BOOLEAN; - - case BYTES: - return VARBINARY; - - case MAP: - case ARRAY: - // TODO: ^^ handle these https://github.com/pravega/pravega-sql/issues/65 - - case RECORD: - case ENUM: - case UNION: - default: - throw new RuntimeException("unexpected type " + schema); - } - } - - /** - * return lists of common field definitions - * uses list of fields from provided schema; schema is different depending on serialization format - * - * @param format - * @param schemaWithVersion - * @return - */ - private static List mapFieldsFromSchema( - String namePrefix, - SerializationFormat format, - SchemaWithVersion schemaWithVersion) - { - switch (format) { - case Json: - ObjectSchema objectSchema = - (ObjectSchema) JSONSchema.from(schemaWithVersion.getSchemaInfo()).getSchema(); - return mapTable(namePrefix, new JsonSchema(objectSchema)); - - case Avro: - case Custom: // re: Custom - definition for schema itself Custom is always Avro (only custom impl. is csv) - org.apache.avro.Schema schema = - new org.apache.avro.Schema.Parser().parse( - new String(schemaWithVersion.getSchemaInfo().getSchemaData().array(), StandardCharsets.UTF_8)); - return mapTable(namePrefix, new AvroSchema(schema, format == SerializationFormat.Custom)); - - case Protobuf: - return mapTable(namePrefix, new ProtobufSchema(ProtobufCommon.descriptorFor(schemaWithVersion))); - - default: - throw new IllegalArgumentException("unexpected format " + format); - } - } - - private static List mapFieldsFromSchema(String namePrefix, String format, String schemaString) - { - // schemaString defined as human-readable string in local file. only avro supported now. - switch (format) { - case AVRO: - org.apache.avro.Schema schema = - new org.apache.avro.Schema.Parser().parse(schemaString); - return mapTable(namePrefix, new AvroSchema(schema, false)); - - default: - throw new UnsupportedOperationException("unexpected format " + format); - } - } - - private static class SchemaColumn - { - String name; - String mapping; - Type type; - - SchemaColumn(String name, String mapping, Type type) - { - this.name = name; - this.mapping = mapping; - this.type = type; - } - } - - static class SchemaWrapper - { - List fields = new ArrayList<>(); - } - - static class SchemaField - { - String name; - Type type; - boolean record; - SchemaWrapper schema; - int ordinalPosition; - - SchemaField(String name, Type type, boolean record, SchemaWrapper schema) - { - this(name, type, record, schema, -1); - } - - SchemaField(String name, Type type, boolean record, SchemaWrapper schema, int ordinalPosition) - { - this.name = name; - this.type = type; - this.record = record; - this.schema = schema; - this.ordinalPosition = ordinalPosition; - } - } - - static class JsonSchema - extends SchemaWrapper - { - JsonSchema(ObjectSchema schema) - { - schema.getPropertySchemas().forEach((key, value) -> { - boolean record = value instanceof ObjectSchema; - fields.add(new SchemaField(key, - record ? null : typeFromSchema(value), - record, - record ? new JsonSchema((ObjectSchema) value) : null)); - }); - } - } - - static class ProtobufSchema - extends SchemaWrapper - { - ProtobufSchema(Descriptors.Descriptor schema) - { - schema.getFields().forEach(f -> { - boolean record = f.getJavaType() == Descriptors.FieldDescriptor.JavaType.MESSAGE; - fields.add(new SchemaField(f.getJsonName(), - record ? null : typeFromSchema(f), - record, - record ? new ProtobufSchema(f.getMessageType()) : null)); - }); - } - } - - static class AvroSchema - extends SchemaWrapper - { - AvroSchema(org.apache.avro.Schema schema, boolean customCsv) - { - final AtomicInteger position = new AtomicInteger(); - schema.getFields().forEach(f -> { - boolean record = f.schema().getType() == RECORD; - fields.add(new SchemaField(f.name(), - record ? null : typeFromSchema(f.schema()), - record, - record ? new AvroSchema(f.schema(), customCsv) : null, - customCsv ? position.getAndIncrement() : -1)); - }); - } - } - - private static List mapTable(String namePrefix, SchemaWrapper schema) - { - return mapFieldsFromSchema(mapColumns(namePrefix, null /* mappingPrefix */, schema)); - } - - private static List mapColumns(String namePrefix, String mappingPrefix, SchemaWrapper schema) - { - List columnList = new ArrayList<>(); - schema.fields.forEach(field -> { - String name = nestedPrefixFor(namePrefix, field.name); - // for csv we use only position. for avro, json, etc, can be path into nested object - String mapping = field.ordinalPosition >= 0 - ? String.valueOf(field.ordinalPosition) - : nestedPrefixFor(mappingPrefix, field.name); - if (field.record) { - columnList.addAll(mapColumns(name, mapping, field.schema)); - } - else { - columnList.add(new SchemaColumn(name, mapping, field.type)); - } - }); - return columnList; - } - - private static String nestedPrefixFor(String prefix, String name) - { - // (record1, field1) -> record1/field1 - return prefix == null || prefix.isEmpty() - ? name - : prefix + NESTED_RECORD_SEPARATOR + name; - } - - /** - * create field description from list of name,mapping,type tuples. each pair is a field in the schema. - * @param schemaColumns - * @return - */ - static List mapFieldsFromSchema(List schemaColumns) - { - List fields = new ArrayList<>(); - schemaColumns.forEach(sc -> { - fields.add(new PravegaStreamFieldDescription(sc.name, - sc.type, - sc.mapping, - "", - null, - null, - false)); - }); - return fields; - } - private static Optional dataSchema(SerializationFormat format, SchemaWithVersion schemaWithVersion) { // it is intentional that nothing is returned for Custom diff --git a/src/main/java/com/facebook/presto/pravega/util/PravegaStreamDescUtils.java b/src/main/java/com/facebook/presto/pravega/util/PravegaStreamDescUtils.java new file mode 100644 index 0000000..afd234c --- /dev/null +++ b/src/main/java/com/facebook/presto/pravega/util/PravegaStreamDescUtils.java @@ -0,0 +1,346 @@ +/* + * Copyright (c) Pravega Authors. + * + * 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 com.facebook.presto.pravega.util; + +import com.facebook.presto.common.type.Type; +import com.facebook.presto.pravega.PravegaStreamFieldDescription; +import com.facebook.presto.pravega.ProtobufCommon; +import com.google.protobuf.Descriptors; +import io.pravega.schemaregistry.contract.data.SchemaWithVersion; +import io.pravega.schemaregistry.contract.data.SerializationFormat; +import io.pravega.schemaregistry.serializer.json.schemas.JSONSchema; +import org.everit.json.schema.BooleanSchema; +import org.everit.json.schema.NumberSchema; +import org.everit.json.schema.ObjectSchema; +import org.everit.json.schema.Schema; +import org.everit.json.schema.StringSchema; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; + +import static com.facebook.presto.common.type.BigintType.BIGINT; +import static com.facebook.presto.common.type.BooleanType.BOOLEAN; +import static com.facebook.presto.common.type.DoubleType.DOUBLE; +import static com.facebook.presto.common.type.VarbinaryType.VARBINARY; +import static com.facebook.presto.common.type.VarcharType.createUnboundedVarcharType; +import static com.facebook.presto.pravega.util.PravegaSchemaUtils.AVRO; +import static com.facebook.presto.pravega.util.PravegaSchemaUtils.NESTED_RECORD_SEPARATOR; +import static org.apache.avro.Schema.Type.RECORD; + +/** + * a collection of methods to help create PravegaStreamDescription's + * including mapping + */ +public class PravegaStreamDescUtils +{ + private PravegaStreamDescUtils() {} + + // SchemaColumn, SchemaField, SchemaWrapper, + private static class SchemaColumn + { + String name; + String mapping; + Type type; + + SchemaColumn(String name, String mapping, Type type) + { + this.name = name; + this.mapping = mapping; + this.type = type; + } + } + + static class SchemaWrapper + { + List fields = new ArrayList<>(); + } + + static class SchemaField + { + String name; + Type type; + boolean record; + SchemaWrapper schema; + int ordinalPosition; + + SchemaField(String name, Type type, boolean record, SchemaWrapper schema) + { + this(name, type, record, schema, -1); + } + + SchemaField(String name, Type type, boolean record, SchemaWrapper schema, int ordinalPosition) + { + this.name = name; + this.type = type; + this.record = record; + this.schema = schema; + this.ordinalPosition = ordinalPosition; + } + } + + static class JsonSchema + extends SchemaWrapper + { + JsonSchema(ObjectSchema schema) + { + schema.getPropertySchemas().forEach((key, value) -> { + boolean record = value instanceof ObjectSchema; + fields.add(new SchemaField(key, + record ? null : typeFromSchema(value), + record, + record ? new JsonSchema((ObjectSchema) value) : null)); + }); + } + } + + static class ProtobufSchema + extends SchemaWrapper + { + ProtobufSchema(Descriptors.Descriptor schema) + { + schema.getFields().forEach(f -> { + boolean record = f.getJavaType() == Descriptors.FieldDescriptor.JavaType.MESSAGE; + fields.add(new SchemaField(f.getJsonName(), + record ? null : typeFromSchema(f), + record, + record ? new ProtobufSchema(f.getMessageType()) : null)); + }); + } + } + + static class AvroSchema + extends SchemaWrapper + { + AvroSchema(org.apache.avro.Schema schema, boolean customCsv) + { + final AtomicInteger position = new AtomicInteger(); + schema.getFields().forEach(f -> { + boolean record = f.schema().getType() == RECORD; + fields.add(new SchemaField(f.name(), + record ? null : typeFromSchema(f.schema()), + record, + record ? new AvroSchema(f.schema(), customCsv) : null, + customCsv ? position.getAndIncrement() : -1)); + }); + } + } + + /** + * map protobuf java type -> presto sql type + * + * @param fieldDescriptor + * @return + */ + private static Type typeFromSchema(Descriptors.FieldDescriptor fieldDescriptor) + { + switch (fieldDescriptor.getJavaType()) { + case STRING: + return createUnboundedVarcharType(); + + case INT: + case LONG: + return BIGINT; + + case FLOAT: + case DOUBLE: + return DOUBLE; + + case BOOLEAN: + return BOOLEAN; + + case BYTE_STRING: + return VARBINARY; + + default: + throw new RuntimeException("unsupported type " + fieldDescriptor); + } + } + + /** + * map json schema type -> presto sql type + * + * @param schema + * @return + */ + private static Type typeFromSchema(Schema schema) + { + if (schema instanceof NumberSchema) { + return ((NumberSchema) schema).requiresInteger() + ? BIGINT + : DOUBLE; + } + else if (schema instanceof BooleanSchema) { + return BOOLEAN; + } + else if (schema instanceof StringSchema) { + return createUnboundedVarcharType(); + } + else { + throw new RuntimeException("unsupported schema " + schema); + } + } + + /** + * map avro schema type to presto sql type + * + * @param schema + * @return + */ + private static Type typeFromSchema(org.apache.avro.Schema schema) + { + // refer to AvroColumnDecoder#isSupportedType + + switch (schema.getType()) { + case FIXED: + case STRING: + return createUnboundedVarcharType(); + + case INT: + case LONG: + return BIGINT; + + case FLOAT: + case DOUBLE: + return DOUBLE; + + case BOOLEAN: + return BOOLEAN; + + case BYTES: + return VARBINARY; + + case MAP: + case ARRAY: + // TODO: ^^ handle these https://github.com/pravega/pravega-sql/issues/65 + + case RECORD: + case ENUM: + case UNION: + default: + throw new RuntimeException("unexpected type " + schema); + } + } + + /** + * return lists of common field definitions + * uses list of fields from provided schema; schema is different depending on serialization format + * + * @param format + * @param schemaWithVersion + * @return + */ + public static List mapFieldsFromSchema( + String namePrefix, + SerializationFormat format, + SchemaWithVersion schemaWithVersion) + { + switch (format) { + case Json: + ObjectSchema objectSchema = + (ObjectSchema) JSONSchema.from(schemaWithVersion.getSchemaInfo()).getSchema(); + return mapTable(namePrefix, new JsonSchema(objectSchema)); + + case Avro: + case Custom: // re: Custom - definition for schema itself Custom is always Avro (only custom impl. is csv) + org.apache.avro.Schema schema = + new org.apache.avro.Schema.Parser().parse( + new String(schemaWithVersion.getSchemaInfo().getSchemaData().array(), StandardCharsets.UTF_8)); + return mapTable(namePrefix, new AvroSchema(schema, format == SerializationFormat.Custom)); + + case Protobuf: + return mapTable(namePrefix, new ProtobufSchema(ProtobufCommon.descriptorFor(schemaWithVersion))); + + default: + throw new IllegalArgumentException("unexpected format " + format); + } + } + + public static List mapFieldsFromSchema(String namePrefix, String format, String schemaString) + { + // schemaString defined as human-readable string in local file. only avro supported now. + switch (format) { + case AVRO: + org.apache.avro.Schema schema = + new org.apache.avro.Schema.Parser().parse(schemaString); + return mapTable(namePrefix, new AvroSchema(schema, false)); + + default: + throw new UnsupportedOperationException("unexpected format " + format); + } + } + + private static List mapTable(String namePrefix, SchemaWrapper schema) + { + return mapFieldsFromSchema(mapColumns(namePrefix, null /* mappingPrefix */, schema)); + } + + private static List mapColumns(String namePrefix, String mappingPrefix, SchemaWrapper schema) + { + List columnList = new ArrayList<>(); + schema.fields.forEach(field -> { + String name = nestedPrefixFor(namePrefix, field.name); + // for csv we use only position. for avro, json, etc, can be path into nested object + String mapping = field.ordinalPosition >= 0 + ? String.valueOf(field.ordinalPosition) + : nestedPrefixFor(mappingPrefix, field.name); + if (field.record) { + columnList.addAll(mapColumns(name, mapping, field.schema)); + } + else { + columnList.add(new SchemaColumn(name, mapping, field.type)); + } + }); + return columnList; + } + + /** + * create field description from list of name,mapping,type tuples. each pair is a field in the schema. + * + * @param schemaColumns + * @return + */ + static List mapFieldsFromSchema(List schemaColumns) + { + List fields = new ArrayList<>(); + schemaColumns.forEach(sc -> { + fields.add(new PravegaStreamFieldDescription(sc.name, + sc.type, + sc.mapping, + "", + null, + null, + false)); + }); + return fields; + } + + /** + * simply appends 'name' to an existing prefix (if any) using separator + * + * @param prefix starting prefix + * @param name name of the column to prefix + * @return prefix + */ + private static String nestedPrefixFor(String prefix, String name) + { + // (record1, field1) -> record1/field1 + return prefix == null || prefix.isEmpty() + ? name + : prefix + NESTED_RECORD_SEPARATOR + name; + } +} diff --git a/src/test/java/com/facebook/presto/pravega/CodecSupplier.java b/src/test/java/com/facebook/presto/pravega/CodecSupplier.java new file mode 100644 index 0000000..fdae003 --- /dev/null +++ b/src/test/java/com/facebook/presto/pravega/CodecSupplier.java @@ -0,0 +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 com.facebook.presto.pravega; + +import com.facebook.airlift.json.JsonCodec; +import com.facebook.airlift.json.JsonCodecFactory; +import com.facebook.airlift.json.ObjectMapperProvider; +import com.facebook.presto.common.type.Type; +import com.facebook.presto.metadata.Metadata; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.deser.std.FromStringDeserializer; +import com.google.common.collect.ImmutableMap; + +import java.util.function.Supplier; + +import static com.facebook.presto.common.type.TypeSignature.parseTypeSignature; + +public final class CodecSupplier + implements Supplier> +{ + private final Metadata metadata; + private final JsonCodecFactory codecFactory; + private final Class clazz; + + public CodecSupplier(Class clazz, Metadata metadata) + { + this.clazz = clazz; + this.metadata = metadata; + ObjectMapperProvider objectMapperProvider = new ObjectMapperProvider(); + objectMapperProvider.setJsonDeserializers(ImmutableMap.of(Type.class, new TypeDeserializer())); + this.codecFactory = new JsonCodecFactory(objectMapperProvider); + } + + @Override + public JsonCodec get() + { + return codecFactory.jsonCodec(clazz); + } + + private class TypeDeserializer + extends FromStringDeserializer + { + private static final long serialVersionUID = 1L; + + public TypeDeserializer() + { + super(Type.class); + } + + @Override + protected Type _deserialize(String value, DeserializationContext context) + { + Type type = metadata.getType(parseTypeSignature(value)); + if (type == null) { + throw new IllegalArgumentException(String.valueOf("Unknown type " + value)); + } + return type; + } + } +} diff --git a/src/test/java/com/facebook/presto/pravega/DockerContainer.java b/src/test/java/com/facebook/presto/pravega/DockerContainer.java new file mode 100644 index 0000000..ce8a1a9 --- /dev/null +++ b/src/test/java/com/facebook/presto/pravega/DockerContainer.java @@ -0,0 +1,254 @@ +/* + * 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 com.facebook.presto.pravega; + +import com.facebook.airlift.log.Logger; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.spotify.docker.client.DefaultDockerClient; +import com.spotify.docker.client.DockerClient; +import com.spotify.docker.client.exceptions.ContainerNotFoundException; +import com.spotify.docker.client.messages.Container; +import com.spotify.docker.client.messages.ContainerConfig; +import com.spotify.docker.client.messages.HostConfig; +import com.spotify.docker.client.messages.PortBinding; +import net.jodah.failsafe.Failsafe; +import net.jodah.failsafe.RetryPolicy; +import net.jodah.failsafe.function.CheckedConsumer; + +import java.io.Closeable; +import java.net.Socket; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Optional; +import java.util.Set; + +import static com.facebook.airlift.testing.Closeables.closeAllSuppress; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static com.google.common.collect.ImmutableMap.toImmutableMap; +import static com.google.common.collect.ImmutableSet.toImmutableSet; +import static com.google.common.collect.MoreCollectors.toOptional; +import static java.lang.String.format; +import static java.time.temporal.ChronoUnit.MINUTES; +import static java.time.temporal.ChronoUnit.SECONDS; +import static java.util.Objects.requireNonNull; + +public final class DockerContainer + implements Closeable +{ + private static final Logger LOG = Logger.get(DockerContainer.class); + + private static final boolean DEBUG = false; + + private static final String HOST_IP = "127.0.0.1"; + private final String image; + private final Map environment; + private DockerClient dockerClient; + private String containerId; + + private Map hostPorts; + + public DockerContainer(String image, String command, List ports, Map environment, CheckedConsumer healthCheck) + { + this.image = requireNonNull(image, "image is null"); + this.environment = ImmutableMap.copyOf(requireNonNull(environment, "environment is null")); + try { + startContainer(ports, command, healthCheck); + } + catch (Exception e) { + closeAllSuppress(e, this); + throw new RuntimeException(e); + } + } + + private void startContainer(List ports, String command, CheckedConsumer healthCheck) + throws Exception + { + dockerClient = DefaultDockerClient.fromEnv().build(); + if (dockerClient.listImages(DockerClient.ListImagesParam.byName(image)).isEmpty()) { + checkState(!image.endsWith("-SNAPSHOT"), "Unavailable snapshot image %s, please build before running tests", image); + LOG.info("Pulling image %s...", image); + dockerClient.pull(image); + } + if (DEBUG) { + Optional testingContainer = dockerClient.listContainers().stream() + .filter(container -> container.image().equals(image)) + .collect(toOptional()); + if (testingContainer.isPresent()) { + containerId = testingContainer.get().id(); + LOG.info("Container for %s already exists with id: %s", image, containerId); + calculateHostPorts(ports); + } + else { + createContainer(ports, command); + } + } + else { + createContainer(ports, command); + } + + checkState(isContainerUp(), "Container was not started properly"); + + LOG.info("Auto-assigned host ports are %s", hostPorts); + + waitForContainer(healthCheck); + } + + private boolean isContainerUp() + { + try { + return dockerClient.inspectContainer(containerId).state().running(); + } + catch (ContainerNotFoundException e) { + return false; + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + private void createContainer(List ports, String cmd) + throws Exception + { + LOG.info("Starting docker container from image %s", image); + + Map> portBindings = ports.stream() + .collect(toImmutableMap(Object::toString, port -> ImmutableList.of(PortBinding.create(HOST_IP, "0")))); + Set exposedPorts = ports.stream() + .map(Object::toString) + .collect(toImmutableSet()); + + ContainerConfig.Builder builder = ContainerConfig.builder() + .hostConfig(HostConfig.builder() + .portBindings(portBindings) + .build()) + .exposedPorts(exposedPorts) + .env(environment.entrySet().stream() + .map(entry -> format("%s=%s", entry.getKey(), entry.getValue())) + .collect(toImmutableList())) + .image(image); + if (cmd != null) { + builder.cmd(cmd); + } + + containerId = dockerClient.createContainer(builder.build()).id(); + + LOG.info("Started docker container with id: %s", containerId); + + dockerClient.startContainer(containerId); + + calculateHostPorts(ports); + + waitForContainerPorts(ports); + } + + private void waitForContainer(CheckedConsumer healthCheck) + { + RetryPolicy retryPolicy = new RetryPolicy<>() + .withMaxDuration(Duration.of(10, MINUTES)) + .withMaxAttempts(Integer.MAX_VALUE) // limited by MaxDuration + .abortOn(error -> !isContainerUp()) + .onRetry(event -> LOG.info(format("Waiting for container for %s [%s]...", image, event.getLastFailure()))) + .withDelay(Duration.of(10, SECONDS)); + Failsafe.with(retryPolicy).run(() -> healthCheck.accept(this::getHostPort)); + } + + private void waitForContainerPorts(List ports) + { + List hostPorts = ports.stream() + .map(this::getHostPort) + .collect(toImmutableList()); + + RetryPolicy retryPolicy = new RetryPolicy<>() + .withMaxDuration(Duration.of(10, MINUTES)) + .withMaxAttempts(Integer.MAX_VALUE) // limited by MaxDuration + .abortOn(error -> !isContainerUp()) + .withDelay(Duration.of(5, SECONDS)) + .onRetry(event -> LOG.info("Waiting for ports %s that are exposed on %s on %s ...", ports, HOST_IP, hostPorts)); + + Failsafe.with(retryPolicy).run(() -> { + for (int port : ports) { + try (Socket socket = new Socket(HOST_IP, getHostPort(port))) { + checkState(socket.isConnected()); + } + } + }); + } + + private void calculateHostPorts(List ports) + throws Exception + { + hostPorts = dockerClient.inspectContainer(containerId).networkSettings() + .ports() + .entrySet() + .stream() + .filter(entry -> ports.contains(extractPort(entry))) + .collect(toImmutableMap( + entry -> extractPort(entry), + entry -> entry.getValue().stream() + .peek(portBinding -> { + checkState(portBinding.hostIp().equals(HOST_IP), "Unexpected port binding found: %s", portBinding); + }) + .map(PortBinding::hostPort) + .collect(toOptional()) + .map(Integer::parseInt) + .orElseThrow(() -> new IllegalStateException("Could not extract port mapping from: " + entry)))); + } + + public int getHostPort(int port) + { + checkArgument(hostPorts.keySet().contains(port), "Port %s is not bound", port); + return hostPorts.get(port); + } + + private static int extractPort(Entry> entry) + { + checkArgument(!entry.getKey().contains("/udp"), "UDP port binding is not supported"); + return Integer.parseInt(entry.getKey().replace("/tcp", "")); + } + + private void removeContainer(String containerId) + { + try { + LOG.info("Killing container %s", containerId); + dockerClient.killContainer(containerId); + dockerClient.removeContainer(containerId); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + public void close() + { + if (dockerClient == null) { + return; + } + if (!DEBUG && containerId != null) { + removeContainer(containerId); + } + dockerClient.close(); + dockerClient = null; + } + + public interface HostPortProvider + { + int getHostPort(int containerPort); + } +} diff --git a/src/test/java/com/facebook/presto/pravega/EmbeddedPravega.java b/src/test/java/com/facebook/presto/pravega/EmbeddedPravega.java new file mode 100644 index 0000000..e0a2535 --- /dev/null +++ b/src/test/java/com/facebook/presto/pravega/EmbeddedPravega.java @@ -0,0 +1,54 @@ +/* + * Copyright (c) Pravega Authors. + * + * 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 com.facebook.presto.pravega; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; + +import java.io.Closeable; +import java.net.URI; + +public class EmbeddedPravega + implements Closeable +{ + private final DockerContainer dockerContainer; + + public EmbeddedPravega() + { + this.dockerContainer = new DockerContainer( + "pravega/pravega:latest", + "standalone", + ImmutableList.of(9090, 12345), + ImmutableMap.of( + "HOST_IP", "127.0.0.1"), + EmbeddedPravega::healthCheck); + } + + public URI getController() + { + return URI.create("tcp://localhost:" + dockerContainer.getHostPort(9090)); + } + + private static void healthCheck(DockerContainer.HostPortProvider hostPortProvider) + { + } + + @Override + public void close() + { + dockerContainer.close(); + } +} diff --git a/src/test/java/com/facebook/presto/pravega/KeyValueTable.java b/src/test/java/com/facebook/presto/pravega/KeyValueTable.java new file mode 100644 index 0000000..3dada57 --- /dev/null +++ b/src/test/java/com/facebook/presto/pravega/KeyValueTable.java @@ -0,0 +1,35 @@ +/* + * Copyright (c) Pravega Authors. + * + * 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 com.facebook.presto.pravega; + +import java.util.ArrayList; +import java.util.List; + +public class KeyValueTable +{ + private static final List keyValueTables; + static { + keyValueTables = new ArrayList<>(); + keyValueTables.add("employee"); + } + + private KeyValueTable() {} + + public static List getTables() + { + return keyValueTables; + } +} diff --git a/src/test/java/com/facebook/presto/pravega/PravegaKeyValueLoader.java b/src/test/java/com/facebook/presto/pravega/PravegaKeyValueLoader.java new file mode 100644 index 0000000..bc78c31 --- /dev/null +++ b/src/test/java/com/facebook/presto/pravega/PravegaKeyValueLoader.java @@ -0,0 +1,137 @@ +/* + * Copyright (c) Pravega Authors. + * + * 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 com.facebook.presto.pravega; + +import com.facebook.presto.pravega.util.ByteBufferInputStream; +import io.pravega.client.ClientConfig; +import io.pravega.client.KeyValueTableFactory; +import io.pravega.client.admin.KeyValueTableManager; +import io.pravega.client.admin.StreamManager; +import io.pravega.client.stream.Serializer; +import io.pravega.client.tables.KeyValueTable; +import io.pravega.client.tables.KeyValueTableClientConfiguration; +import io.pravega.client.tables.KeyValueTableConfiguration; +import org.apache.avro.Schema; +import org.apache.avro.file.DataFileStream; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.DatumReader; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.net.URI; +import java.nio.ByteBuffer; + +public class PravegaKeyValueLoader + implements AutoCloseable +{ + static class AvroSerializer + implements Serializer + { + private final DatumReader datumReader; + + public AvroSerializer(Schema schema) + { + this.datumReader = new GenericDatumReader(schema); + } + + @Override + public ByteBuffer serialize(GenericRecord record) + { + try { + GenericDatumWriter writer = new GenericDatumWriter<>(record.getSchema()); + DataFileWriter dataFileWriter = new DataFileWriter<>(writer); + + ByteArrayOutputStream os = new ByteArrayOutputStream(); + dataFileWriter.create(record.getSchema(), os); + dataFileWriter.append(record); + dataFileWriter.close(); + return ByteBuffer.wrap(os.toByteArray()); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @Override + public GenericRecord deserialize(ByteBuffer serializedValue) + { + try (DataFileStream dataFileReader = + new DataFileStream<>(new ByteBufferInputStream(serializedValue), datumReader)) { + return dataFileReader.next(); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + } + + private final KeyValueTableManager tableManager; + + private final KeyValueTableFactory tableFactory; + + private final KeyValueTable table; + + public PravegaKeyValueLoader(URI controller, + StreamManager streamManager, + String schema, + String table, + Schema keySchema, + Schema valueSchema) + { + if (!streamManager.checkScopeExists(schema)) { + streamManager.createScope(schema); + } + + ClientConfig clientConfig = ClientConfig.builder().controllerURI(controller).build(); + + this.tableManager = KeyValueTableManager.create(clientConfig); + this.tableManager.createKeyValueTable(schema, table, + KeyValueTableConfiguration.builder().partitionCount(1).build()); + + this.tableFactory = KeyValueTableFactory.withScope(schema, clientConfig); + + this.table = tableFactory.forKeyValueTable(table, + new AvroSerializer(keySchema), + new AvroSerializer(valueSchema), + KeyValueTableClientConfiguration.builder().build()); + } + + public void put(String keyFamily, GenericRecord key, GenericRecord value) + { + table.put(keyFamily, key, value).join(); + } + + @Override + public void close() + { + closeQuietly(table); + closeQuietly(tableFactory); + closeQuietly(tableManager); + } + + private void closeQuietly(AutoCloseable closeable) + { + try { + closeable.close(); + } + catch (Exception quiet) { + } + } +} diff --git a/src/test/java/com/facebook/presto/pravega/PravegaLoader.java b/src/test/java/com/facebook/presto/pravega/PravegaLoader.java new file mode 100644 index 0000000..16f38e4 --- /dev/null +++ b/src/test/java/com/facebook/presto/pravega/PravegaLoader.java @@ -0,0 +1,228 @@ +/* + * Copyright (c) Pravega Authors. + * + * 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 com.facebook.presto.pravega; + +import com.facebook.presto.Session; +import com.facebook.presto.client.Column; +import com.facebook.presto.client.QueryData; +import com.facebook.presto.client.QueryStatusInfo; +import com.facebook.presto.common.type.TimeZoneKey; +import com.facebook.presto.common.type.Type; +import com.facebook.presto.common.type.Varchars; +import com.facebook.presto.server.testing.TestingPrestoServer; +import com.facebook.presto.spi.PrestoWarning; +import com.facebook.presto.tests.AbstractTestingPrestoClient; +import com.facebook.presto.tests.ResultsSession; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import io.pravega.client.ClientConfig; +import io.pravega.client.EventStreamClientFactory; +import io.pravega.client.admin.StreamManager; +import io.pravega.client.stream.EventStreamWriter; +import io.pravega.client.stream.EventWriterConfig; +import io.pravega.client.stream.ScalingPolicy; +import io.pravega.client.stream.Serializer; +import io.pravega.client.stream.StreamConfiguration; + +import java.io.UncheckedIOException; +import java.net.URI; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.Instant; +import java.time.format.DateTimeFormatter; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; + +import static com.facebook.presto.common.type.BigintType.BIGINT; +import static com.facebook.presto.common.type.BooleanType.BOOLEAN; +import static com.facebook.presto.common.type.DateTimeEncoding.unpackMillisUtc; +import static com.facebook.presto.common.type.DateType.DATE; +import static com.facebook.presto.common.type.DoubleType.DOUBLE; +import static com.facebook.presto.common.type.IntegerType.INTEGER; +import static com.facebook.presto.common.type.TimeType.TIME; +import static com.facebook.presto.common.type.TimeWithTimeZoneType.TIME_WITH_TIME_ZONE; +import static com.facebook.presto.common.type.TimestampType.TIMESTAMP; +import static com.facebook.presto.common.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; +import static com.facebook.presto.util.DateTimeUtils.parseTimeLiteral; +import static com.facebook.presto.util.DateTimeUtils.parseTimestampWithTimeZone; +import static com.facebook.presto.util.DateTimeUtils.parseTimestampWithoutTimeZone; +import static com.google.common.base.Preconditions.checkState; +import static java.util.Objects.requireNonNull; + +public class PravegaLoader + extends AbstractTestingPrestoClient +{ + private static final DateTimeFormatter ISO8601_FORMATTER = DateTimeFormatter.ISO_LOCAL_DATE_TIME; + + static class MapSerializer + implements Serializer> + { + private final ObjectMapper objectMapper = new ObjectMapper(); + + @Override + public ByteBuffer serialize(ImmutableMap map) + { + try { + String s = objectMapper.writeValueAsString(map); + return ByteBuffer.wrap(s.getBytes(StandardCharsets.UTF_8)); + } + catch (JsonProcessingException e) { + throw new UncheckedIOException(e); + } + } + + @Override + public ImmutableMap deserialize(ByteBuffer byteBuffer) + { + return null; // not needed during ingestion + } + } + + private final EventStreamClientFactory factory; + + private final EventStreamWriter> writer; + + private final Random random = new Random(); + + public PravegaLoader(URI controller, + StreamManager streamManager, + String schema, + String stream, + TestingPrestoServer prestoServer, + Session defaultSession) + { + super(prestoServer, defaultSession); + + if (!streamManager.checkScopeExists(schema)) { + streamManager.createScope(schema); + } + + StreamConfiguration dataStreamConfig = StreamConfiguration.builder() + .scalingPolicy(ScalingPolicy.fixed(10)) + .build(); + streamManager.createStream(schema, stream, dataStreamConfig); + + this.factory = EventStreamClientFactory.withScope( + schema, ClientConfig.builder().controllerURI(controller).build()); + + this.writer = factory.createEventWriter(stream, + new MapSerializer(), + EventWriterConfig.builder().automaticallyNoteTime(true).build()); + } + + @Override + public ResultsSession getResultSession(Session session) + { + requireNonNull(session, "session is null"); + return new PravegaLoadingSession(session); + } + + private class PravegaLoadingSession + implements ResultsSession + { + private final AtomicReference> types = new AtomicReference<>(); + + private final TimeZoneKey timeZoneKey; + + private PravegaLoadingSession(Session session) + { + this.timeZoneKey = session.getTimeZoneKey(); + } + + @Override + public void setWarnings(List warnings) {} + + @Override + public void addResults(QueryStatusInfo statusInfo, QueryData data) + { + if (types.get() == null && statusInfo.getColumns() != null) { + types.set(getTypes(statusInfo.getColumns())); + } + + if (data.getData() != null) { + checkState(types.get() != null, "Data without types received!"); + List columns = statusInfo.getColumns(); + for (List fields : data.getData()) { + ImmutableMap.Builder builder = ImmutableMap.builder(); + for (int i = 0; i < fields.size(); i++) { + Type type = types.get().get(i); + Object value = convertValue(fields.get(i), type); + if (value != null) { + builder.put(columns.get(i).getName(), value); + } + } + + writer.writeEvent(String.valueOf(random.nextInt(999)), builder.build()); + } + } + } + + @Override + public Void build(Map setSessionProperties, Set resetSessionProperties) + { + return null; + } + + private Object convertValue(Object value, Type type) + { + if (value == null) { + return null; + } + + if (BOOLEAN.equals(type) || Varchars.isVarcharType(type)) { + return value; + } + if (BIGINT.equals(type)) { + return ((Number) value).longValue(); + } + if (INTEGER.equals(type)) { + return ((Number) value).intValue(); + } + if (DOUBLE.equals(type)) { + return ((Number) value).doubleValue(); + } + if (DATE.equals(type)) { + return value; + } + if (TIME.equals(type)) { + return ISO8601_FORMATTER.format(Instant.ofEpochMilli(parseTimeLiteral(timeZoneKey, (String) value))); + } + if (TIMESTAMP.equals(type)) { + return ISO8601_FORMATTER.format(Instant.ofEpochMilli(parseTimestampWithoutTimeZone(timeZoneKey, (String) value))); + } + if (TIME_WITH_TIME_ZONE.equals(type) || TIMESTAMP_WITH_TIME_ZONE.equals(type)) { + return ISO8601_FORMATTER.format(Instant.ofEpochMilli(unpackMillisUtc(parseTimestampWithTimeZone(timeZoneKey, (String) value)))); + } + throw new AssertionError("unhandled type: " + type); + } + } + + @Override + public void close() + { + try { + writer.close(); + factory.close(); + } + finally { + super.close(); + } + } +} diff --git a/src/test/java/com/facebook/presto/pravega/PravegaQueryRunner.java b/src/test/java/com/facebook/presto/pravega/PravegaQueryRunner.java new file mode 100644 index 0000000..f4ef510 --- /dev/null +++ b/src/test/java/com/facebook/presto/pravega/PravegaQueryRunner.java @@ -0,0 +1,175 @@ +/* + * Copyright (c) Pravega Authors. + * + * 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 com.facebook.presto.pravega; + +import com.facebook.airlift.json.JsonCodec; +import com.facebook.airlift.log.Logger; +import com.facebook.airlift.log.Logging; +import com.facebook.presto.Session; +import com.facebook.presto.common.QualifiedObjectName; +import com.facebook.presto.metadata.Metadata; +import com.facebook.presto.spi.SchemaTableName; +import com.facebook.presto.testing.QueryRunner; +import com.facebook.presto.tests.DistributedQueryRunner; +import com.facebook.presto.tests.TestingPrestoClient; +import com.facebook.presto.tpch.TpchPlugin; +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import com.google.common.collect.ImmutableMap; +import io.airlift.tpch.TpchTable; +import io.pravega.client.admin.StreamManager; + +import java.net.URI; +import java.util.Map; +import java.util.Optional; + +import static com.facebook.airlift.testing.Closeables.closeAllSuppress; +import static com.facebook.presto.pravega.PravegaTestUtils.getKvStreamDesc; +import static com.facebook.presto.pravega.PravegaTestUtils.getStreamDesc; +import static com.facebook.presto.testing.TestingSession.testSessionBuilder; +import static com.facebook.presto.tpch.TpchMetadata.TINY_SCHEMA_NAME; +import static io.airlift.units.Duration.nanosSince; +import static java.util.Locale.ENGLISH; +import static java.util.concurrent.TimeUnit.SECONDS; + +public final class PravegaQueryRunner +{ + private PravegaQueryRunner() + { + } + + private static final Logger log = Logger.get("PravegaQueryRunner"); + private static final String PRAVEGA_CATALOG = "pravega"; + private static final String KV_SCHEMA = "kv"; + private static final String KV_KEY_FAMILY = "kf1"; // TODO: could randomly pick 1 from objectArgs in table desc + private static final String TPCH_SCHEMA = "tpch"; + + public static DistributedQueryRunner createQueryRunner(URI controller, Iterable> tpchTables, Iterable keyValueTables) + throws Exception + { + DistributedQueryRunner queryRunner = null; + try { + queryRunner = new DistributedQueryRunner(createSession(), 2); + + queryRunner.installPlugin(new TpchPlugin()); + queryRunner.createCatalog("tpch", "tpch"); + + PravegaTableDescriptionSupplier tableDescriptionSupplier = + createSchemas(queryRunner.getCoordinator().getMetadata(), tpchTables, keyValueTables); + + installPlugin(controller, queryRunner, tableDescriptionSupplier); + + TestingPrestoClient prestoClient = queryRunner.getClient(); + + log.info("Loading data..."); + long startTime = System.nanoTime(); + try (StreamManager streamManager = StreamManager.create(controller)) { + log.info("Loading tpch data..."); + for (TpchTable table : tpchTables) { + loadTpchStream(controller, streamManager, prestoClient, table); + } + log.info("Loading tpch complete"); + + log.info("Loading pravega data..."); + for (String table : keyValueTables) { + loadPravegaKVTable(controller, streamManager, table); + } + log.info("Loading pravega complete"); + } + + log.info("Loading complete in %s", nanosSince(startTime).toString(SECONDS)); + + return queryRunner; + } + catch (Throwable e) { + closeAllSuppress(e, queryRunner); + throw e; + } + } + + private static void loadTpchStream(URI controller, StreamManager streamManager, TestingPrestoClient prestoClient, TpchTable table) + { + long start = System.nanoTime(); + log.info("Running import for %s", table.getTableName()); + PravegaTestUtils.loadTpchStream(controller, streamManager, prestoClient, TPCH_SCHEMA, table.getTableName(), new QualifiedObjectName("tpch", TINY_SCHEMA_NAME, table.getTableName().toLowerCase(ENGLISH))); + log.info("Imported %s in %s", 0, table.getTableName(), nanosSince(start).convertToMostSuccinctTimeUnit()); + } + + private static void loadPravegaKVTable(URI controller, StreamManager streamManager, String table) + { + long start = System.nanoTime(); + log.info("Running import for %s", table); + PravegaTestUtils.loadKeyValueTable(controller, streamManager, KV_SCHEMA, table, KV_KEY_FAMILY); + log.info("Imported %s in %s", 0, table, nanosSince(start).convertToMostSuccinctTimeUnit()); + } + + private static PravegaTableDescriptionSupplier createSchemas(Metadata metadata, Iterable> tables, Iterable keyValueTables) + { + JsonCodec streamDescCodec = new CodecSupplier<>(PravegaStreamDescription.class, metadata).get(); + + Cache schemaCache = CacheBuilder.newBuilder().build(); + Cache> tableCache = CacheBuilder.newBuilder().build(); + + for (TpchTable table : tables) { + SchemaTableName schemaTableName = new SchemaTableName(TPCH_SCHEMA, table.getTableName()); + PravegaTableName pravegaTableName = new PravegaTableName(schemaTableName); + + schemaCache.put(schemaTableName.getSchemaName(), new Object()); + tableCache.put(pravegaTableName, Optional.of(getStreamDesc(streamDescCodec, "tpch", table.getTableName()))); + } + + for (String table : keyValueTables) { + SchemaTableName schemaTableName = new SchemaTableName(KV_SCHEMA, table); + PravegaTableName pravegaTableName = new PravegaTableName(schemaTableName); + + schemaCache.put(schemaTableName.getSchemaName(), new Object()); + tableCache.put(pravegaTableName, Optional.of(getKvStreamDesc(table))); + } + + // all schemas + tables will be served from these provided caches + return new PravegaTableDescriptionSupplier(null, schemaCache, tableCache); + } + + public static Session createSession() + { + return testSessionBuilder().setCatalog(PRAVEGA_CATALOG).setSchema(TPCH_SCHEMA).build(); + } + + public static void installPlugin(URI controller, QueryRunner queryRunner, PravegaTableDescriptionSupplier tableDescriptionSupplier) + { + PravegaPlugin pravegaPlugin = new PravegaPlugin(); + pravegaPlugin.setTableDescriptionSupplier(tableDescriptionSupplier); + queryRunner.installPlugin(pravegaPlugin); + + Map config = ImmutableMap.of( + "pravega.controller", controller.toASCIIString(), + "pravega.schema-registry", "http://localhost:9092"); + + queryRunner.createCatalog("pravega", "pravega", config); + } + + public static void main(String[] args) + throws Exception + { + // you need an already running pravega - this code won't start one + Logging.initialize(); + DistributedQueryRunner queryRunner = createQueryRunner(URI.create("tcp://127.0.0.1:9090"), TpchTable.getTables(), KeyValueTable.getTables()); + Thread.sleep(10); + Logger log = Logger.get(PravegaQueryRunner.class); + log.info("======== SERVER STARTED ========"); + log.info("\n====\n%s\n====", queryRunner.getCoordinator().getBaseUrl()); + } +} diff --git a/src/test/java/com/facebook/presto/pravega/PravegaTestUtils.java b/src/test/java/com/facebook/presto/pravega/PravegaTestUtils.java new file mode 100644 index 0000000..2cc4caa --- /dev/null +++ b/src/test/java/com/facebook/presto/pravega/PravegaTestUtils.java @@ -0,0 +1,169 @@ +/* + * Copyright (c) Pravega Authors. + * + * 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 com.facebook.presto.pravega; + +import com.facebook.airlift.json.JsonCodec; +import com.facebook.presto.common.QualifiedObjectName; +import com.facebook.presto.tests.TestingPrestoClient; +import com.google.common.io.ByteStreams; +import com.google.common.io.CharStreams; +import io.pravega.client.admin.StreamManager; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecordBuilder; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.UncheckedIOException; +import java.net.URI; +import java.util.ArrayList; +import java.util.List; + +import static com.facebook.presto.pravega.util.PravegaStreamDescUtils.mapFieldsFromSchema; +import static java.lang.String.format; + +public final class PravegaTestUtils +{ + private PravegaTestUtils() {} + + public static PravegaStreamDescription getStreamDesc(JsonCodec streamDescriptionCodec, String directory, String table) + { + try (InputStream inputStream = PravegaTestUtils.class.getResourceAsStream(String.format("/%s/%s.json", directory, table))) { + return streamDescriptionCodec.fromJson(ByteStreams.toByteArray(inputStream)); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + public static String readSchema(String directory, String schema) + { + try (InputStreamReader reader = new InputStreamReader(PravegaTestUtils.class.getResourceAsStream(String.format("/%s/%s", directory, schema)))) { + return CharStreams.toString(reader); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + public static void loadTpchStream(URI controller, StreamManager streamManager, TestingPrestoClient prestoClient, String schema, String stream, QualifiedObjectName tpchTableName) + { + try (PravegaLoader tpchLoader = new PravegaLoader(controller, streamManager, schema, stream, prestoClient.getServer(), prestoClient.getDefaultSession())) { + tpchLoader.execute(format("SELECT * from %s", tpchTableName)); + } + } + + public static void loadKeyValueTable(URI controller, StreamManager streamManager, String schema, String table, String keyFamily) + { + PravegaStreamDescription tableDesc = getKvStreamDesc(table); + + Schema keySchema = avroSchema(tableDesc, 0); + GenericRecordBuilder keyBuilder = new GenericRecordBuilder(keySchema); + + Schema valueSchema = avroSchema(tableDesc, 1); + GenericRecordBuilder valueBuilder = new GenericRecordBuilder(valueSchema); + + try (PravegaKeyValueLoader keyValueLoader = + new PravegaKeyValueLoader(controller, + streamManager, schema, table, + avroSchema(tableDesc, 0), + avroSchema(tableDesc, 1))) { + try (InputStream inputStream = PravegaTestUtils.class.getResourceAsStream(String.format("/kv/%s.records", table)); + BufferedReader reader = new BufferedReader(new InputStreamReader(inputStream))) { + // each line in file is a record, key + value + // '|' character separates key fields from values fields + // fields separated by ',' + String datum = reader.readLine(); + while (datum != null && !datum.isEmpty()) { + String keyString = datum.split("\\|")[0]; + String valueString = datum.split("\\|")[1]; + + String[] keyFieldValues = keyString.split(","); + String[] valueFieldValues = valueString.split(","); + + for (int i = 0; i < keyFieldValues.length; i++) { + setAvroValue(keyBuilder, keySchema.getFields().get(i), keyFieldValues[i]); + } + + for (int i = 0; i < valueFieldValues.length; i++) { + setAvroValue(valueBuilder, valueSchema.getFields().get(i), valueFieldValues[i]); + } + + keyValueLoader.put(keyFamily, keyBuilder.build(), valueBuilder.build()); + + keySchema.getFields().forEach(keyBuilder::clear); + valueSchema.getFields().forEach(valueBuilder::clear); + + datum = reader.readLine(); + } + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + } + + static void setAvroValue(GenericRecordBuilder builder, Schema.Field field, String value) + { + Object object; + switch (field.schema().getType()) { + case INT: + object = Integer.parseInt(value); + break; + case LONG: + object = Long.parseLong(value); + break; + default: + object = value; + break; + } + builder.set(field, object); + } + + public static PravegaStreamDescription getKvStreamDesc(String table) + { + JsonCodec jsonCodec = JsonCodec.jsonCodec(PravegaStreamDescription.class); + + PravegaStreamDescription streamDescription = getStreamDesc(jsonCodec, "kv", table); + streamDescription.getEvent().orElseThrow(IllegalArgumentException::new); + + PravegaStreamFieldGroup keyEvent = streamDescription.getEvent().get().get(0); + keyEvent.getDataSchema().orElseThrow(IllegalArgumentException::new); + String keySchema = PravegaTestUtils.readSchema("kv", keyEvent.getDataSchema().get()); + + PravegaStreamFieldGroup valueEvent = streamDescription.getEvent().get().get(1); + valueEvent.getDataSchema().orElseThrow(IllegalArgumentException::new); + String valueSchema = PravegaTestUtils.readSchema("kv", valueEvent.getDataSchema().get()); + + List keyFields = + mapFieldsFromSchema("key", keyEvent.getDataFormat(), keySchema); + + List valueFields = + mapFieldsFromSchema("value", valueEvent.getDataFormat(), valueSchema); + + List newFieldGroups = new ArrayList<>(2); + newFieldGroups.add(new PravegaStreamFieldGroup(keyEvent, keySchema, keyFields)); + newFieldGroups.add(new PravegaStreamFieldGroup(valueEvent, valueSchema, valueFields)); + + return new PravegaStreamDescription(streamDescription, newFieldGroups); + } + + public static Schema avroSchema(PravegaStreamDescription streamDescription, int event) + { + return new Schema.Parser().parse(streamDescription.getEvent().get().get(event).getDataSchema().get()); + } +} diff --git a/src/test/java/com/facebook/presto/pravega/TestPravegaDistributed.java b/src/test/java/com/facebook/presto/pravega/TestPravegaDistributed.java new file mode 100644 index 0000000..ec975ef --- /dev/null +++ b/src/test/java/com/facebook/presto/pravega/TestPravegaDistributed.java @@ -0,0 +1,58 @@ +/* + * Copyright (c) Pravega Authors. + * + * 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 com.facebook.presto.pravega; + +import com.facebook.presto.tests.AbstractTestQueries; +import io.airlift.tpch.TpchTable; +import org.testng.annotations.AfterClass; +import org.testng.annotations.Test; + +@Test +public class TestPravegaDistributed + extends AbstractTestQueries +{ + private final EmbeddedPravega pravega; + + public TestPravegaDistributed() + { + this(new EmbeddedPravega()); + } + + public TestPravegaDistributed(EmbeddedPravega pravega) + { + super(() -> PravegaQueryRunner.createQueryRunner(pravega.getController(), TpchTable.getTables(), java.util.Collections.emptyList())); + this.pravega = pravega; + } + + // non-passing tests + @Override + public void testAccessControl(){} + + @Override + public void testDescribeOutputNonSelect(){} + + @Override + public void testInformationSchemaUppercaseName() {} + + @Override + public void testShowTablesLikeWithEscape() {} + + @AfterClass(alwaysRun = true) + public void destroy() + { + pravega.close(); + } +} diff --git a/src/test/java/com/facebook/presto/pravega/TestPravegaKeyValue.java b/src/test/java/com/facebook/presto/pravega/TestPravegaKeyValue.java new file mode 100644 index 0000000..2597898 --- /dev/null +++ b/src/test/java/com/facebook/presto/pravega/TestPravegaKeyValue.java @@ -0,0 +1,86 @@ +/* + * Copyright (c) Pravega Authors. + * + * 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 com.facebook.presto.pravega; + +import com.facebook.presto.Session; +import com.facebook.presto.testing.MaterializedResult; +import com.facebook.presto.tests.DistributedQueryRunner; +import org.testng.annotations.AfterClass; +import org.testng.annotations.Test; + +import java.util.List; + +import static com.facebook.presto.pravega.PravegaTestUtils.getKvStreamDesc; +import static com.facebook.presto.testing.TestingSession.testSessionBuilder; +import static org.testng.Assert.assertEquals; + +@Test +public class TestPravegaKeyValue +{ + private final EmbeddedPravega pravega; + + private final DistributedQueryRunner queryRunner; + + public TestPravegaKeyValue() + throws Exception + { + this.pravega = new EmbeddedPravega(); + this.queryRunner = PravegaQueryRunner.createQueryRunner(pravega.getController(), java.util.Collections.emptyList(), KeyValueTable.getTables()); + } + + @Test + public void testLoadSimpleSchema() + { + PravegaStreamDescription streamDescription = getKvStreamDesc("employee"); + streamDescription.getEvent().orElseThrow(IllegalStateException::new); + + List keyFields = streamDescription.getEvent().get().get(0).getFields(); + assertEquals(1, keyFields.size()); + assertEquals("key/id", keyFields.get(0).getName()); + + List valueFields = streamDescription.getEvent().get().get(1).getFields(); + assertEquals(2, valueFields.size()); + assertEquals("value/first", valueFields.get(0).getName()); + assertEquals("value/last", valueFields.get(1).getName()); + } + + @Test + public void testSelectCount() + { + MaterializedResult result = queryRunner.execute(kvSession(), "select count(*) from employee"); + assertEquals(result.getRowCount(), 1); + assertEquals(result.getMaterializedRows().get(0).getField(0), 3L); + } + + @Test + public void testSelectConstraint() + { + MaterializedResult result = queryRunner.execute(kvSession(), "select \"value/last\" from employee where \"key/id\" = 2"); + assertEquals(result.getRowCount(), 1); + assertEquals(result.getMaterializedRows().get(0).getField(0), "doe"); + } + + public static Session kvSession() + { + return testSessionBuilder().setCatalog("pravega").setSchema("kv").build(); + } + + @AfterClass(alwaysRun = true) + public void destroy() + { + pravega.close(); + } +} diff --git a/src/test/java/com/facebook/presto/pravega/TestPravegaPlugin.java b/src/test/java/com/facebook/presto/pravega/TestPravegaPlugin.java index e308be8..afa06a8 100644 --- a/src/test/java/com/facebook/presto/pravega/TestPravegaPlugin.java +++ b/src/test/java/com/facebook/presto/pravega/TestPravegaPlugin.java @@ -13,13 +13,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package com.facebook.presto.pravega; import com.facebook.presto.spi.connector.Connector; import com.facebook.presto.spi.connector.ConnectorFactory; import com.facebook.presto.testing.TestingConnectorContext; import com.google.common.collect.ImmutableMap; +import org.testng.annotations.Test; import static com.facebook.airlift.testing.Assertions.assertInstanceOf; import static com.google.common.collect.Iterables.getOnlyElement; @@ -27,6 +27,7 @@ public class TestPravegaPlugin { + @Test public void testSpinup() { PravegaPlugin plugin = new PravegaPlugin(); @@ -35,10 +36,10 @@ public void testSpinup() assertInstanceOf(factory, PravegaConnectorFactory.class); Connector c = factory.create( - "test-connector", + "pravega", ImmutableMap.builder() - .put("pravega.table-names", "test") - .put("pravega.nodes", "localhost:9092") + .put("pravega.controller", "tcp://localhost:9090") + .put("pravega.schema-registry", "http://localhost:9092") .build(), new TestingConnectorContext()); assertNotNull(c); diff --git a/src/test/resources/kv/employee-id.avsc b/src/test/resources/kv/employee-id.avsc new file mode 100644 index 0000000..9fab0c0 --- /dev/null +++ b/src/test/resources/kv/employee-id.avsc @@ -0,0 +1,8 @@ +{ + "namespace": "io.pravega.avro", + "type": "record", + "name": "Id", + "fields": [ + {"name": "id", "type": "int"} + ] +} \ No newline at end of file diff --git a/src/test/resources/kv/employee-value.avsc b/src/test/resources/kv/employee-value.avsc new file mode 100644 index 0000000..78abbe1 --- /dev/null +++ b/src/test/resources/kv/employee-value.avsc @@ -0,0 +1,9 @@ +{ + "namespace": "io.pravega.avro", + "type": "record", + "name": "Value", + "fields": [ + {"name": "first", "type": "string"}, + {"name": "last", "type": "string"} + ] +} \ No newline at end of file diff --git a/src/test/resources/kv/employee.json b/src/test/resources/kv/employee.json new file mode 100644 index 0000000..82272d8 --- /dev/null +++ b/src/test/resources/kv/employee.json @@ -0,0 +1,17 @@ +{ + "schemaName": "kv", + "tableName": "employee", + "objectName": "employee", + "objectType": "KV_TABLE", + "objectArgs": ["kf1"], + "event": [ + { + "dataFormat": "avro", + "dataSchema": "employee-id.avsc" + }, + { + "dataFormat": "avro", + "dataSchema": "employee-value.avsc" + } + ] +} \ No newline at end of file diff --git a/src/test/resources/kv/employee.records b/src/test/resources/kv/employee.records new file mode 100644 index 0000000..98e6013 --- /dev/null +++ b/src/test/resources/kv/employee.records @@ -0,0 +1,3 @@ +1|jonh,smith +2|janet,doe +3|susan,dewar \ No newline at end of file diff --git a/src/test/resources/tpch/customer.json b/src/test/resources/tpch/customer.json new file mode 100644 index 0000000..d59682d --- /dev/null +++ b/src/test/resources/tpch/customer.json @@ -0,0 +1,50 @@ +{ + "tableName": "customer", + "schemaName": "tpch", + "objectName": "customer", + "event": [{ + "dataFormat": "json", + "fields": [ + { + "name": "custkey", + "mapping": "custkey", + "type": "BIGINT" + }, + { + "name": "name", + "mapping": "name", + "type": "VARCHAR(25)" + }, + { + "name": "address", + "mapping": "address", + "type": "VARCHAR(40)" + }, + { + "name": "nationkey", + "mapping": "nationkey", + "type": "BIGINT" + }, + { + "name": "phone", + "mapping": "phone", + "type": "VARCHAR(15)" + }, + { + "name": "acctbal", + "mapping": "acctbal", + "type": "DOUBLE" + }, + { + "name": "mktsegment", + "mapping": "mktsegment", + "type": "VARCHAR(10)" + }, + { + "name": "comment", + "mapping": "comment", + "type": "VARCHAR(117)" + } + ] + }] +} diff --git a/src/test/resources/tpch/lineitem.json b/src/test/resources/tpch/lineitem.json new file mode 100644 index 0000000..ed70b06 --- /dev/null +++ b/src/test/resources/tpch/lineitem.json @@ -0,0 +1,93 @@ +{ + "tableName": "lineitem", + "schemaName": "tpch", + "objectName": "lineitem", + "event": [{ + "dataFormat": "json", + "fields": [ + { + "name": "orderkey", + "mapping": "orderkey", + "type": "BIGINT" + }, + { + "name": "partkey", + "mapping": "partkey", + "type": "BIGINT" + }, + { + "name": "suppkey", + "mapping": "suppkey", + "type": "BIGINT" + }, + { + "name": "linenumber", + "mapping": "linenumber", + "type": "INTEGER" + }, + { + "name": "quantity", + "mapping": "quantity", + "type": "DOUBLE" + }, + { + "name": "extendedprice", + "mapping": "extendedprice", + "type": "DOUBLE" + }, + { + "name": "discount", + "mapping": "discount", + "type": "DOUBLE" + }, + { + "name": "tax", + "mapping": "tax", + "type": "DOUBLE" + }, + { + "name": "returnflag", + "mapping": "returnflag", + "type": "VARCHAR(1)" + }, + { + "name": "linestatus", + "mapping": "linestatus", + "type": "VARCHAR(1)" + }, + { + "name": "shipdate", + "mapping": "shipdate", + "type": "DATE", + "dataFormat": "iso8601" + }, + { + "name": "commitdate", + "mapping": "commitdate", + "type": "DATE", + "dataFormat": "iso8601" + }, + { + "name": "receiptdate", + "mapping": "receiptdate", + "type": "DATE", + "dataFormat": "iso8601" + }, + { + "name": "shipinstruct", + "mapping": "shipinstruct", + "type": "VARCHAR(25)" + }, + { + "name": "shipmode", + "mapping": "shipmode", + "type": "VARCHAR(7)" + }, + { + "name": "comment", + "mapping": "comment", + "type": "VARCHAR(44)" + } + ] + }] +} diff --git a/src/test/resources/tpch/nation.json b/src/test/resources/tpch/nation.json new file mode 100644 index 0000000..81117c8 --- /dev/null +++ b/src/test/resources/tpch/nation.json @@ -0,0 +1,30 @@ +{ + "tableName": "nation", + "schemaName": "tpch", + "objectName": "nation", + "event": [{ + "dataFormat": "json", + "fields": [ + { + "name": "nationkey", + "mapping": "nationkey", + "type": "BIGINT" + }, + { + "name": "name", + "mapping": "name", + "type": "VARCHAR(25)" + }, + { + "name": "regionkey", + "mapping": "regionkey", + "type": "BIGINT" + }, + { + "name": "comment", + "mapping": "comment", + "type": "VARCHAR(152)" + } + ] + }] +} diff --git a/src/test/resources/tpch/orders.json b/src/test/resources/tpch/orders.json new file mode 100644 index 0000000..145e226 --- /dev/null +++ b/src/test/resources/tpch/orders.json @@ -0,0 +1,58 @@ +{ + "tableName": "orders", + "schemaName": "tpch", + "objectName": "orders", + "event": [ + { + "dataFormat": "json", + "fields": [ + { + "name": "orderkey", + "mapping": "orderkey", + "type": "BIGINT" + }, + { + "name": "custkey", + "mapping": "custkey", + "type": "BIGINT" + }, + { + "name": "orderstatus", + "mapping": "orderstatus", + "type": "VARCHAR(1)" + }, + { + "name": "totalprice", + "mapping": "totalprice", + "type": "DOUBLE" + }, + { + "name": "orderdate", + "mapping": "orderdate", + "type": "DATE", + "dataFormat": "iso8601" + }, + { + "name": "orderpriority", + "mapping": "orderpriority", + "type": "VARCHAR(15)" + }, + { + "name": "clerk", + "mapping": "clerk", + "type": "VARCHAR(15)" + }, + { + "name": "shippriority", + "mapping": "shippriority", + "type": "INTEGER" + }, + { + "name": "comment", + "mapping": "comment", + "type": "VARCHAR(79)" + } + ] + } + ] +} diff --git a/src/test/resources/tpch/part.json b/src/test/resources/tpch/part.json new file mode 100644 index 0000000..87cfad3 --- /dev/null +++ b/src/test/resources/tpch/part.json @@ -0,0 +1,55 @@ +{ + "tableName": "part", + "schemaName": "tpch", + "objectName": "part", + "event": [{ + "dataFormat": "json", + "fields": [ + { + "name": "partkey", + "mapping": "partkey", + "type": "BIGINT" + }, + { + "name": "name", + "mapping": "name", + "type": "VARCHAR(55)" + }, + { + "name": "mfgr", + "mapping": "mfgr", + "type": "VARCHAR(25)" + }, + { + "name": "brand", + "mapping": "brand", + "type": "VARCHAR(10)" + }, + { + "name": "type", + "mapping": "type", + "type": "VARCHAR(25)" + }, + { + "name": "size", + "mapping": "size", + "type": "INTEGER" + }, + { + "name": "container", + "mapping": "container", + "type": "VARCHAR(10)" + }, + { + "name": "retailprice", + "mapping": "retailprice", + "type": "DOUBLE" + }, + { + "name": "comment", + "mapping": "comment", + "type": "VARCHAR(23)" + } + ] + }] +} diff --git a/src/test/resources/tpch/partsupp.json b/src/test/resources/tpch/partsupp.json new file mode 100644 index 0000000..7df0e37 --- /dev/null +++ b/src/test/resources/tpch/partsupp.json @@ -0,0 +1,35 @@ +{ + "tableName": "partsupp", + "schemaName": "tpch", + "objectName": "partsupp", + "event": [{ + "dataFormat": "json", + "fields": [ + { + "name": "partkey", + "mapping": "partkey", + "type": "BIGINT" + }, + { + "name": "suppkey", + "mapping": "suppkey", + "type": "BIGINT" + }, + { + "name": "availqty", + "mapping": "availqty", + "type": "INTEGER" + }, + { + "name": "supplycost", + "mapping": "supplycost", + "type": "DOUBLE" + }, + { + "name": "comment", + "mapping": "comment", + "type": "VARCHAR(199)" + } + ] + }] +} diff --git a/src/test/resources/tpch/region.json b/src/test/resources/tpch/region.json new file mode 100644 index 0000000..2abc65f --- /dev/null +++ b/src/test/resources/tpch/region.json @@ -0,0 +1,25 @@ +{ + "tableName": "region", + "schemaName": "tpch", + "objectName": "region", + "event": [{ + "dataFormat": "json", + "fields": [ + { + "name": "regionkey", + "mapping": "regionkey", + "type": "BIGINT" + }, + { + "name": "name", + "mapping": "name", + "type": "VARCHAR(25)" + }, + { + "name": "comment", + "mapping": "comment", + "type": "VARCHAR(152)" + } + ] + }] +} diff --git a/src/test/resources/tpch/supplier.json b/src/test/resources/tpch/supplier.json new file mode 100644 index 0000000..2ad5a35 --- /dev/null +++ b/src/test/resources/tpch/supplier.json @@ -0,0 +1,45 @@ +{ + "tableName": "supplier", + "schemaName": "tpch", + "objectName": "supplier", + "event": [{ + "dataFormat": "json", + "fields": [ + { + "name": "suppkey", + "mapping": "suppkey", + "type": "BIGINT" + }, + { + "name": "name", + "mapping": "name", + "type": "VARCHAR(25)" + }, + { + "name": "address", + "mapping": "address", + "type": "VARCHAR(40)" + }, + { + "name": "nationkey", + "mapping": "nationkey", + "type": "BIGINT" + }, + { + "name": "phone", + "mapping": "phone", + "type": "VARCHAR(15)" + }, + { + "name": "acctbal", + "mapping": "acctbal", + "type": "DOUBLE" + }, + { + "name": "comment", + "mapping": "comment", + "type": "VARCHAR(101)" + } + ] + }] +} From 856d5018c9f35b844af0595e17ff9992524eb8a2 Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Tue, 16 Mar 2021 06:54:53 -0700 Subject: [PATCH 02/32] add missing reference to docker client Signed-off-by: Andrew Robertson --- build.gradle | 1 + gradle.properties | 1 + 2 files changed, 2 insertions(+) diff --git a/build.gradle b/build.gradle index 60e7a36..aae8db3 100644 --- a/build.gradle +++ b/build.gradle @@ -61,6 +61,7 @@ dependencies { testImplementation "com.facebook.presto:presto-tests:${prestoVersion}" testImplementation "com.facebook.airlift:testing:${airliftTestingVersion}" + testImplementation "com.starburstdata:starburst-spotify-docker-client:${spotifyDockerClientVersion}" compileOnly "io.airlift:slice:${airliftSliceVersion}" compileOnly "io.airlift:units:${airliftUnitsVersion}" diff --git a/gradle.properties b/gradle.properties index b93efe0..0b5d456 100644 --- a/gradle.properties +++ b/gradle.properties @@ -28,5 +28,6 @@ pravegaVersion=0.9.0 pravegaSchemaRegistryVersion=0.2.0 prestoVersion=0.247 protobufVersion=3.11.4 +spotifyDockerClientVersion=8.11.7-0.6 zookeeperVersion=3.5.7 zookeeperClientVersion=0.10 From 3930e8749c68f41aa205b722a5c0b6e952a589b0 Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Wed, 17 Mar 2021 11:33:23 -0700 Subject: [PATCH 03/32] update header in 3 test files Signed-off-by: Andrew Robertson --- src/test/java/io/pravega/connectors/presto/CodecSupplier.java | 3 +++ .../java/io/pravega/connectors/presto/DockerContainer.java | 3 +++ src/test/java/io/pravega/connectors/presto/PravegaLoader.java | 3 +++ 3 files changed, 9 insertions(+) diff --git a/src/test/java/io/pravega/connectors/presto/CodecSupplier.java b/src/test/java/io/pravega/connectors/presto/CodecSupplier.java index 1910a09..bedc930 100644 --- a/src/test/java/io/pravega/connectors/presto/CodecSupplier.java +++ b/src/test/java/io/pravega/connectors/presto/CodecSupplier.java @@ -10,6 +10,9 @@ * 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. + * + * Note: This class file is from PrestoDb + * https://github.com/prestodb/presto/blob/0.247/presto-kafka/src/test/java/com/facebook/presto/kafka/util/CodecSupplier.java */ package io.pravega.connectors.presto; diff --git a/src/test/java/io/pravega/connectors/presto/DockerContainer.java b/src/test/java/io/pravega/connectors/presto/DockerContainer.java index 3e6bb0b..5db1385 100644 --- a/src/test/java/io/pravega/connectors/presto/DockerContainer.java +++ b/src/test/java/io/pravega/connectors/presto/DockerContainer.java @@ -10,6 +10,9 @@ * 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. + * + * Note: This class file is from PrestoDb (with minor change to set "cmd" in ContainerConfig.Builder) + * https://github.com/prestodb/presto/blob/0.247/presto-testing-docker/src/main/java/com/facebook/presto/testing/docker/DockerContainer.java */ package io.pravega.connectors.presto; diff --git a/src/test/java/io/pravega/connectors/presto/PravegaLoader.java b/src/test/java/io/pravega/connectors/presto/PravegaLoader.java index 9383223..2b253d1 100644 --- a/src/test/java/io/pravega/connectors/presto/PravegaLoader.java +++ b/src/test/java/io/pravega/connectors/presto/PravegaLoader.java @@ -12,6 +12,9 @@ * 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. + * + * Note: This file contains some code from PrestoDb (majority of the "ResultsSession" implementation) + * https://github.com/prestodb/presto/blob/0.247/presto-kafka/src/test/java/com/facebook/presto/kafka/util/KafkaLoader.java */ package io.pravega.connectors.presto; From dbe84ca385cd22655f3343b7822987380981f69f Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Thu, 18 Mar 2021 08:33:27 -0700 Subject: [PATCH 04/32] only run integration tests if -Pintegration flag is set Signed-off-by: Andrew Robertson --- build.gradle | 12 ++++++++++-- .../presto/PravegaTableDescriptionSupplier.java | 6 +++--- src/test/integration-test.xml | 8 ++++++++ .../presto/{ => integration}/CodecSupplier.java | 2 +- .../presto/{ => integration}/DockerContainer.java | 2 +- .../presto/{ => integration}/EmbeddedPravega.java | 2 +- .../presto/{ => integration}/KeyValueTable.java | 2 +- .../{ => integration}/PravegaKeyValueLoader.java | 2 +- .../presto/{ => integration}/PravegaLoader.java | 2 +- .../presto/{ => integration}/PravegaQueryRunner.java | 10 +++++++--- .../presto/{ => integration}/PravegaTestUtils.java | 5 ++++- .../{ => integration}/TestPravegaDistributed.java | 2 +- .../{ => integration}/TestPravegaKeyValue.java | 6 ++++-- src/test/unit-test.xml | 8 ++++++++ 14 files changed, 51 insertions(+), 18 deletions(-) create mode 100644 src/test/integration-test.xml rename src/test/java/io/pravega/connectors/presto/{ => integration}/CodecSupplier.java (98%) rename src/test/java/io/pravega/connectors/presto/{ => integration}/DockerContainer.java (99%) rename src/test/java/io/pravega/connectors/presto/{ => integration}/EmbeddedPravega.java (96%) rename src/test/java/io/pravega/connectors/presto/{ => integration}/KeyValueTable.java (95%) rename src/test/java/io/pravega/connectors/presto/{ => integration}/PravegaKeyValueLoader.java (98%) rename src/test/java/io/pravega/connectors/presto/{ => integration}/PravegaLoader.java (99%) rename src/test/java/io/pravega/connectors/presto/{ => integration}/PravegaQueryRunner.java (94%) rename src/test/java/io/pravega/connectors/presto/{ => integration}/PravegaTestUtils.java (96%) rename src/test/java/io/pravega/connectors/presto/{ => integration}/TestPravegaDistributed.java (96%) rename src/test/java/io/pravega/connectors/presto/{ => integration}/TestPravegaKeyValue.java (91%) create mode 100644 src/test/unit-test.xml diff --git a/build.gradle b/build.gradle index aae8db3..e507721 100644 --- a/build.gradle +++ b/build.gradle @@ -97,6 +97,14 @@ distributions { } } +def integration = project.hasProperty("integration") + test { - useTestNG() -} + dependsOn 'cleanTest' + useTestNG() { + suites "src/test/unit-test.xml" + if (integration) { + suites "src/test/integration-test.xml" + } + } +} \ No newline at end of file diff --git a/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java b/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java index a0d2130..b638f33 100644 --- a/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java +++ b/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java @@ -115,9 +115,9 @@ public class PravegaTableDescriptionSupplier } @VisibleForTesting - PravegaTableDescriptionSupplier(PravegaConnectorConfig pravegaConnectorConfig, - Cache schemaCache, - Cache> tableCache) + public PravegaTableDescriptionSupplier(PravegaConnectorConfig pravegaConnectorConfig, + Cache schemaCache, + Cache> tableCache) { this.pravegaConnectorConfig = pravegaConnectorConfig; this.schemaCache = schemaCache; diff --git a/src/test/integration-test.xml b/src/test/integration-test.xml new file mode 100644 index 0000000..647e33d --- /dev/null +++ b/src/test/integration-test.xml @@ -0,0 +1,8 @@ + + + + + + + + diff --git a/src/test/java/io/pravega/connectors/presto/CodecSupplier.java b/src/test/java/io/pravega/connectors/presto/integration/CodecSupplier.java similarity index 98% rename from src/test/java/io/pravega/connectors/presto/CodecSupplier.java rename to src/test/java/io/pravega/connectors/presto/integration/CodecSupplier.java index bedc930..9d65e58 100644 --- a/src/test/java/io/pravega/connectors/presto/CodecSupplier.java +++ b/src/test/java/io/pravega/connectors/presto/integration/CodecSupplier.java @@ -14,7 +14,7 @@ * Note: This class file is from PrestoDb * https://github.com/prestodb/presto/blob/0.247/presto-kafka/src/test/java/com/facebook/presto/kafka/util/CodecSupplier.java */ -package io.pravega.connectors.presto; +package io.pravega.connectors.presto.integration; import com.facebook.airlift.json.JsonCodec; import com.facebook.airlift.json.JsonCodecFactory; diff --git a/src/test/java/io/pravega/connectors/presto/DockerContainer.java b/src/test/java/io/pravega/connectors/presto/integration/DockerContainer.java similarity index 99% rename from src/test/java/io/pravega/connectors/presto/DockerContainer.java rename to src/test/java/io/pravega/connectors/presto/integration/DockerContainer.java index 5db1385..41ef287 100644 --- a/src/test/java/io/pravega/connectors/presto/DockerContainer.java +++ b/src/test/java/io/pravega/connectors/presto/integration/DockerContainer.java @@ -14,7 +14,7 @@ * Note: This class file is from PrestoDb (with minor change to set "cmd" in ContainerConfig.Builder) * https://github.com/prestodb/presto/blob/0.247/presto-testing-docker/src/main/java/com/facebook/presto/testing/docker/DockerContainer.java */ -package io.pravega.connectors.presto; +package io.pravega.connectors.presto.integration; import com.facebook.airlift.log.Logger; import com.google.common.collect.ImmutableList; diff --git a/src/test/java/io/pravega/connectors/presto/EmbeddedPravega.java b/src/test/java/io/pravega/connectors/presto/integration/EmbeddedPravega.java similarity index 96% rename from src/test/java/io/pravega/connectors/presto/EmbeddedPravega.java rename to src/test/java/io/pravega/connectors/presto/integration/EmbeddedPravega.java index d86a126..4094d20 100644 --- a/src/test/java/io/pravega/connectors/presto/EmbeddedPravega.java +++ b/src/test/java/io/pravega/connectors/presto/integration/EmbeddedPravega.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.pravega.connectors.presto; +package io.pravega.connectors.presto.integration; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; diff --git a/src/test/java/io/pravega/connectors/presto/KeyValueTable.java b/src/test/java/io/pravega/connectors/presto/integration/KeyValueTable.java similarity index 95% rename from src/test/java/io/pravega/connectors/presto/KeyValueTable.java rename to src/test/java/io/pravega/connectors/presto/integration/KeyValueTable.java index 30f47e4..3259b56 100644 --- a/src/test/java/io/pravega/connectors/presto/KeyValueTable.java +++ b/src/test/java/io/pravega/connectors/presto/integration/KeyValueTable.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.pravega.connectors.presto; +package io.pravega.connectors.presto.integration; import java.util.ArrayList; import java.util.List; diff --git a/src/test/java/io/pravega/connectors/presto/PravegaKeyValueLoader.java b/src/test/java/io/pravega/connectors/presto/integration/PravegaKeyValueLoader.java similarity index 98% rename from src/test/java/io/pravega/connectors/presto/PravegaKeyValueLoader.java rename to src/test/java/io/pravega/connectors/presto/integration/PravegaKeyValueLoader.java index 6b615a6..53b2f75 100644 --- a/src/test/java/io/pravega/connectors/presto/PravegaKeyValueLoader.java +++ b/src/test/java/io/pravega/connectors/presto/integration/PravegaKeyValueLoader.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.pravega.connectors.presto; +package io.pravega.connectors.presto.integration; import io.pravega.client.ClientConfig; import io.pravega.client.KeyValueTableFactory; diff --git a/src/test/java/io/pravega/connectors/presto/PravegaLoader.java b/src/test/java/io/pravega/connectors/presto/integration/PravegaLoader.java similarity index 99% rename from src/test/java/io/pravega/connectors/presto/PravegaLoader.java rename to src/test/java/io/pravega/connectors/presto/integration/PravegaLoader.java index 2b253d1..36b5c17 100644 --- a/src/test/java/io/pravega/connectors/presto/PravegaLoader.java +++ b/src/test/java/io/pravega/connectors/presto/integration/PravegaLoader.java @@ -16,7 +16,7 @@ * Note: This file contains some code from PrestoDb (majority of the "ResultsSession" implementation) * https://github.com/prestodb/presto/blob/0.247/presto-kafka/src/test/java/com/facebook/presto/kafka/util/KafkaLoader.java */ -package io.pravega.connectors.presto; +package io.pravega.connectors.presto.integration; import com.facebook.presto.Session; import com.facebook.presto.client.Column; diff --git a/src/test/java/io/pravega/connectors/presto/PravegaQueryRunner.java b/src/test/java/io/pravega/connectors/presto/integration/PravegaQueryRunner.java similarity index 94% rename from src/test/java/io/pravega/connectors/presto/PravegaQueryRunner.java rename to src/test/java/io/pravega/connectors/presto/integration/PravegaQueryRunner.java index 0370007..51ce818 100644 --- a/src/test/java/io/pravega/connectors/presto/PravegaQueryRunner.java +++ b/src/test/java/io/pravega/connectors/presto/integration/PravegaQueryRunner.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.pravega.connectors.presto; +package io.pravega.connectors.presto.integration; import com.facebook.airlift.json.JsonCodec; import com.facebook.airlift.log.Logger; @@ -31,6 +31,10 @@ import com.google.common.collect.ImmutableMap; import io.airlift.tpch.TpchTable; import io.pravega.client.admin.StreamManager; +import io.pravega.connectors.presto.PravegaPlugin; +import io.pravega.connectors.presto.PravegaStreamDescription; +import io.pravega.connectors.presto.PravegaTableDescriptionSupplier; +import io.pravega.connectors.presto.PravegaTableName; import java.net.URI; import java.util.Map; @@ -40,8 +44,8 @@ import static com.facebook.presto.testing.TestingSession.testSessionBuilder; import static com.facebook.presto.tpch.TpchMetadata.TINY_SCHEMA_NAME; import static io.airlift.units.Duration.nanosSince; -import static io.pravega.connectors.presto.PravegaTestUtils.getKvStreamDesc; -import static io.pravega.connectors.presto.PravegaTestUtils.getStreamDesc; +import static io.pravega.connectors.presto.integration.PravegaTestUtils.getKvStreamDesc; +import static io.pravega.connectors.presto.integration.PravegaTestUtils.getStreamDesc; import static java.util.Locale.ENGLISH; import static java.util.concurrent.TimeUnit.SECONDS; diff --git a/src/test/java/io/pravega/connectors/presto/PravegaTestUtils.java b/src/test/java/io/pravega/connectors/presto/integration/PravegaTestUtils.java similarity index 96% rename from src/test/java/io/pravega/connectors/presto/PravegaTestUtils.java rename to src/test/java/io/pravega/connectors/presto/integration/PravegaTestUtils.java index 26405c4..2452608 100644 --- a/src/test/java/io/pravega/connectors/presto/PravegaTestUtils.java +++ b/src/test/java/io/pravega/connectors/presto/integration/PravegaTestUtils.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.pravega.connectors.presto; +package io.pravega.connectors.presto.integration; import com.facebook.airlift.json.JsonCodec; import com.facebook.presto.common.QualifiedObjectName; @@ -21,6 +21,9 @@ import com.google.common.io.ByteStreams; import com.google.common.io.CharStreams; import io.pravega.client.admin.StreamManager; +import io.pravega.connectors.presto.PravegaStreamDescription; +import io.pravega.connectors.presto.PravegaStreamFieldDescription; +import io.pravega.connectors.presto.PravegaStreamFieldGroup; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecordBuilder; diff --git a/src/test/java/io/pravega/connectors/presto/TestPravegaDistributed.java b/src/test/java/io/pravega/connectors/presto/integration/TestPravegaDistributed.java similarity index 96% rename from src/test/java/io/pravega/connectors/presto/TestPravegaDistributed.java rename to src/test/java/io/pravega/connectors/presto/integration/TestPravegaDistributed.java index 60e5162..a6b5f96 100644 --- a/src/test/java/io/pravega/connectors/presto/TestPravegaDistributed.java +++ b/src/test/java/io/pravega/connectors/presto/integration/TestPravegaDistributed.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.pravega.connectors.presto; +package io.pravega.connectors.presto.integration; import com.facebook.presto.tests.AbstractTestQueries; import io.airlift.tpch.TpchTable; diff --git a/src/test/java/io/pravega/connectors/presto/TestPravegaKeyValue.java b/src/test/java/io/pravega/connectors/presto/integration/TestPravegaKeyValue.java similarity index 91% rename from src/test/java/io/pravega/connectors/presto/TestPravegaKeyValue.java rename to src/test/java/io/pravega/connectors/presto/integration/TestPravegaKeyValue.java index b5f60bd..4c91f83 100644 --- a/src/test/java/io/pravega/connectors/presto/TestPravegaKeyValue.java +++ b/src/test/java/io/pravega/connectors/presto/integration/TestPravegaKeyValue.java @@ -13,18 +13,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.pravega.connectors.presto; +package io.pravega.connectors.presto.integration; import com.facebook.presto.Session; import com.facebook.presto.testing.MaterializedResult; import com.facebook.presto.tests.DistributedQueryRunner; +import io.pravega.connectors.presto.PravegaStreamDescription; +import io.pravega.connectors.presto.PravegaStreamFieldDescription; import org.testng.annotations.AfterClass; import org.testng.annotations.Test; import java.util.List; import static com.facebook.presto.testing.TestingSession.testSessionBuilder; -import static io.pravega.connectors.presto.PravegaTestUtils.getKvStreamDesc; +import static io.pravega.connectors.presto.integration.PravegaTestUtils.getKvStreamDesc; import static org.testng.Assert.assertEquals; @Test diff --git a/src/test/unit-test.xml b/src/test/unit-test.xml new file mode 100644 index 0000000..9f6b1d9 --- /dev/null +++ b/src/test/unit-test.xml @@ -0,0 +1,8 @@ + + + + + + + + From 4cd341f01736855c297a0a49557e94f7ec640bac Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Mon, 22 Mar 2021 11:48:13 -0700 Subject: [PATCH 05/32] use InProcPravega (instead of external docker container) Signed-off-by: Andrew Robertson --- build.gradle | 5 +- gradle.properties | 1 - .../presto/integration/DockerContainer.java | 257 ------------------ .../presto/integration/EmbeddedPravega.java | 122 +++++++-- .../integration/TestPravegaDistributed.java | 1 + .../integration/TestPravegaKeyValue.java | 1 - src/test/resources/client.truststore.jks | Bin 0 -> 847 bytes src/test/resources/passwd | 1 + src/test/resources/server-cert.crt | 18 ++ src/test/resources/server-key.key | 102 +++++++ src/test/resources/server.keystore.jks | Bin 0 -> 3719 bytes src/test/resources/server.keystore.jks.passwd | 1 + 12 files changed, 227 insertions(+), 282 deletions(-) delete mode 100644 src/test/java/io/pravega/connectors/presto/integration/DockerContainer.java create mode 100755 src/test/resources/client.truststore.jks create mode 100644 src/test/resources/passwd create mode 100644 src/test/resources/server-cert.crt create mode 100644 src/test/resources/server-key.key create mode 100755 src/test/resources/server.keystore.jks create mode 100644 src/test/resources/server.keystore.jks.passwd diff --git a/build.gradle b/build.gradle index e507721..a7b0a10 100644 --- a/build.gradle +++ b/build.gradle @@ -35,6 +35,7 @@ dependencies { compile "javax.validation:validation-api:${javaxValidationVersion}" compile "javax.inject:javax.inject:1" compile "com.fasterxml.jackson.core:jackson-databind:${jacksonVersion}" + compile "io.pravega:pravega-client:${pravegaVersion}" compile "io.pravega:pravega-common:${pravegaVersion}" compile "io.pravega:pravega-shared-protocol:${pravegaVersion}" @@ -61,7 +62,8 @@ dependencies { testImplementation "com.facebook.presto:presto-tests:${prestoVersion}" testImplementation "com.facebook.airlift:testing:${airliftTestingVersion}" - testImplementation "com.starburstdata:starburst-spotify-docker-client:${spotifyDockerClientVersion}" + + testCompile (group: 'io.pravega', name: 'pravega-standalone', version: pravegaVersion) compileOnly "io.airlift:slice:${airliftSliceVersion}" compileOnly "io.airlift:units:${airliftUnitsVersion}" @@ -100,6 +102,7 @@ distributions { def integration = project.hasProperty("integration") test { + maxHeapSize = "2048m" dependsOn 'cleanTest' useTestNG() { suites "src/test/unit-test.xml" diff --git a/gradle.properties b/gradle.properties index 0b5d456..b93efe0 100644 --- a/gradle.properties +++ b/gradle.properties @@ -28,6 +28,5 @@ pravegaVersion=0.9.0 pravegaSchemaRegistryVersion=0.2.0 prestoVersion=0.247 protobufVersion=3.11.4 -spotifyDockerClientVersion=8.11.7-0.6 zookeeperVersion=3.5.7 zookeeperClientVersion=0.10 diff --git a/src/test/java/io/pravega/connectors/presto/integration/DockerContainer.java b/src/test/java/io/pravega/connectors/presto/integration/DockerContainer.java deleted file mode 100644 index 41ef287..0000000 --- a/src/test/java/io/pravega/connectors/presto/integration/DockerContainer.java +++ /dev/null @@ -1,257 +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. - * - * Note: This class file is from PrestoDb (with minor change to set "cmd" in ContainerConfig.Builder) - * https://github.com/prestodb/presto/blob/0.247/presto-testing-docker/src/main/java/com/facebook/presto/testing/docker/DockerContainer.java - */ -package io.pravega.connectors.presto.integration; - -import com.facebook.airlift.log.Logger; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.spotify.docker.client.DefaultDockerClient; -import com.spotify.docker.client.DockerClient; -import com.spotify.docker.client.exceptions.ContainerNotFoundException; -import com.spotify.docker.client.messages.Container; -import com.spotify.docker.client.messages.ContainerConfig; -import com.spotify.docker.client.messages.HostConfig; -import com.spotify.docker.client.messages.PortBinding; -import net.jodah.failsafe.Failsafe; -import net.jodah.failsafe.RetryPolicy; -import net.jodah.failsafe.function.CheckedConsumer; - -import java.io.Closeable; -import java.net.Socket; -import java.time.Duration; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Optional; -import java.util.Set; - -import static com.facebook.airlift.testing.Closeables.closeAllSuppress; -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkState; -import static com.google.common.collect.ImmutableList.toImmutableList; -import static com.google.common.collect.ImmutableMap.toImmutableMap; -import static com.google.common.collect.ImmutableSet.toImmutableSet; -import static com.google.common.collect.MoreCollectors.toOptional; -import static java.lang.String.format; -import static java.time.temporal.ChronoUnit.MINUTES; -import static java.time.temporal.ChronoUnit.SECONDS; -import static java.util.Objects.requireNonNull; - -public final class DockerContainer - implements Closeable -{ - private static final Logger LOG = Logger.get(DockerContainer.class); - - private static final boolean DEBUG = false; - - private static final String HOST_IP = "127.0.0.1"; - private final String image; - private final Map environment; - private DockerClient dockerClient; - private String containerId; - - private Map hostPorts; - - public DockerContainer(String image, String command, List ports, Map environment, CheckedConsumer healthCheck) - { - this.image = requireNonNull(image, "image is null"); - this.environment = ImmutableMap.copyOf(requireNonNull(environment, "environment is null")); - try { - startContainer(ports, command, healthCheck); - } - catch (Exception e) { - closeAllSuppress(e, this); - throw new RuntimeException(e); - } - } - - private void startContainer(List ports, String command, CheckedConsumer healthCheck) - throws Exception - { - dockerClient = DefaultDockerClient.fromEnv().build(); - if (dockerClient.listImages(DockerClient.ListImagesParam.byName(image)).isEmpty()) { - checkState(!image.endsWith("-SNAPSHOT"), "Unavailable snapshot image %s, please build before running tests", image); - LOG.info("Pulling image %s...", image); - dockerClient.pull(image); - } - if (DEBUG) { - Optional testingContainer = dockerClient.listContainers().stream() - .filter(container -> container.image().equals(image)) - .collect(toOptional()); - if (testingContainer.isPresent()) { - containerId = testingContainer.get().id(); - LOG.info("Container for %s already exists with id: %s", image, containerId); - calculateHostPorts(ports); - } - else { - createContainer(ports, command); - } - } - else { - createContainer(ports, command); - } - - checkState(isContainerUp(), "Container was not started properly"); - - LOG.info("Auto-assigned host ports are %s", hostPorts); - - waitForContainer(healthCheck); - } - - private boolean isContainerUp() - { - try { - return dockerClient.inspectContainer(containerId).state().running(); - } - catch (ContainerNotFoundException e) { - return false; - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - - private void createContainer(List ports, String cmd) - throws Exception - { - LOG.info("Starting docker container from image %s", image); - - Map> portBindings = ports.stream() - .collect(toImmutableMap(Object::toString, port -> ImmutableList.of(PortBinding.create(HOST_IP, "0")))); - Set exposedPorts = ports.stream() - .map(Object::toString) - .collect(toImmutableSet()); - - ContainerConfig.Builder builder = ContainerConfig.builder() - .hostConfig(HostConfig.builder() - .portBindings(portBindings) - .build()) - .exposedPorts(exposedPorts) - .env(environment.entrySet().stream() - .map(entry -> format("%s=%s", entry.getKey(), entry.getValue())) - .collect(toImmutableList())) - .image(image); - if (cmd != null) { - builder.cmd(cmd); - } - - containerId = dockerClient.createContainer(builder.build()).id(); - - LOG.info("Started docker container with id: %s", containerId); - - dockerClient.startContainer(containerId); - - calculateHostPorts(ports); - - waitForContainerPorts(ports); - } - - private void waitForContainer(CheckedConsumer healthCheck) - { - RetryPolicy retryPolicy = new RetryPolicy<>() - .withMaxDuration(Duration.of(10, MINUTES)) - .withMaxAttempts(Integer.MAX_VALUE) // limited by MaxDuration - .abortOn(error -> !isContainerUp()) - .onRetry(event -> LOG.info(format("Waiting for container for %s [%s]...", image, event.getLastFailure()))) - .withDelay(Duration.of(10, SECONDS)); - Failsafe.with(retryPolicy).run(() -> healthCheck.accept(this::getHostPort)); - } - - private void waitForContainerPorts(List ports) - { - List hostPorts = ports.stream() - .map(this::getHostPort) - .collect(toImmutableList()); - - RetryPolicy retryPolicy = new RetryPolicy<>() - .withMaxDuration(Duration.of(10, MINUTES)) - .withMaxAttempts(Integer.MAX_VALUE) // limited by MaxDuration - .abortOn(error -> !isContainerUp()) - .withDelay(Duration.of(5, SECONDS)) - .onRetry(event -> LOG.info("Waiting for ports %s that are exposed on %s on %s ...", ports, HOST_IP, hostPorts)); - - Failsafe.with(retryPolicy).run(() -> { - for (int port : ports) { - try (Socket socket = new Socket(HOST_IP, getHostPort(port))) { - checkState(socket.isConnected()); - } - } - }); - } - - private void calculateHostPorts(List ports) - throws Exception - { - hostPorts = dockerClient.inspectContainer(containerId).networkSettings() - .ports() - .entrySet() - .stream() - .filter(entry -> ports.contains(extractPort(entry))) - .collect(toImmutableMap( - entry -> extractPort(entry), - entry -> entry.getValue().stream() - .peek(portBinding -> { - checkState(portBinding.hostIp().equals(HOST_IP), "Unexpected port binding found: %s", portBinding); - }) - .map(PortBinding::hostPort) - .collect(toOptional()) - .map(Integer::parseInt) - .orElseThrow(() -> new IllegalStateException("Could not extract port mapping from: " + entry)))); - } - - public int getHostPort(int port) - { - checkArgument(hostPorts.keySet().contains(port), "Port %s is not bound", port); - return hostPorts.get(port); - } - - private static int extractPort(Entry> entry) - { - checkArgument(!entry.getKey().contains("/udp"), "UDP port binding is not supported"); - return Integer.parseInt(entry.getKey().replace("/tcp", "")); - } - - private void removeContainer(String containerId) - { - try { - LOG.info("Killing container %s", containerId); - dockerClient.killContainer(containerId); - dockerClient.removeContainer(containerId); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - - @Override - public void close() - { - if (dockerClient == null) { - return; - } - if (!DEBUG && containerId != null) { - removeContainer(containerId); - } - dockerClient.close(); - dockerClient = null; - } - - public interface HostPortProvider - { - int getHostPort(int containerPort); - } -} diff --git a/src/test/java/io/pravega/connectors/presto/integration/EmbeddedPravega.java b/src/test/java/io/pravega/connectors/presto/integration/EmbeddedPravega.java index 4094d20..4161034 100644 --- a/src/test/java/io/pravega/connectors/presto/integration/EmbeddedPravega.java +++ b/src/test/java/io/pravega/connectors/presto/integration/EmbeddedPravega.java @@ -1,5 +1,5 @@ /* - * Copyright (c) Pravega Authors. + * Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -7,48 +7,126 @@ * * 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. + * Note: this class is based on SetupUtils from pravega/flink-connectors + * https://github.com/pravega/flink-connectors/blob/v0.9.0/src/test/java/io/pravega/connectors/flink/utils/SetupUtils.java */ package io.pravega.connectors.presto.integration; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; +import com.facebook.airlift.log.Logger; +import io.pravega.local.InProcPravegaCluster; import java.io.Closeable; +import java.io.IOException; +import java.io.InputStream; import java.net.URI; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardCopyOption; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; -public class EmbeddedPravega +public final class EmbeddedPravega implements Closeable { - private final DockerContainer dockerContainer; + private static final Logger log = Logger.get(EmbeddedPravega.class); - public EmbeddedPravega() + private static final String PRAVEGA_USERNAME = "admin"; + private static final String PRAVEGA_PASSWORD = "1111_aaaa"; + private static final String PASSWD_FILE = "passwd"; + private static final String KEY_FILE = "server-key.key"; + private static final String CERT_FILE = "server-cert.crt"; + private static final String STANDALONE_KEYSTORE_FILE = "server.keystore.jks"; + private static final String STANDALONE_TRUSTSTORE_FILE = "client.truststore.jks"; + private static final String STANDALONE_KEYSTORE_PASSWD_FILE = "server.keystore.jks.passwd"; + + private final AtomicBoolean started = new AtomicBoolean(false); + + private final InProcPravegaCluster embeddedPravega; + + private final int controllerPort; + + private final static AtomicInteger servers = new AtomicInteger(); + + public EmbeddedPravega() throws Exception { - this.dockerContainer = new DockerContainer( - "pravega/pravega:latest", - "standalone", - ImmutableList.of(9090, 12345), - ImmutableMap.of( - "HOST_IP", "127.0.0.1"), - EmbeddedPravega::healthCheck); + int server = servers.getAndIncrement(); + this.controllerPort = 9090 + server; + int zkPort = 2181 + server; + int hostPort = 12345 + server; + int restPort = 8080 + server; + + this.embeddedPravega = InProcPravegaCluster.builder() + .isInProcZK(true) + .secureZK(false) + .zkUrl("localhost:" + zkPort) + .zkPort(zkPort) + .isInMemStorage(true) + .isInProcController(true) + .controllerCount(1) + .restServerPort(restPort) + .enableRestServer(true) + .isInProcSegmentStore(true) + .segmentStoreCount(1) + .containerCount(4) + .enableMetrics(false) + .enableAuth(false) + .enableTls(false) + .certFile(getFileFromResource(CERT_FILE)) + .keyFile(getFileFromResource(KEY_FILE)) + .jksKeyFile(getFileFromResource(STANDALONE_KEYSTORE_FILE)) + .jksTrustFile(getFileFromResource(STANDALONE_TRUSTSTORE_FILE)) + .keyPasswordFile(getFileFromResource(STANDALONE_KEYSTORE_PASSWD_FILE)) + .passwdFile(getFileFromResource(PASSWD_FILE)) + .userName(PRAVEGA_USERNAME) + .passwd(PRAVEGA_PASSWORD) + .build(); + + this.embeddedPravega.setControllerPorts(new int[]{controllerPort}); + this.embeddedPravega.setSegmentStorePorts(new int[]{hostPort}); + this.embeddedPravega.start(); } public URI getController() { - return URI.create("tcp://localhost:" + dockerContainer.getHostPort(9090)); + return URI.create("tcp://localhost:" + controllerPort); } - private static void healthCheck(DockerContainer.HostPortProvider hostPortProvider) + static String getFileFromResource(String resourceName) { + try { + Path tempPath = Files.createTempFile("test-", ".tmp"); + tempPath.toFile().deleteOnExit(); + try (InputStream stream = EmbeddedPravega.class.getClassLoader().getResourceAsStream(resourceName)) { + Files.copy(stream, tempPath, StandardCopyOption.REPLACE_EXISTING); + } + return tempPath.toFile().getAbsolutePath(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + public void stop() + { + if (!this.started.compareAndSet(true, false)) { + log.warn("Services not yet started or already stopped, not attempting to stop"); + return; + } + + try { + embeddedPravega.close(); + } + catch (Exception e) { + log.warn("Services did not stop cleanly (" + e.getMessage() + ")", e); + } } @Override public void close() { - dockerContainer.close(); + try { + stop(); + } + catch (Exception quiet) {} } -} +} \ No newline at end of file diff --git a/src/test/java/io/pravega/connectors/presto/integration/TestPravegaDistributed.java b/src/test/java/io/pravega/connectors/presto/integration/TestPravegaDistributed.java index a6b5f96..5e12138 100644 --- a/src/test/java/io/pravega/connectors/presto/integration/TestPravegaDistributed.java +++ b/src/test/java/io/pravega/connectors/presto/integration/TestPravegaDistributed.java @@ -27,6 +27,7 @@ public class TestPravegaDistributed private final EmbeddedPravega pravega; public TestPravegaDistributed() + throws Exception { this(new EmbeddedPravega()); } diff --git a/src/test/java/io/pravega/connectors/presto/integration/TestPravegaKeyValue.java b/src/test/java/io/pravega/connectors/presto/integration/TestPravegaKeyValue.java index 4c91f83..762b7b0 100644 --- a/src/test/java/io/pravega/connectors/presto/integration/TestPravegaKeyValue.java +++ b/src/test/java/io/pravega/connectors/presto/integration/TestPravegaKeyValue.java @@ -29,7 +29,6 @@ import static io.pravega.connectors.presto.integration.PravegaTestUtils.getKvStreamDesc; import static org.testng.Assert.assertEquals; -@Test public class TestPravegaKeyValue { private final EmbeddedPravega pravega; diff --git a/src/test/resources/client.truststore.jks b/src/test/resources/client.truststore.jks new file mode 100755 index 0000000000000000000000000000000000000000..04d192f9e4fca617df51c28cd423833894733886 GIT binary patch literal 847 zcmezO_TO6u1_mY|W(3o0$%#ez`6WPZR?w1fd)FR?K>|cBR4C9fwZBdfjAp;C<`->KtNGqS!#NsZg5Fr za<;Csqk)__uc4)ZiIJ&+v8jcjVU#$pks*+40_Eb@+{CDaY#Adf19KB2KLgNFTue=j zj0_KytJZAnJk^v^_)=|6)07h*qqaSM9FqD^Kq=0$r-6k--7~f2lIjDqjcX5yO4-g* ztMXagaWQ$5^NnyBp~dmj>Qm+K+E;4lKHpk@V#29{`?JNDr@q;(ws3ED&8eNYUo|Xn zY`iU?qQB_igD8Sp(}H`K z|E$*ezr~?q*_AAYsZ7j_42+9|4FU~hf#EO9$0Eie@<2V#B~yFmly&`E_O>gPoC~$ZU7T`||lL*~_jQDt!6= zS?B2FeVPUHRJc{A9`3L96s*lTRrbj9xv1T`!;So^Y&REunH?2<+1dKp6&-b}3U^DH zGyjcuieMiSq~yv{!ZGwYVwYSX@BGe zba%|#b5$L*_K^iOy||0l5(#lHToFq7MyMUhCV}zk62OnN3ND^uHbW U{9=#zyc8`-vl~Ym1^4*@0Ay=bFaQ7m literal 0 HcmV?d00001 diff --git a/src/test/resources/passwd b/src/test/resources/passwd new file mode 100644 index 0000000..9b93429 --- /dev/null +++ b/src/test/resources/passwd @@ -0,0 +1 @@ +admin:353030303a633132666135376233353937356534613430383430373939343839333733616463363433616532363238653930346230333035393666643961316264616661393a3639376330623663396634343864643262663335326463653062613965336439613864306264323839633037626166663563613166333733653631383732353134643961303435613237653130353633633031653364366565316434626534656565636335663666306465663064376165313765646263656638373764396361:prn::*,READ_UPDATE; diff --git a/src/test/resources/server-cert.crt b/src/test/resources/server-cert.crt new file mode 100644 index 0000000..a676149 --- /dev/null +++ b/src/test/resources/server-cert.crt @@ -0,0 +1,18 @@ +-----BEGIN CERTIFICATE----- +MIIC7zCCAdegAwIBAgIJAPTGGecfnvl8MA0GCSqGSIb3DQEBCwUAMBsxGTAXBgNV +BAMMEFByYXZlZ2EtU3RhY2stQ0EwHhcNMTkwNDI1MDQwMjI1WhcNMjEwNDI0MDQw +MjI1WjAhMR8wHQYDVQQDExZzZXJ2ZXIucHJhdmVnYXN0YWNrLmlvMIIBIjANBgkq +hkiG9w0BAQEFAAOCAQ8AMIIBCgKCAQEAt59VBKum67iB7HOOTbrRWX9b0YoQkzbK +4LnK94jIkzKAw6x3dCnSnIkzr9fedrqm4e6p4FY9oCnmOTSsR8IYFaxJg+iUm+O1 +ZsNG25ufiKqQ3q8NhzY9iGrhfVs1nrPk1jx6GdAXs/OmmG83AbcC0ykAviE6OEsg ++tZnWnDEcrqpO5olPcG5Fjrt61e6eZvILzamL0VCaOL2zSShlzt/U5EqVYwQ8R6e +tpzw23Jtx3mjyMZkUfB+7wpP9npZZt3kOOFSgGxPMIdEJ8WXeCO0tgwCK8yKE9g9 +5DaQkzgoDdtabwsYK7U4D/Ua2bVgr2FXjQ3zuin8aIDzaGi0OeAhBQIDAQABozAw +LjAsBgNVHREEJTAjghZzZXJ2ZXIucHJhdmVnYXN0YWNrLmlvgglsb2NhbGhvc3Qw +DQYJKoZIhvcNAQELBQADggEBAHbgmpAtVWZg6OBJUanQT8XTJ54KejfG4bWZ/qPk +HtfE/42/Kcu1OZP+wxwG8VaU2d4JZE3vnnlIHMaV15HXfQypJRsN4AVz7M6oBg8h +9TckSBLtLlderH+fiuwST5ll1zG4upmlbwirTad2jaKdw/oWuhOp3/dMAcfLSyHJ +RKUAkTEJIqLh3rtJ7bYbpMV46HC+C7PvfROrEvxraaZ21gPQXyKynqPLAyhscoWR +dvrY4370ljO8yc99FSCeTOvjG7dHziJ/CQSuZTRGF3QwPquEOxLW7o2+JzImIW/7 +LT2eDnKnx0k1/E6z3IamX+X8ruqTpOPTHB307T1E5Qt5xzI= +-----END CERTIFICATE----- \ No newline at end of file diff --git a/src/test/resources/server-key.key b/src/test/resources/server-key.key new file mode 100644 index 0000000..4d01e85 --- /dev/null +++ b/src/test/resources/server-key.key @@ -0,0 +1,102 @@ +Bag Attributes + friendlyName: server + localKeyID: 54 69 6D 65 20 31 35 35 36 31 36 35 30 30 35 35 31 34 +Key Attributes: +-----BEGIN PRIVATE KEY----- +MIIEvwIBADANBgkqhkiG9w0BAQEFAASCBKkwggSlAgEAAoIBAQC3n1UEq6bruIHs +c45NutFZf1vRihCTNsrgucr3iMiTMoDDrHd0KdKciTOv1952uqbh7qngVj2gKeY5 +NKxHwhgVrEmD6JSb47Vmw0bbm5+IqpDerw2HNj2IauF9WzWes+TWPHoZ0Bez86aY +bzcBtwLTKQC+ITo4SyD61mdacMRyuqk7miU9wbkWOu3rV7p5m8gvNqYvRUJo4vbN +JKGXO39TkSpVjBDxHp62nPDbcm3HeaPIxmRR8H7vCk/2ellm3eQ44VKAbE8wh0Qn +xZd4I7S2DAIrzIoT2D3kNpCTOCgN21pvCxgrtTgP9RrZtWCvYVeNDfO6KfxogPNo +aLQ54CEFAgMBAAECggEBAIaxwCG+8KD1JYkXcwmdwrfN7sKUbngutz+Fylct5UPk +xgPDXZS+oRnmox2YUj3xyTzuX3qKiIYWdiT8eGvi+tvbF8pFL83yiTyqfi3VyH7t +73sKGfhXV2fcTzAvWfPirjhhAIgy11BPqKMk4crGPrI636qYoSZuJfEamZIEKnal +Kv+9QsBYzTzWqeRozwYhgMW0Ej80MbPZ4Vc24RZ/UpoN0CWDTDVsIpFsxypjjo16 +n4mviGc69unAdPMb1r2mi4mMpcWcNeN3IlFquqGDNl4srPGlpvpYQJhTIQufVePJ +zUZIGuUXUn+v1gL10zLPZvJf7TDE7VyEAp5+e4akasECgYEA7ElNJ+pD+ci8wRDj +wyiKYl8fAchizYgnAsh8cvRj9+x9iloJM66zQ7AQasmyfRoSzop5yNtHOvxDaIlU +2OC+6lxJQQfck5+KOAAiBFUezvnSnnvbAtpUqV55z5QWkmGG/XkUXenN8aXwvfV6 +Ozymqw0AQlRcoTMcOm/F28a2O0kCgYEAxvE2iUXBOAO4GRdmj3jE700W/ifHj1J7 +JpcjqNch54ZCJOaxYtS9xGmA/IEofRWuwmQJu+D5r0cZLGZQw5+JSoay46GDAiBA +CeeapkuY7701hy7OWN05GRU9f1gvwIjQBzRd7BVlFq9b6NTfzxT4zErZ0WMmTnPZ +j3QxxY+cW90CgYBEI2NHlmvh1TxxZIB53/AALDQR3Ip5gMDXMZ+JuuY1MZKGWi57 +I+74Hwas9Nsu7lzsCMzT2kAxR1vdAnVgUXdcuB47zZuTSpeeTJOCslU/FJXjNA5V +RUbSzLbN4IOd1NgLsmXoKytDeznwXaDRAk2B2C7x8cgAiG+kfmUNWsFkWQKBgQDA +ji50FaEcH3VsS9sIGSO8X4hWvZgJYBM02es4++mkF64j22psN2/qERg4zqaq03bQ +TlWhlPL3NUDnWazoe50+FjxF2UldoGt7asGkHd3Mhf2ud55gItemMInSnT+zAnkf +0A7seBTdLSQvqwNpg2oBCp3jbi1gl/9HO3cFOqhjvQKBgQDCo32Anu7hcVZW0QzN +Jh7l7yRwSJMjtOlO6EjVlCROIZBrlKSgFxRF3NdGuEuFt2DqDIkMaJ7MEcMpNFFL +L3Z/o/WV/BSxgIh1GiKSanMNYzT16U//qZFAP436unYbDf+zKJ8d41PbW8UAc1n2 +9J0wlCSyQ14id9S2+nEfYGeWFQ== +-----END PRIVATE KEY----- +Bag Attributes + friendlyName: caroot + 2.16.840.1.113894.746875.1.1: +subject=/CN=Pravega-Stack-CA +issuer=/CN=Pravega-Stack-CA +-----BEGIN CERTIFICATE----- +MIIDDDCCAfSgAwIBAgIJAL0NWaqWdxAsMA0GCSqGSIb3DQEBCwUAMBsxGTAXBgNV +BAMMEFByYXZlZ2EtU3RhY2stQ0EwHhcNMTkwNDI1MDM1ODExWhcNMjEwNDI0MDM1 +ODExWjAbMRkwFwYDVQQDDBBQcmF2ZWdhLVN0YWNrLUNBMIIBIjANBgkqhkiG9w0B +AQEFAAOCAQ8AMIIBCgKCAQEA4CN6rLGJyoJkcekmrIKUyPFatuPjVGX+ECJeOYyA +BAgnSWWE0iXgNrGtwhUaPZomekyjiNFjskPYVxwSo1+Wf2Uf3T95K23ntX/IkMpw +35sXp2XsuyahvWt8yrnb6oCgQYHbECQvosHQ+nwHwqGo7ZM2AYwl7YG7ilLFVfcS +wtnBwM8/mU3MJ5tF6pbBEH4fDegYdqt5lSuugJc6Iba9+T0AFZggLhNU1Q127VAn +sLhv/CV1ye6lXhCwFvQKppgiWdgwMwmui1TV/AYGHKPa2NzLFhegXjkzmGBeSDW2 +71Wq8NPmyjOITPep2Il11bmpZlONp/l7LP+0QHim1GoAlQIDAQABo1MwUTAdBgNV +HQ4EFgQU4CdeRGkrmZSuj7QPTZRLhUYimckwHwYDVR0jBBgwFoAU4CdeRGkrmZSu +j7QPTZRLhUYimckwDwYDVR0TAQH/BAUwAwEB/zANBgkqhkiG9w0BAQsFAAOCAQEA +cja7eO93nwRrptcxIPT38yzFk74pcJ4kCyWVw497SRF9bMp24knnFT6uw4EPJQbZ +ovSbWlvTQzvm1CwnOnhHORzM/wLBkYvxIFkTOkQBpEuViks01BoPEOTAgcH22aJ9 +SlYkOmg8csQFCLePlZXrD+Jmh8xOrmuomyVeo4oBcXjUnPmkUG4lEqFvi0y0VHMv +Fg0+8YWa4FkE/ZKzqpO4oWb8HxAtuJ6yRzTy5/X01dae1xrPrs0tQ/jQj6cndsdp +wf+F4TDeQHcV+v2TiBXUl2sYjkV0kOfYggbJUZTyFvMGJuq4HMsNyIm2AVBtWEgZ +S4cqFeExqdaOvo+7mobMsg== +-----END CERTIFICATE----- +Bag Attributes + friendlyName: server + localKeyID: 54 69 6D 65 20 31 35 35 36 31 36 35 30 30 35 35 31 34 +subject=/CN=server.pravegastack.io +issuer=/CN=Pravega-Stack-CA +-----BEGIN CERTIFICATE----- +MIIC7zCCAdegAwIBAgIJAPTGGecfnvl8MA0GCSqGSIb3DQEBCwUAMBsxGTAXBgNV +BAMMEFByYXZlZ2EtU3RhY2stQ0EwHhcNMTkwNDI1MDQwMjI1WhcNMjEwNDI0MDQw +MjI1WjAhMR8wHQYDVQQDExZzZXJ2ZXIucHJhdmVnYXN0YWNrLmlvMIIBIjANBgkq +hkiG9w0BAQEFAAOCAQ8AMIIBCgKCAQEAt59VBKum67iB7HOOTbrRWX9b0YoQkzbK +4LnK94jIkzKAw6x3dCnSnIkzr9fedrqm4e6p4FY9oCnmOTSsR8IYFaxJg+iUm+O1 +ZsNG25ufiKqQ3q8NhzY9iGrhfVs1nrPk1jx6GdAXs/OmmG83AbcC0ykAviE6OEsg ++tZnWnDEcrqpO5olPcG5Fjrt61e6eZvILzamL0VCaOL2zSShlzt/U5EqVYwQ8R6e +tpzw23Jtx3mjyMZkUfB+7wpP9npZZt3kOOFSgGxPMIdEJ8WXeCO0tgwCK8yKE9g9 +5DaQkzgoDdtabwsYK7U4D/Ua2bVgr2FXjQ3zuin8aIDzaGi0OeAhBQIDAQABozAw +LjAsBgNVHREEJTAjghZzZXJ2ZXIucHJhdmVnYXN0YWNrLmlvgglsb2NhbGhvc3Qw +DQYJKoZIhvcNAQELBQADggEBAHbgmpAtVWZg6OBJUanQT8XTJ54KejfG4bWZ/qPk +HtfE/42/Kcu1OZP+wxwG8VaU2d4JZE3vnnlIHMaV15HXfQypJRsN4AVz7M6oBg8h +9TckSBLtLlderH+fiuwST5ll1zG4upmlbwirTad2jaKdw/oWuhOp3/dMAcfLSyHJ +RKUAkTEJIqLh3rtJ7bYbpMV46HC+C7PvfROrEvxraaZ21gPQXyKynqPLAyhscoWR +dvrY4370ljO8yc99FSCeTOvjG7dHziJ/CQSuZTRGF3QwPquEOxLW7o2+JzImIW/7 +LT2eDnKnx0k1/E6z3IamX+X8ruqTpOPTHB307T1E5Qt5xzI= +-----END CERTIFICATE----- +Bag Attributes + friendlyName: CN=Pravega-Stack-CA +subject=/CN=Pravega-Stack-CA +issuer=/CN=Pravega-Stack-CA +-----BEGIN CERTIFICATE----- +MIIDDDCCAfSgAwIBAgIJAL0NWaqWdxAsMA0GCSqGSIb3DQEBCwUAMBsxGTAXBgNV +BAMMEFByYXZlZ2EtU3RhY2stQ0EwHhcNMTkwNDI1MDM1ODExWhcNMjEwNDI0MDM1 +ODExWjAbMRkwFwYDVQQDDBBQcmF2ZWdhLVN0YWNrLUNBMIIBIjANBgkqhkiG9w0B +AQEFAAOCAQ8AMIIBCgKCAQEA4CN6rLGJyoJkcekmrIKUyPFatuPjVGX+ECJeOYyA +BAgnSWWE0iXgNrGtwhUaPZomekyjiNFjskPYVxwSo1+Wf2Uf3T95K23ntX/IkMpw +35sXp2XsuyahvWt8yrnb6oCgQYHbECQvosHQ+nwHwqGo7ZM2AYwl7YG7ilLFVfcS +wtnBwM8/mU3MJ5tF6pbBEH4fDegYdqt5lSuugJc6Iba9+T0AFZggLhNU1Q127VAn +sLhv/CV1ye6lXhCwFvQKppgiWdgwMwmui1TV/AYGHKPa2NzLFhegXjkzmGBeSDW2 +71Wq8NPmyjOITPep2Il11bmpZlONp/l7LP+0QHim1GoAlQIDAQABo1MwUTAdBgNV +HQ4EFgQU4CdeRGkrmZSuj7QPTZRLhUYimckwHwYDVR0jBBgwFoAU4CdeRGkrmZSu +j7QPTZRLhUYimckwDwYDVR0TAQH/BAUwAwEB/zANBgkqhkiG9w0BAQsFAAOCAQEA +cja7eO93nwRrptcxIPT38yzFk74pcJ4kCyWVw497SRF9bMp24knnFT6uw4EPJQbZ +ovSbWlvTQzvm1CwnOnhHORzM/wLBkYvxIFkTOkQBpEuViks01BoPEOTAgcH22aJ9 +SlYkOmg8csQFCLePlZXrD+Jmh8xOrmuomyVeo4oBcXjUnPmkUG4lEqFvi0y0VHMv +Fg0+8YWa4FkE/ZKzqpO4oWb8HxAtuJ6yRzTy5/X01dae1xrPrs0tQ/jQj6cndsdp +wf+F4TDeQHcV+v2TiBXUl2sYjkV0kOfYggbJUZTyFvMGJuq4HMsNyIm2AVBtWEgZ +S4cqFeExqdaOvo+7mobMsg== +-----END CERTIFICATE----- \ No newline at end of file diff --git a/src/test/resources/server.keystore.jks b/src/test/resources/server.keystore.jks new file mode 100755 index 0000000000000000000000000000000000000000..9aa934b0673f2a6648059739139721f1704fa3aa GIT binary patch literal 3719 zcmeH}_g9li7svA?Axkk3f&uA8AyR!p?-rywJ0jbiWbPzEVArk3D*q~CR z2#SDIrG=s>(gXw{AVCBvFY3GR?%A{N{ssHPoH_T*oiq1*@63EZ+Y8$Z5C{bNzkzVD zCwO@HKp=qgsiH+*2-B}}N}w790z-jB*dCAwFtT7!019P+)FZ9#r~08}KqP`iD$*cw z4G92jOc0PC#|!cxV3u&$9+Wx3-q-Q6z3gcpdj}U;oui--4-%&aD#|N?3QDRtoHY+p z9=97R{x|&R%|t-#&xZj_5Ev2Qfb15<3MB#nWNd$6>BE=~;w7(-5~alCwq@(8$w>>x zZIqaeT6`Fs8EfcxqfdNH`QdNRF??DX5`o6K(Y+3jbcW9gpmS|gLmh?R9S)Foo2d+K zyVc?Op3GC|_^D1JyWZtmN6pAwSk}?-5!8W0IgLFV*M51PU9><_2I9pR!s}vBwOFp9 zpTBN=)_pkLqzg;dol9**g$N^O_xY9tq)1c4($qz(>esa(m^&hJTowaJ-vx8*gKCdW z@vH5N`8KEr+zi%&J7QMDpaKgu&SGE_fe^@jGyJxbn$Gfrv5Q1`@Ki%&h z+@)X$C&E};k!c|{NP1EZu!*NMB?cLC2D^6nP8iN$w5d(u9O8)AISjJ(xqcmuX_^dH z-Cti#plb)(m=w1xV#TF?9LO_Di8WHB@o}K0o`pAld7Tq{;>-bcye6TUiTQD2O3FOP zgi}rZjEW8hKBej`2YQSq0C50Ks6%XEQNUY0@l+mYbwNcizw zidBAtK$u`45xxVmBUq&o0348lass>e6i$TCZ)CSo^qgxPvdfye-S@V4mh(=YiZVQp zwXUPcTg)75aR16PVqezE&Yv=8+Z0Uv8Wwo*UPw^QQjq+}9s`wdztt3&U#*u|8QooA zWmE-1XfX95Y1F3luu~r;J~h?_!>ucBG|o zkAn=hsILF4NKYp9o5K^n3AQRy>pNsiQyr_syp-zn2_CY>$KLWdAiC1BTr)T3^Ucgv z{HH6~?FCd#K95KGFCkML0a_OoMksN06s`&O{mUnWX?3FYeVL+5?Ek;%`l`0z(C!!x_vI+KIy4!ak=>jcDldr7dTr0mb^C!Q2W3b5U`kW}X z?Flr`?ZH5#BmT8GGvSnmQ>g>~xL*FI0Oj@{%_ua70DFg5TZWfospR zG~A|5Rq%}HF-&rLm@8U;Z((dn4^3?7E=g4_u@QRs$~do5eZgBa-Rd{3ID9-c9__K7>UZHqXPwT$RF4Y+hii(Bop?^*MFo$gk3WIW`1?X~I@X>_BSKkJ1Qa+fbr#D=0Ft*Udq^mg_MO27L7tz)qN=Aq1 zP01fReY@#Xr)<94vQ_~}1b2sgStg`DaC=eUyiu%3z*c-*>7ksU&!+PtnY|U|w8cnx z&_T@zCo{*O^(TxoTE0DB(4G9W_va2uqx+_&Ha3GDZq)F_UF@oIokBjJ?HPCB@4E?# zLC%kS>{DozOPrUu;2xe~YRJy|8e#WZPd)$5g>+o8Uz-H-B0a8_OW4&m^suvsyfZPz8}!j^&NJ2D%2asp;XS9dKI0co!Ty3c!=-NK zCC!aAw(?LhIcGOt=Z(ABF&p;y1#vQ4RAbxTHRX!()Ina0?WgIF@|d$?#idoNBSxq9 z3kmAyFJe$yJ1+!R=EH+$EHAIqCRffMJ6SR+&zB_hYp;X#8G`%ulwtqWA2jhhIY3uH zB9y*s(nG(K1LFnnj4);W+FxAdpUwXda!>^2<(2**ho2!RiWB}}dvF*R_s>sR&hsaI zcz=fvxho#KCHyl#JiccMFDaO>4*%qR)1&D7TrdY4{Iv*^O5Yk z8>3J*H0oPhFuqtkc19~pa#~HXRR8%tOsV1Z*<|u$rBjpM2>D+0{ad4D$S7s4Xy@_Z zb4rx*sX@&^-X5Ovm4Z7SD!^lCza*qVR9)3bWMlBMwP!P-wpb%WT&uB$TYX{vY;6F! z?T~W8A>Cv6i7zh?WT$C_o=%doj7KdCQL64PjS$>g19ID5TspZFLT5Gm5@_Z0Zc26h zRG6z77Uw$ZJ%_dD#t3{>q@&a`?Cqim0HP7 zeAo&eU!gW8%Jh>TKu`{p`R@OmaB*-y@&AZPWO4Oyuy@6Kc>DaNjX&VPcP!(UtfiCP z?3m%n;vTb>ek_GGQ02vVW%_pRl+aM~PQp{k&Pp}Xc9Q^N`AqWbQI<<4bV`7Mz>Acj zq@mzF#p3+PF(&U%uZj>HqN^$g4A2X5XKhMD@5O#Xo25Gr;i_xX^F5eLObUGyax$AX zxNEtJ->(@1t(``q?Z@*WNjMgK2@P0z z2~m%4vRV{&LSd_+(xz$o+sFdj51Z6EQr=|0fFNT*>-Yz@fL8f`x%ppi{+FBomYYqn X6HXJWf)~QI6*v?P%g)Ego3Z`}PS#*p literal 0 HcmV?d00001 diff --git a/src/test/resources/server.keystore.jks.passwd b/src/test/resources/server.keystore.jks.passwd new file mode 100644 index 0000000..2cc55bc --- /dev/null +++ b/src/test/resources/server.keystore.jks.passwd @@ -0,0 +1 @@ +1111_aaaa \ No newline at end of file From ac70c64ddefee552489182f9ac91e343f11e7cde Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Mon, 22 Mar 2021 12:13:22 -0700 Subject: [PATCH 06/32] rename test classes Signed-off-by: Andrew Robertson --- .../{TestPravegaPlugin.java => PravegaPluginTest.java} | 2 +- ...tPravegaDistributed.java => PravegaDistributedTest.java} | 6 +++--- .../{TestPravegaKeyValue.java => PravegaKeyValueTest.java} | 5 +++-- 3 files changed, 7 insertions(+), 6 deletions(-) rename src/test/java/io/pravega/connectors/presto/{TestPravegaPlugin.java => PravegaPluginTest.java} (98%) rename src/test/java/io/pravega/connectors/presto/integration/{TestPravegaDistributed.java => PravegaDistributedTest.java} (92%) rename src/test/java/io/pravega/connectors/presto/integration/{TestPravegaKeyValue.java => PravegaKeyValueTest.java} (97%) diff --git a/src/test/java/io/pravega/connectors/presto/TestPravegaPlugin.java b/src/test/java/io/pravega/connectors/presto/PravegaPluginTest.java similarity index 98% rename from src/test/java/io/pravega/connectors/presto/TestPravegaPlugin.java rename to src/test/java/io/pravega/connectors/presto/PravegaPluginTest.java index a9943e1..0b1d2f3 100644 --- a/src/test/java/io/pravega/connectors/presto/TestPravegaPlugin.java +++ b/src/test/java/io/pravega/connectors/presto/PravegaPluginTest.java @@ -25,7 +25,7 @@ import static com.google.common.collect.Iterables.getOnlyElement; import static org.testng.Assert.assertNotNull; -public class TestPravegaPlugin +public class PravegaPluginTest { @Test public void testSpinup() diff --git a/src/test/java/io/pravega/connectors/presto/integration/TestPravegaDistributed.java b/src/test/java/io/pravega/connectors/presto/integration/PravegaDistributedTest.java similarity index 92% rename from src/test/java/io/pravega/connectors/presto/integration/TestPravegaDistributed.java rename to src/test/java/io/pravega/connectors/presto/integration/PravegaDistributedTest.java index 5e12138..5f24563 100644 --- a/src/test/java/io/pravega/connectors/presto/integration/TestPravegaDistributed.java +++ b/src/test/java/io/pravega/connectors/presto/integration/PravegaDistributedTest.java @@ -21,18 +21,18 @@ import org.testng.annotations.Test; @Test -public class TestPravegaDistributed +public class PravegaDistributedTest extends AbstractTestQueries { private final EmbeddedPravega pravega; - public TestPravegaDistributed() + public PravegaDistributedTest() throws Exception { this(new EmbeddedPravega()); } - public TestPravegaDistributed(EmbeddedPravega pravega) + public PravegaDistributedTest(EmbeddedPravega pravega) { super(() -> PravegaQueryRunner.createQueryRunner(pravega.getController(), TpchTable.getTables(), java.util.Collections.emptyList())); this.pravega = pravega; diff --git a/src/test/java/io/pravega/connectors/presto/integration/TestPravegaKeyValue.java b/src/test/java/io/pravega/connectors/presto/integration/PravegaKeyValueTest.java similarity index 97% rename from src/test/java/io/pravega/connectors/presto/integration/TestPravegaKeyValue.java rename to src/test/java/io/pravega/connectors/presto/integration/PravegaKeyValueTest.java index 762b7b0..1091887 100644 --- a/src/test/java/io/pravega/connectors/presto/integration/TestPravegaKeyValue.java +++ b/src/test/java/io/pravega/connectors/presto/integration/PravegaKeyValueTest.java @@ -29,13 +29,14 @@ import static io.pravega.connectors.presto.integration.PravegaTestUtils.getKvStreamDesc; import static org.testng.Assert.assertEquals; -public class TestPravegaKeyValue +@Test +public class PravegaKeyValueTest { private final EmbeddedPravega pravega; private final DistributedQueryRunner queryRunner; - public TestPravegaKeyValue() + public PravegaKeyValueTest() throws Exception { this.pravega = new EmbeddedPravega(); From 60fa0d66d64959a75374ba778b63b0619ca0676a Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Mon, 22 Mar 2021 12:16:06 -0700 Subject: [PATCH 07/32] github actions -> jdk11 Signed-off-by: Andrew Robertson --- .github/workflows/pravega-build.yml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/pravega-build.yml b/.github/workflows/pravega-build.yml index 6dd4780..4c43e68 100644 --- a/.github/workflows/pravega-build.yml +++ b/.github/workflows/pravega-build.yml @@ -8,9 +8,9 @@ jobs: steps: - uses: actions/checkout@v2 - - name: Set up JDK 1.8 + - name: Set up JDK 11 uses: actions/setup-java@v1 with: - java-version: 1.8 + java-version: 11 - name: Build with Gradle - run: ./gradlew build + run: ./gradlew build -Pintegration From f95d21e4edb22824bd47eca201a0f776d73dc938 Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Tue, 23 Mar 2021 13:02:21 -0700 Subject: [PATCH 08/32] pluggable schema registry. simplified PravegaTableDescriptionSupplier Signed-off-by: Andrew Robertson --- .../presto/PravegaConnectorConfig.java | 95 +++- .../connectors/presto/PravegaMetadata.java | 3 +- .../presto/PravegaStreamDescription.java | 11 +- .../presto/PravegaStreamFieldGroup.java | 2 +- .../PravegaTableDescriptionSupplier.java | 425 +++--------------- .../connectors/presto/PravegaTableHandle.java | 29 +- .../CompositeSchemaRegistry.java | 102 +++++ .../ConfluentSchemaRegistry.java | 71 +++ .../schemamangement/LocalSchemaRegistry.java | 146 ++++++ .../PravegaSchemaRegistry.java | 196 ++++++++ .../schemamangement/SchemaRegistry.java | 29 ++ .../schemamangement/SchemaSupplier.java | 27 ++ .../presto/util/PravegaNameUtils.java | 14 +- .../presto/util/PravegaStreamDescUtils.java | 36 ++ .../integration/PravegaQueryRunner.java | 2 +- 15 files changed, 792 insertions(+), 396 deletions(-) create mode 100644 src/main/java/io/pravega/connectors/presto/schemamangement/CompositeSchemaRegistry.java create mode 100644 src/main/java/io/pravega/connectors/presto/schemamangement/ConfluentSchemaRegistry.java create mode 100644 src/main/java/io/pravega/connectors/presto/schemamangement/LocalSchemaRegistry.java create mode 100644 src/main/java/io/pravega/connectors/presto/schemamangement/PravegaSchemaRegistry.java create mode 100644 src/main/java/io/pravega/connectors/presto/schemamangement/SchemaRegistry.java create mode 100644 src/main/java/io/pravega/connectors/presto/schemamangement/SchemaSupplier.java diff --git a/src/main/java/io/pravega/connectors/presto/PravegaConnectorConfig.java b/src/main/java/io/pravega/connectors/presto/PravegaConnectorConfig.java index e209adc..941a47b 100644 --- a/src/main/java/io/pravega/connectors/presto/PravegaConnectorConfig.java +++ b/src/main/java/io/pravega/connectors/presto/PravegaConnectorConfig.java @@ -17,14 +17,14 @@ package io.pravega.connectors.presto; import com.facebook.airlift.configuration.Config; +import com.facebook.airlift.configuration.ConfigSecuritySensitive; import javax.validation.constraints.NotNull; import java.io.File; import java.net.URI; -public class PravegaConnectorConfig -{ +public class PravegaConnectorConfig { /** * Pravega Controller URI */ @@ -50,59 +50,114 @@ public class PravegaConnectorConfig */ private File tableDescriptionDir = new File("etc/pravega/"); + + private String awsAccessKey; + + private String awsSecretKey; + + private String awsRegion = "us-east-1"; + + private String awsGlueSchemaRegistry; + + private URI confluentSchemaRegistry; + @NotNull - public URI getControllerURI() - { + public URI getControllerURI() { return this.controllerURI; } @NotNull - public URI getSchemaRegistryURI() - { + public URI getSchemaRegistryURI() { return this.schemaRegistryURI; } @NotNull - public File getTableDescriptionDir() - { + public File getTableDescriptionDir() { return tableDescriptionDir; } @Config("pravega.table-description-dir") - public PravegaConnectorConfig setTableDescriptionDir(File tableDescriptionDir) - { + public PravegaConnectorConfig setTableDescriptionDir(File tableDescriptionDir) { this.tableDescriptionDir = tableDescriptionDir; return this; } - public int getTableCacheExpireSecs() - { + public int getTableCacheExpireSecs() { return this.tableCacheExpireSecs; } @Config("pravega.controller") - public PravegaConnectorConfig setControllerURI(URI controllerURI) - { + public PravegaConnectorConfig setControllerURI(URI controllerURI) { this.controllerURI = controllerURI; return this; } @Config("pravega.schema-registry") - public PravegaConnectorConfig setSchemaRegistryURI(URI schemaRegistryURI) - { + public PravegaConnectorConfig setSchemaRegistryURI(URI schemaRegistryURI) { this.schemaRegistryURI = schemaRegistryURI; return this; } - public boolean isHideInternalColumns() - { + public boolean isHideInternalColumns() { return hideInternalColumns; } @Config("pravega.hide-internal-columns") - public PravegaConnectorConfig setHideInternalColumns(boolean hideInternalColumns) - { + public PravegaConnectorConfig setHideInternalColumns(boolean hideInternalColumns) { this.hideInternalColumns = hideInternalColumns; return this; } + + @Config("pravega.aws.accessKey") + @ConfigSecuritySensitive + public PravegaConnectorConfig setAwsAccessKey(String awsAccessKey) { + this.awsAccessKey = awsAccessKey; + return this; + } + + public String getAwsAccessKey() { + return awsAccessKey; + } + + @Config("pravega.aws.secretKey") + @ConfigSecuritySensitive + public PravegaConnectorConfig setAwsSecretKey(String awsSecretKey) { + this.awsSecretKey = awsSecretKey; + return this; + } + + public String getAwsSecretKey() { + return awsSecretKey; + } + + @Config("pravega.aws.region") + public PravegaConnectorConfig setAwsRegion(String awsRegion) { + this.awsRegion = awsRegion; + return this; + } + + public String getAwsRegion() { + return awsRegion; + } + + @Config("pravega.aws.glue.schema-registry") + public PravegaConnectorConfig setAwsGlueSchemaRegistry(String awsGlueSchemaRegistry) { + this.awsGlueSchemaRegistry = awsGlueSchemaRegistry; + return this; + } + + public String getAwsGlueSchemaRegistry() { + return awsGlueSchemaRegistry; + } + + @Config("pravega.confluentSchemaRegistry") + public PravegaConnectorConfig setConfluentSchemaRegistry(URI confluentSchemaRegistry) { + this.confluentSchemaRegistry = confluentSchemaRegistry; + return this; + } + + public URI getConfluentSchemaRegistry() + { + return confluentSchemaRegistry; + } } diff --git a/src/main/java/io/pravega/connectors/presto/PravegaMetadata.java b/src/main/java/io/pravega/connectors/presto/PravegaMetadata.java index 93744d1..21bbddd 100644 --- a/src/main/java/io/pravega/connectors/presto/PravegaMetadata.java +++ b/src/main/java/io/pravega/connectors/presto/PravegaMetadata.java @@ -104,8 +104,7 @@ public PravegaTableHandle getTableHandle(ConnectorSession session, SchemaTableNa return null; } - return new PravegaTableHandle(connectorId, - schemaTableName.getSchemaName(), + return new PravegaTableHandle(schemaTableName.getSchemaName(), schemaTableName.getTableName(), table.getObjectName(), table.getObjectType(), diff --git a/src/main/java/io/pravega/connectors/presto/PravegaStreamDescription.java b/src/main/java/io/pravega/connectors/presto/PravegaStreamDescription.java index 77454d3..c98f94b 100644 --- a/src/main/java/io/pravega/connectors/presto/PravegaStreamDescription.java +++ b/src/main/java/io/pravega/connectors/presto/PravegaStreamDescription.java @@ -13,7 +13,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package io.pravega.connectors.presto; import com.fasterxml.jackson.annotation.JsonCreator; @@ -67,6 +66,16 @@ public PravegaStreamDescription(PravegaStreamDescription streamDescription, List this.event = Optional.of(event); } + public PravegaStreamDescription(PravegaStreamDescription streamDescription, List event, List objectArgs) + { + this.tableName = streamDescription.tableName; + this.schemaName = streamDescription.schemaName; + this.objectName = streamDescription.objectName; + this.objectType = streamDescription.objectType; + this.objectArgs = Optional.of(objectArgs); + this.event = Optional.of(event); + } + @JsonProperty public Optional getSchemaName() { diff --git a/src/main/java/io/pravega/connectors/presto/PravegaStreamFieldGroup.java b/src/main/java/io/pravega/connectors/presto/PravegaStreamFieldGroup.java index 1164df4..b211214 100644 --- a/src/main/java/io/pravega/connectors/presto/PravegaStreamFieldGroup.java +++ b/src/main/java/io/pravega/connectors/presto/PravegaStreamFieldGroup.java @@ -33,7 +33,7 @@ public class PravegaStreamFieldGroup private final String dataFormat; private final Optional dataSchema; private final Optional> fields; - private final Optional mapping; + private final Optional mapping; // column prefix @JsonCreator public PravegaStreamFieldGroup( diff --git a/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java b/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java index b638f33..cfec988 100644 --- a/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java +++ b/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java @@ -21,50 +21,19 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.cache.Cache; import com.google.common.cache.CacheBuilder; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Streams; -import io.pravega.client.ClientConfig; -import io.pravega.client.admin.StreamManager; -import io.pravega.client.stream.Stream; -import io.pravega.schemaregistry.client.SchemaRegistryClient; -import io.pravega.schemaregistry.client.SchemaRegistryClientConfig; -import io.pravega.schemaregistry.client.SchemaRegistryClientFactory; -import io.pravega.schemaregistry.contract.data.GroupProperties; -import io.pravega.schemaregistry.contract.data.SchemaWithVersion; -import io.pravega.schemaregistry.contract.data.SerializationFormat; +import io.pravega.connectors.presto.schemamangement.CompositeSchemaRegistry; import javax.inject.Inject; -import java.io.File; -import java.io.IOException; -import java.io.UncheckedIOException; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collections; -import java.util.LinkedList; import java.util.List; -import java.util.Locale; import java.util.Optional; import java.util.concurrent.TimeUnit; import java.util.regex.Pattern; import java.util.stream.Collectors; -import static io.pravega.connectors.presto.ProtobufCommon.encodeSchema; -import static io.pravega.connectors.presto.util.PravegaNameUtils.groupId; -import static io.pravega.connectors.presto.util.PravegaNameUtils.kvFieldMapping; -import static io.pravega.connectors.presto.util.PravegaNameUtils.kvTable; import static io.pravega.connectors.presto.util.PravegaNameUtils.multiSourceStream; -import static io.pravega.connectors.presto.util.PravegaNameUtils.temp_streamNameToTableName; -import static io.pravega.connectors.presto.util.PravegaNameUtils.temp_tableNameToStreamName; -import static io.pravega.connectors.presto.util.PravegaSchemaUtils.GROUP_PROPERTIES_INLINE_KEY; -import static io.pravega.connectors.presto.util.PravegaSchemaUtils.GROUP_PROPERTIES_INLINE_KV_KEY; -import static io.pravega.connectors.presto.util.PravegaSchemaUtils.GROUP_PROPERTIES_INLINE_KV_VALUE; -import static io.pravega.connectors.presto.util.PravegaSchemaUtils.INLINE_SUFFIX; -import static io.pravega.connectors.presto.util.PravegaSchemaUtils.readSchema; -import static io.pravega.connectors.presto.util.PravegaStreamDescUtils.mapFieldsFromSchema; -import static java.nio.file.Files.readAllBytes; -import static java.util.Arrays.asList; import static java.util.Objects.requireNonNull; // pravega scope is a namespace for streams. stream is unique within scope. @@ -83,7 +52,7 @@ public class PravegaTableDescriptionSupplier { private static final Logger log = Logger.get(PravegaTableDescriptionSupplier.class); - private final PravegaConnectorConfig pravegaConnectorConfig; + private final CompositeSchemaRegistry schemaRegistry; private Cache schemaCache; @@ -91,14 +60,11 @@ public class PravegaTableDescriptionSupplier private JsonCodec streamDescriptionCodec; - // "inline" means that event was written using schema registry wrapped client and schema encoding id - // is within the raw event data in pravega @Inject PravegaTableDescriptionSupplier(PravegaConnectorConfig pravegaConnectorConfig, JsonCodec streamDescriptionCodec) { requireNonNull(pravegaConnectorConfig, "pravegaConfig is null"); - this.pravegaConnectorConfig = pravegaConnectorConfig; this.streamDescriptionCodec = streamDescriptionCodec; // there will be many successive calls to listSchemas + listTables in short time period @@ -112,16 +78,17 @@ public class PravegaTableDescriptionSupplier this.tableCache = CacheBuilder.newBuilder() .expireAfterWrite(pravegaConnectorConfig.getTableCacheExpireSecs(), TimeUnit.SECONDS) .build(); + + this.schemaRegistry = new CompositeSchemaRegistry(pravegaConnectorConfig, streamDescriptionCodec); } @VisibleForTesting - public PravegaTableDescriptionSupplier(PravegaConnectorConfig pravegaConnectorConfig, - Cache schemaCache, + public PravegaTableDescriptionSupplier(Cache schemaCache, Cache> tableCache) { - this.pravegaConnectorConfig = pravegaConnectorConfig; this.schemaCache = schemaCache; this.tableCache = tableCache; + this.schemaRegistry = null; } public List listSchemas() @@ -130,16 +97,10 @@ public List listSchemas() // they are inserted to cache at same time so will all be same state final List schemas = schemaCache.asMap().keySet().stream().collect(Collectors.toList()); if (schemas.isEmpty()) { - listLocalSchemas().forEach(schema -> schemaCache.put(schema, new Object())); - - try (StreamManager streamManager = - StreamManager.create(ClientConfig.builder().controllerURI( - pravegaConnectorConfig.getControllerURI()).build())) { - Streams.stream(streamManager.listScopes()).filter(s -> !internalObject(s)).forEach(schema -> { - schemas.add(schema); - schemaCache.put(schema, new Object()); - }); - } + schemaRegistry.listSchemas().forEach(schema -> { + schemas.add(schema); + schemaCache.put(schema, new Object()); + }); } else { log.info("serving listSchemas() from cache"); @@ -149,75 +110,48 @@ public List listSchemas() public List listTables(Optional schema) { - List schemas = schema.isPresent() ? Collections.singletonList(schema.get()) : listSchemas(); + List schemas = schema.map(Collections::singletonList).orElseGet(this::listSchemas); + + List tableList = new ArrayList<>(); + + for (String s : schemas) { + List tableListForSchema = + tableCache.asMap().keySet().stream() + .filter(streamDesc -> streamDesc.getSchemaTableName().getSchemaName().startsWith(s)) + .collect(Collectors.toList()); - StreamManager streamManager = null; + if (tableListForSchema.isEmpty()) { - try { - List tableList = new ArrayList<>(); + List compositeStreams = new ArrayList<>(); - for (String s : schemas) { - List tableListForSchema = - tableCache.asMap().keySet().stream() - .filter(streamDesc -> streamDesc.getSchemaTableName().getSchemaName().startsWith(s)) - .collect(Collectors.toList()); + schemaRegistry.listTables(s).forEach(table -> { - // not all tables inserted to cache at same time - if (tableListForSchema.isEmpty()) { - if (streamManager == null) { - streamManager = StreamManager.create( - ClientConfig.builder().controllerURI(pravegaConnectorConfig.getControllerURI()).build()); + boolean hidden = + compositeStreams.stream().anyMatch(p -> p.matcher(table.getTableName()).matches()); + + PravegaTableName pravegaTableName = new PravegaTableName(s, table.getTableName(), hidden); + + // don't clobber existing entry + if (tableCache.getIfPresent(pravegaTableName) == null || + !tableCache.getIfPresent(pravegaTableName).isPresent()) { + tableCache.put(pravegaTableName, Optional.empty()); } - List compositeStreams = new ArrayList<>(); - - // local takes precedence. list before pravega. ifAbsent used later to not clobber. - listLocalTables(s).forEach(table -> { - PravegaTableName pravegaTableName = new PravegaTableName(s, table); - - // don't clobber existing entry - if (tableCache.getIfPresent(pravegaTableName) == null || - !tableCache.getIfPresent(pravegaTableName).isPresent()) { - tableCache.put(pravegaTableName, Optional.empty()); - } - - // load .json def to get stream name in order to determine type - PravegaStreamDescription localTable = getLocalTable(pravegaTableName.getSchemaTableName()); - if (multiSourceStream(localTable)) { - compositeStreams.add(Pattern.compile(localTable.getObjectName())); - } - }); - - // (underlying streams used by kv table are seen as internal and thus are skipped) - Streams.stream(streamManager.listStreams(s)) - .filter(stream -> !internalStream(stream)) - .forEach(stream -> { - boolean hidden = - compositeStreams.stream().anyMatch(p -> p.matcher(stream.getStreamName()).matches()); - // ifAbsent - don't clobber table description if we have it - PravegaTableName tableName = new PravegaTableName(s, temp_streamNameToTableName(stream.getStreamName()), hidden); - if (tableCache.getIfPresent(tableName) == null || - !tableCache.getIfPresent(tableName).isPresent()) { - tableCache.put(tableName, Optional.empty()); - } - }); - } - else { - log.info("serving listTables(%s) from cache", s); - } - - tableList.addAll(tableCache.asMap().keySet().stream() - .filter(pravegaStreamDescription -> - pravegaStreamDescription.getSchemaTableName().getSchemaName().startsWith(s)) - .collect(Collectors.toList())); + if (multiSourceStream(table)) { + compositeStreams.add(Pattern.compile(table.getObjectName())); + } + }); } - return tableList; - } - finally { - if (streamManager != null) { - streamManager.close(); + else { + log.info("serving listTables(%s) from cache", s); } + + tableList.addAll(tableCache.asMap().keySet().stream() + .filter(pravegaStreamDescription -> + pravegaStreamDescription.getSchemaTableName().getSchemaName().startsWith(s)) + .collect(Collectors.toList())); } + return tableList; } public PravegaStreamDescription getTable(SchemaTableName schemaTableName) @@ -229,274 +163,53 @@ public PravegaStreamDescription getTable(SchemaTableName schemaTableName) return cachedTable.get(); } - PravegaStreamDescription table = getLocalTable(schemaTableName); - if (table != null) { - log.info("found local schema for '%s'", schemaTableName); - - // kv this is list of key family (defined in local schema file) - // for multi source stream this is list of composite streams (empty here, to be filled in later) - Optional> objectArgs = table.getObjectArgs(); - - // field definitions can come from 1 of 4 places - // (1) defined in local .json schema ("event/fields") - // (2) uri in "dataSchema" field - // (3) lookup from a source stream (if multi source stream) - // (4) lookup directly in schema registry (if kv table) - - Optional> fieldGroups = Optional.empty(); + PravegaStreamDescription table = schemaRegistry.getTable(schemaTableName); - if (fieldsDefined(table)) { - // case (1) - no-op - log.info("fields defined in schema file %s", schemaTableName); - fieldGroups = Optional.of(new LinkedList<>(table.getEvent().get())); - } - else if (table.getEvent().isPresent() && - table.getEvent().get().get(0).getDataSchema().isPresent()) { - fieldGroups = Optional.of(new LinkedList<>()); - - // case (2) uri containing schema - List finalFieldGroups = fieldGroups.get(); - for (int i = 0; i < table.getEvent().get().size(); i++) { - PravegaStreamFieldGroup event = table.getEvent().get().get(i); - String colPrefix = event.getMapping().orElse( - table.getEvent().get().size() > 1 ? kvFieldMapping(i) : ""); - Optional dataSchema = Optional.of(readSchema(event.getDataSchema().get())); - PravegaStreamFieldGroup fieldGroup = - new PravegaStreamFieldGroup(event.getDataFormat(), - Optional.empty(), - dataSchema, - Optional.of( - mapFieldsFromSchema(colPrefix, event.getDataFormat(), dataSchema.get()))); - finalFieldGroups.add(fieldGroup); - } - } - else if (kvTable(table)) { - fieldGroups = fieldGroupsFromSchemaRegistry(schemaTableName); + if (multiSourceStream(table)) { + List objectArgs = multiSourceStreamComponents(schemaTableName, table.getObjectName()); + if (objectArgs.isEmpty()) { + throw new IllegalArgumentException("could not get component streams for " + schemaTableName); } - if (multiSourceStream(table)) { - // stream name will be some regex. - // find all the possible source streams. - Pattern pattern = Pattern.compile(table.getObjectName()); - - List sourceTableNames = - listTables(Optional.of(schemaTableName.getSchemaName())).stream() - .filter(t -> pattern.matcher(t.getSchemaTableName().getTableName()).matches()) - .collect(Collectors.toList()); - - objectArgs = Optional.of(sourceTableNames.stream() - .map(PravegaTableName::getSchemaTableName) - .map(SchemaTableName::getTableName) - .collect(Collectors.toList())); - - if (!fieldGroups.isPresent()) { - // case (3) schema not already defined, look one up - // lookup actual schema from any of them - implies all sources are the same - PravegaStreamDescription sourceTable = sourceTableNames.isEmpty() - ? null - : getTable(sourceTableNames.get(0).getSchemaTableName()); - if (sourceTable == null) { - throw new IllegalArgumentException("no stream found for multi source"); - } - fieldGroups = Optional.of(new LinkedList<>()); - fieldGroups.get().add(new PravegaStreamFieldGroup( - sourceTable.getEvent().get().get(0).getDataFormat(), - Optional.empty(), - sourceTable.getEvent().get().get(0).getDataSchema(), - Optional.of(sourceTable.getEvent().get().get(0).getFields()))); - } + List fieldGroups = table.getEvent().orElse(new ArrayList<>(1)); + if (fieldGroups.isEmpty()) { + fieldGroups = schemaRegistry.getSchema(new SchemaTableName(schemaTableName.getSchemaName(), objectArgs.get(0))); } - fieldGroups.orElseThrow(() -> - new IllegalArgumentException("unable to determine schema for " + schemaTableName)); - - // our final table definition. use schema that we looked up, and set all source stream names here - table = new PravegaStreamDescription(schemaTableName.getTableName(), - Optional.of(schemaTableName.getSchemaName()), - table.getObjectName(), - Optional.of(table.getObjectType()), - objectArgs, - fieldGroups); - - tableCache.put(pravegaTableName, Optional.of(table)); - return table; + table = new PravegaStreamDescription(table, fieldGroups, objectArgs); + } + else if (!fieldsDefined(table)) { + table = new PravegaStreamDescription(table, schemaRegistry.getSchema(schemaTableName)); } - Optional> fieldGroups = fieldGroupsFromSchemaRegistry(schemaTableName); - - table = new PravegaStreamDescription( - schemaTableName.getTableName(), - Optional.of(schemaTableName.getSchemaName()), - temp_tableNameToStreamName(schemaTableName.getTableName()), - Optional.of(ObjectType.STREAM), - Optional.empty() /* args */, - fieldGroups); tableCache.put(pravegaTableName, Optional.of(table)); return table; } - /** - * construct PravegaStreamFieldGroup by looking up schema in schema registry - * - * @param schemaTableName - * @return - */ - private Optional> fieldGroupsFromSchemaRegistry(final SchemaTableName schemaTableName) + private List multiSourceStreamComponents(SchemaTableName schemaTableName, String sourcePattern) { - log.info("look up description of '%s' from pravega", schemaTableName); - String groupName = groupId(schemaTableName.getSchemaName(), temp_tableNameToStreamName(schemaTableName.getTableName())); - - SchemaRegistryClientConfig registryConfig = - SchemaRegistryClientConfig.builder() - .schemaRegistryUri(pravegaConnectorConfig.getSchemaRegistryURI()).build(); - SchemaRegistryClient registryClient = SchemaRegistryClientFactory.withDefaultNamespace(registryConfig); - - List fieldGroups = new ArrayList<>(2); - - GroupProperties properties = - registryClient.getGroupProperties(groupName); - - List schemas = registryClient.getSchemas(groupName); - if (schemas.size() == 0 || schemas.size() > 2) { - throw new IllegalStateException(schemaTableName + " has " + schemas.size() + " registered schemas. expecting either 1 or 2"); - } - - // kv table will have > 1 schema. key+value likely different types - boolean kv = schemas.size() > 1; + Pattern pattern = Pattern.compile(sourcePattern); - for (int i = 0; i < schemas.size(); i++) { - // colPrefix used for display so can differentiate between fields from key or value - String colPrefix = kv ? kvFieldMapping(i) : ""; - - SerializationFormat format = schemas.get(i).getSchemaInfo().getSerializationFormat(); - fieldGroups.add(new PravegaStreamFieldGroup( - dataFormat(properties.getProperties(), format, kv, i), - Optional.of(colPrefix), - dataSchema(format, schemas.get(i)), - Optional.of(mapFieldsFromSchema(colPrefix, format, schemas.get(i))))); - } - - return Optional.of(fieldGroups); - } - - private static boolean fieldsDefined(PravegaStreamDescription table) - { - // event is optional, fields within event is also optional - // for kv table - 0 or 2 schemas. so fine to just check for 1. - return table.getEvent().isPresent() && (table.getEvent().get().get(0).getFields() != null); - } - - private List listLocalSchemas() - { - return localSchemaStream() - .map(file -> file.getName().split("\\.")[0]) + return listTables(Optional.of(schemaTableName.getSchemaName())).stream() + .map(PravegaTableName::getSchemaTableName) + .map(SchemaTableName::getTableName) + .filter(tableName -> pattern.matcher(tableName).matches()) .collect(Collectors.toList()); } - // scope.stream -> schema.table - private List listLocalTables(String schema) - { - return localSchemaStream() - .filter(file -> file.getName().endsWith(".json")) - .filter(file -> file.getName().startsWith(schema)) - .filter(file -> file.getName().split("\\.").length == 3) - .map(file -> file.getName().split("\\.")[1]) - .collect(Collectors.toList()); - } - - private PravegaStreamDescription getLocalTable(SchemaTableName schemaTableName) + private static boolean fieldsDefined(PravegaStreamDescription table) { - try { - File file = new File(pravegaConnectorConfig.getTableDescriptionDir(), - String.format("%s.%s.json", schemaTableName.getSchemaName(), schemaTableName.getTableName())); - if (!file.exists()) { - return null; - } - return streamDescriptionCodec.fromJson(readAllBytes(file.toPath())); - } - catch (IOException e) { - log.error("%s", e); - throw new UncheckedIOException(e); + if (!table.getEvent().isPresent() || + table.getEvent().get().isEmpty()) { + return false; } - catch (RuntimeException e) { - log.error("%s", e); - throw e; - } - } - - private java.util.stream.Stream localSchemaStream() - { - return listFiles(pravegaConnectorConfig.getTableDescriptionDir()).stream() - .filter(file -> file.isFile() && file.getName().endsWith(".json")); - } - private static List listFiles(File dir) - { - if ((dir != null) && dir.isDirectory()) { - File[] files = dir.listFiles(); - if (files != null) { - log.debug("Considering files: %s", asList(files)); - return ImmutableList.copyOf(files); + for (PravegaStreamFieldGroup fieldGroup : table.getEvent().get()) { + if (fieldGroup.getFields() == null) { + return false; } } - return ImmutableList.of(); - } - - private static String dataFormat(ImmutableMap groupProperties, - SerializationFormat format, - boolean kvTable, - int kvIdx) - { - /* - TODO: auto-detect https://github.com/pravega/pravega-sql/issues/58 - - (1) no schema registry. - (2) Register and evolve schemas in registry but do not use registry client while writing data - (3) Register schemas in the registry and use registry client to encode schema Id with payload - "inline" is for #3. for e.g. "avro" -> "avro-inline". PravegaRecordSetProvider is interested in this - - hopefully this can all go away (see linked issue 58 above) - - but for now the following is our convention - if "inline" exists in our properties, all data uses SR - else if it is a kv table key+value may be different. both, neither, or either may use SR - look for "inlinekey" / "inlinevalue" - */ - - String key = GROUP_PROPERTIES_INLINE_KEY; - - if (kvTable && !groupProperties.containsKey(key)) { - key = kvIdx == 0 ? GROUP_PROPERTIES_INLINE_KV_KEY : GROUP_PROPERTIES_INLINE_KV_VALUE; - } - - String finalFormat = format == SerializationFormat.Custom - ? format.getFullTypeName().toLowerCase(Locale.ENGLISH) - : format.name().toLowerCase(Locale.ENGLISH); - return finalFormat + (groupProperties.containsKey(key) ? INLINE_SUFFIX : ""); - } - private static Optional dataSchema(SerializationFormat format, SchemaWithVersion schemaWithVersion) - { - // it is intentional that nothing is returned for Custom - // pass schema to row decoders. refer to PravegaRecordSetProvider - switch (format) { - case Protobuf: - return Optional.of(encodeSchema(schemaWithVersion)); - case Avro: - return Optional.of(new String(schemaWithVersion.getSchemaInfo().getSchemaData().array(), StandardCharsets.UTF_8)); - default: - return Optional.empty(); - } - } - - private static boolean internalStream(Stream stream) - { - return internalObject(stream.getStreamName()); - } - - private static boolean internalObject(String object) - { - return object.startsWith("_") /* pravega internal */ || - object.endsWith("-SC") /* application internal - stream cuts */; + return true; } } diff --git a/src/main/java/io/pravega/connectors/presto/PravegaTableHandle.java b/src/main/java/io/pravega/connectors/presto/PravegaTableHandle.java index 4cc23fb..78bb238 100644 --- a/src/main/java/io/pravega/connectors/presto/PravegaTableHandle.java +++ b/src/main/java/io/pravega/connectors/presto/PravegaTableHandle.java @@ -34,11 +34,6 @@ public final class PravegaTableHandle implements ConnectorTableHandle { - /** - * connector id - */ - private final String connectorId; - /** * The schema name for this table. Is set through configuration and read */ @@ -64,7 +59,6 @@ public final class PravegaTableHandle @JsonCreator public PravegaTableHandle( - @JsonProperty("connectorId") String connectorId, @JsonProperty("schemaName") String schemaName, @JsonProperty("tableName") String tableName, @JsonProperty("objectName") String objectName, @@ -73,7 +67,6 @@ public PravegaTableHandle( @JsonProperty("schema") List schema, @JsonProperty("schemaRegistryGroupId") String schemaRegistryGroupId) { - this.connectorId = requireNonNull(connectorId, "connectorId is null"); this.schemaName = requireNonNull(schemaName, "schemaName is null"); this.tableName = requireNonNull(tableName, "tableName is null"); this.objectName = requireNonNull(objectName, "objectName is null"); @@ -83,10 +76,20 @@ public PravegaTableHandle( this.schemaRegistryGroupId = requireNonNull(schemaRegistryGroupId, "schemaRegistryGroupId is null"); } - @JsonProperty - public String getConnectorId() + public PravegaTableHandle( + @JsonProperty("schemaName") String schemaName, + @JsonProperty("tableName") String tableName, + @JsonProperty("objectName") String objectName, + @JsonProperty("objectType") ObjectType objectType, + @JsonProperty("objectArgs") Optional> objectArgs) { - return connectorId; + this.schemaName = requireNonNull(schemaName, "schemaName is null"); + this.tableName = requireNonNull(tableName, "tableName is null"); + this.objectName = requireNonNull(objectName, "objectName is null"); + this.objectType = requireNonNull(objectType, "objectType is null"); + this.objectArgs = objectArgs; + this.schema = null; + this.schemaRegistryGroupId = null; } @JsonProperty @@ -139,7 +142,7 @@ public SchemaTableName toSchemaTableName() @Override public int hashCode() { - return Objects.hash(connectorId, schemaName, tableName, objectName, objectType, schema); + return Objects.hash(schemaName, tableName, objectName, objectType, schema); } @Override @@ -153,8 +156,7 @@ public boolean equals(Object obj) } PravegaTableHandle other = (PravegaTableHandle) obj; - return Objects.equals(this.connectorId, other.connectorId) - && Objects.equals(this.schemaName, other.schemaName) + return Objects.equals(this.schemaName, other.schemaName) && Objects.equals(this.tableName, other.tableName) && Objects.equals(this.objectName, other.objectName) && Objects.equals(this.objectType, other.objectType) @@ -165,7 +167,6 @@ public boolean equals(Object obj) public String toString() { return toStringHelper(this) - .add("connectorId", connectorId) .add("schemaName", schemaName) .add("tableName", tableName) .add("objectName", objectName) diff --git a/src/main/java/io/pravega/connectors/presto/schemamangement/CompositeSchemaRegistry.java b/src/main/java/io/pravega/connectors/presto/schemamangement/CompositeSchemaRegistry.java new file mode 100644 index 0000000..33c84c3 --- /dev/null +++ b/src/main/java/io/pravega/connectors/presto/schemamangement/CompositeSchemaRegistry.java @@ -0,0 +1,102 @@ +/* + * Copyright (c) Pravega Authors. + * + * 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.pravega.connectors.presto.schemamangement; + +import com.facebook.airlift.json.JsonCodec; +import com.facebook.presto.spi.SchemaTableName; +import io.pravega.connectors.presto.PravegaConnectorConfig; +import io.pravega.connectors.presto.PravegaStreamDescription; +import io.pravega.connectors.presto.PravegaStreamFieldGroup; +import io.pravega.connectors.presto.PravegaTableHandle; + +import java.util.ArrayList; +import java.util.List; + +public class CompositeSchemaRegistry + implements SchemaSupplier, SchemaRegistry +{ + private final List schemaSuppliers; + + private final List schemaRegistries; + + public CompositeSchemaRegistry(PravegaConnectorConfig config, JsonCodec streamDescriptionCodec) + { + schemaSuppliers = new ArrayList<>(); + schemaRegistries = new ArrayList<>(); + + // local will override, always add first + if (config.getTableDescriptionDir() != null && + config.getTableDescriptionDir().exists() && + config.getTableDescriptionDir().isDirectory()) { + LocalSchemaRegistry schemaRegistry = + new LocalSchemaRegistry(config.getTableDescriptionDir(), streamDescriptionCodec); + schemaSuppliers.add(schemaRegistry); + schemaRegistries.add(schemaRegistry); + } + + if (config.getSchemaRegistryURI() != null) { + PravegaSchemaRegistry schemaRegistry = + new PravegaSchemaRegistry(config.getControllerURI(), config.getSchemaRegistryURI()); + schemaSuppliers.add(schemaRegistry); + schemaRegistries.add(schemaRegistry); + } + + if (config.getConfluentSchemaRegistry() != null) { + ConfluentSchemaRegistry schemaRegistry = + new ConfluentSchemaRegistry(config.getConfluentSchemaRegistry()); + schemaRegistries.add(schemaRegistry); + } + } + + @Override + public List listSchemas() + { + final List schemas = new ArrayList<>(); + schemaSuppliers.forEach(p -> schemas.addAll(p.listSchemas())); + return schemas; + } + + @Override + public List listTables(String schema) + { + final List tables = new ArrayList<>(); + schemaSuppliers.forEach(p -> tables.addAll(p.listTables(schema))); + return tables; + } + + @Override + public List getSchema(SchemaTableName schemaTableName) { + for (SchemaRegistry schemaRegistry : schemaRegistries) { + List schema = schemaRegistry.getSchema(schemaTableName); + if (schema != null) { + return schema; + } + } + return null; + } + + @Override + public PravegaStreamDescription getTable(SchemaTableName schemaTableName) + { + for (SchemaRegistry schemaRegistry : schemaRegistries) { + PravegaStreamDescription streamDescription = schemaRegistry.getTable(schemaTableName); + if (streamDescription != null) { + return streamDescription; + } + } + return null; + } +} diff --git a/src/main/java/io/pravega/connectors/presto/schemamangement/ConfluentSchemaRegistry.java b/src/main/java/io/pravega/connectors/presto/schemamangement/ConfluentSchemaRegistry.java new file mode 100644 index 0000000..6b53a0d --- /dev/null +++ b/src/main/java/io/pravega/connectors/presto/schemamangement/ConfluentSchemaRegistry.java @@ -0,0 +1,71 @@ +/* + * Copyright (c) Pravega Authors. + * + * 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.pravega.connectors.presto.schemamangement; + +import com.facebook.presto.spi.SchemaTableName; +import io.pravega.connectors.presto.PravegaStreamDescription; +import io.pravega.connectors.presto.PravegaStreamFieldGroup; + +import java.net.URI; +import java.util.List; + +public class ConfluentSchemaRegistry + implements SchemaRegistry +{ + //private final SchemaRegistryClient schemaRegistryClient; + + public ConfluentSchemaRegistry(URI registryURI) + { + //this.schemaRegistryClient = new CachedSchemaRegistryClient(registryURI.toASCIIString(), Integer.MAX_VALUE); + } + + @Override + public List getSchema(SchemaTableName schemaTableName) + { + /* + try { + SchemaMetadata metadata = schemaRegistryClient.getLatestSchemaMetadata(format(schemaTableName)); + + List fields = + mapFieldsFromSchema("", AVRO, metadata.getSchema()); + + return Collections.singletonList( + new PravegaStreamFieldGroup(AVRO, + Optional.empty(), + Optional.of(metadata.getSchema()), + Optional.of(fields))); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + catch (Exception e) { + throw new RuntimeException(e); + } + */ + return null; + } + + @Override + public PravegaStreamDescription getTable(SchemaTableName schemaTableName) + { + return null; + } + + static String format(SchemaTableName schemaTableName) + { + return String.format("%s-%s", schemaTableName.getSchemaName(), schemaTableName.getTableName()); + } +} \ No newline at end of file diff --git a/src/main/java/io/pravega/connectors/presto/schemamangement/LocalSchemaRegistry.java b/src/main/java/io/pravega/connectors/presto/schemamangement/LocalSchemaRegistry.java new file mode 100644 index 0000000..a64eeb9 --- /dev/null +++ b/src/main/java/io/pravega/connectors/presto/schemamangement/LocalSchemaRegistry.java @@ -0,0 +1,146 @@ +/* + * Copyright (c) Pravega Authors. + * + * 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.pravega.connectors.presto.schemamangement; + +import com.facebook.airlift.json.JsonCodec; +import com.facebook.presto.spi.SchemaTableName; +import com.google.common.collect.ImmutableList; +import io.pravega.connectors.presto.ObjectType; +import io.pravega.connectors.presto.PravegaStreamDescription; +import io.pravega.connectors.presto.PravegaStreamFieldGroup; +import io.pravega.connectors.presto.PravegaTableHandle; + +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + +import static io.pravega.connectors.presto.util.PravegaNameUtils.kvFieldMapping; +import static io.pravega.connectors.presto.util.PravegaStreamDescUtils.resolveAllSchemas; +import static java.nio.file.Files.readAllBytes; + +public class LocalSchemaRegistry + implements SchemaSupplier, SchemaRegistry +{ + private final File localTableDir; + + private final JsonCodec streamDescriptionCodec; + + public LocalSchemaRegistry(File localTableDir, JsonCodec streamDescriptionCodec) + { + this.localTableDir = localTableDir; + this.streamDescriptionCodec = streamDescriptionCodec; + } + + @Override + public List listSchemas() + { + return localSchemaStream() + .map(file -> file.getName().split("\\.")[0]) + .collect(Collectors.toList()); + } + + @Override + public List listTables(String schema) + { + final List tables = new ArrayList<>(); + + localSchemaStream() + .filter(file -> file.getName().startsWith(schema)) + .filter(file -> file.getName().split("\\.").length == 3) + .map(file -> file.getName().split("\\.")[1]) + .map(file -> getLocalTable(new SchemaTableName(schema, file))) + .forEach(table -> { + tables.add(new PravegaTableHandle(table.getSchemaName().get(), + table.getTableName(), + table.getObjectName(), + table.getObjectType(), + table.getObjectArgs())); + }); + + return tables; + } + + @Override + public List getSchema(SchemaTableName schemaTableName) { + PravegaStreamDescription streamDescription = getLocalTable(schemaTableName); + return streamDescription == null ? null : streamDescription.getEvent().orElse(null); + } + + @Override + public PravegaStreamDescription getTable(SchemaTableName schemaTableName) + { + // reads table definition from local file + // if table definition has pointers to schema, read it and populate fields + // (for e.g. local schema file or url to schema) + PravegaStreamDescription table = getLocalTable(schemaTableName); + + // either not found or no fields, nothing to do. will be resolved later + if (table == null || !table.getEvent().isPresent()) { + return table; + } + + // fields already defined + if (table.getEvent().get().stream().noneMatch( + schema -> schema.getDataSchema().isPresent())) { + return table; + } + + // at least 1 schema for a fieldGroup must be resolved. read schema from local file or url + List finalSchemas = + resolveAllSchemas(table.getEvent().get(), (i) -> columnPrefix(table, i)); + + return new PravegaStreamDescription(table, finalSchemas); + } + + static String columnPrefix(PravegaStreamDescription table, int schemaIndex) { + // if kv table, returns something like key/{fieldName}, value/{fieldName} + return table.getObjectType() == ObjectType.KV_TABLE ? kvFieldMapping(schemaIndex) : ""; + } + + private java.util.stream.Stream localSchemaStream() + { + return listFiles(localTableDir).stream() + .filter(file -> file.isFile() && file.getName().endsWith(".json")); + } + + private static List listFiles(File dir) + { + if ((dir != null) && dir.isDirectory()) { + File[] files = dir.listFiles(); + if (files != null) { + return ImmutableList.copyOf(files); + } + } + return ImmutableList.of(); + } + + private PravegaStreamDescription getLocalTable(SchemaTableName schemaTableName) + { + try { + File file = new File(localTableDir, String.format("%s.%s.json", + schemaTableName.getSchemaName(), schemaTableName.getTableName())); + return !file.exists() ? + null + : streamDescriptionCodec.fromJson(readAllBytes(file.toPath())); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } +} diff --git a/src/main/java/io/pravega/connectors/presto/schemamangement/PravegaSchemaRegistry.java b/src/main/java/io/pravega/connectors/presto/schemamangement/PravegaSchemaRegistry.java new file mode 100644 index 0000000..8fc67e6 --- /dev/null +++ b/src/main/java/io/pravega/connectors/presto/schemamangement/PravegaSchemaRegistry.java @@ -0,0 +1,196 @@ +/* + * Copyright (c) Pravega Authors. + * + * 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.pravega.connectors.presto.schemamangement; + +import com.facebook.presto.spi.SchemaTableName; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Streams; +import io.pravega.client.ClientConfig; +import io.pravega.client.admin.StreamManager; +import io.pravega.connectors.presto.ObjectType; +import io.pravega.connectors.presto.PravegaStreamDescription; +import io.pravega.connectors.presto.PravegaStreamFieldGroup; +import io.pravega.connectors.presto.PravegaTableHandle; +import io.pravega.schemaregistry.client.SchemaRegistryClient; +import io.pravega.schemaregistry.client.SchemaRegistryClientConfig; +import io.pravega.schemaregistry.client.SchemaRegistryClientFactory; +import io.pravega.schemaregistry.client.exceptions.RegistryExceptions; +import io.pravega.schemaregistry.contract.data.GroupProperties; +import io.pravega.schemaregistry.contract.data.SchemaWithVersion; +import io.pravega.schemaregistry.contract.data.SerializationFormat; + +import java.net.URI; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.Locale; +import java.util.Optional; + +import static io.pravega.connectors.presto.ProtobufCommon.encodeSchema; +import static io.pravega.connectors.presto.util.PravegaNameUtils.groupId; +import static io.pravega.connectors.presto.util.PravegaNameUtils.internalObject; +import static io.pravega.connectors.presto.util.PravegaNameUtils.internalStream; +import static io.pravega.connectors.presto.util.PravegaNameUtils.kvFieldMapping; +import static io.pravega.connectors.presto.util.PravegaNameUtils.temp_streamNameToTableName; +import static io.pravega.connectors.presto.util.PravegaNameUtils.temp_tableNameToStreamName; +import static io.pravega.connectors.presto.util.PravegaSchemaUtils.GROUP_PROPERTIES_INLINE_KEY; +import static io.pravega.connectors.presto.util.PravegaSchemaUtils.GROUP_PROPERTIES_INLINE_KV_KEY; +import static io.pravega.connectors.presto.util.PravegaSchemaUtils.GROUP_PROPERTIES_INLINE_KV_VALUE; +import static io.pravega.connectors.presto.util.PravegaSchemaUtils.INLINE_SUFFIX; +import static io.pravega.connectors.presto.util.PravegaStreamDescUtils.mapFieldsFromSchema; + +public class PravegaSchemaRegistry + implements SchemaSupplier, SchemaRegistry +{ + private final SchemaRegistryClient registryClient; + + private final StreamManager streamManager; + + public PravegaSchemaRegistry(URI controllerURI, URI schemaRegistryURI) + { + SchemaRegistryClientConfig registryConfig = + SchemaRegistryClientConfig.builder().schemaRegistryUri(schemaRegistryURI).build(); + this.registryClient = SchemaRegistryClientFactory.withDefaultNamespace(registryConfig); + this.streamManager = StreamManager.create(ClientConfig.builder().controllerURI(controllerURI).build()); + } + + @Override + public List listSchemas() + { + List schemas = new ArrayList<>(); + Streams.stream(streamManager.listScopes()).filter(s -> !internalObject(s)).forEach(schemas::add); + return schemas; + } + + @Override + public List listTables(String schema) + { + // (underlying streams used by kv table are seen as internal and thus are skipped) + List tables = new ArrayList<>(); + Streams.stream(streamManager.listStreams(schema)) + .filter(stream -> !internalStream(stream)) + .forEach(stream -> { + tables.add(new PravegaTableHandle(schema, + temp_streamNameToTableName(stream.getStreamName()), + stream.getStreamName(), + ObjectType.STREAM, + Optional.empty())); + }); + return tables; + } + + @Override + public List getSchema(SchemaTableName schemaTableName) { + String groupName = groupId(schemaTableName.getSchemaName(), + temp_tableNameToStreamName(schemaTableName.getTableName())); + + GroupProperties properties; + List schemas; + + try { + properties = registryClient.getGroupProperties(groupName); + schemas = registryClient.getSchemas(groupName); + } + catch (RegistryExceptions.ResourceNotFoundException e) { + return null; + } + + if (schemas.size() == 0 || schemas.size() > 2) { + throw new IllegalStateException(schemaTableName + " has " + schemas.size() + " registered schemas. expecting either 1 or 2"); + } + + // kv table will have > 1 schema. key+value likely different types + boolean kv = schemas.size() > 1; + List fieldGroups = new ArrayList<>(2); + for (int i = 0; i < schemas.size(); i++) { + // colPrefix used for display so can differentiate between fields from key or value + String colPrefix = kv ? kvFieldMapping(i) : ""; + + SerializationFormat format = schemas.get(i).getSchemaInfo().getSerializationFormat(); + fieldGroups.add(new PravegaStreamFieldGroup( + dataFormat(properties.getProperties(), format, kv, i), + Optional.of(colPrefix), + dataSchema(format, schemas.get(i)), + Optional.of(mapFieldsFromSchema(colPrefix, format, schemas.get(i))))); + } + + return fieldGroups; + } + + @Override + public PravegaStreamDescription getTable(SchemaTableName schemaTableName) + { + List schema = getSchema(schemaTableName); + if (schema == null) { + return null; + } + + return new PravegaStreamDescription( + schemaTableName.getTableName(), + Optional.of(schemaTableName.getSchemaName()), + temp_tableNameToStreamName(schemaTableName.getTableName()), + Optional.of(ObjectType.STREAM), + Optional.empty() /* args */, + Optional.of(schema)); + } + + private static String dataFormat(ImmutableMap groupProperties, + SerializationFormat format, + boolean kvTable, + int kvIdx) + { + /* + TODO: auto-detect https://github.com/pravega/presto-connector/issues/20 + + (1) no schema registry. + (2) Register and evolve schemas in registry but do not use registry client while writing data + (3) Register schemas in the registry and use registry client to encode schema Id with payload + "inline" is for #3. for e.g. "avro" -> "avro-inline". PravegaRecordSetProvider is interested in this + + hopefully this can all go away (see linked issue 58 above) + + but for now the following is our convention + if "inline" exists in our properties, all data uses SR + else if it is a kv table key+value may be different. both, neither, or either may use SR + look for "inlinekey" / "inlinevalue" + */ + + String key = GROUP_PROPERTIES_INLINE_KEY; + + if (kvTable && !groupProperties.containsKey(key)) { + key = kvIdx == 0 ? GROUP_PROPERTIES_INLINE_KV_KEY : GROUP_PROPERTIES_INLINE_KV_VALUE; + } + + String finalFormat = format == SerializationFormat.Custom + ? format.getFullTypeName().toLowerCase(Locale.ENGLISH) + : format.name().toLowerCase(Locale.ENGLISH); + return finalFormat + (groupProperties.containsKey(key) ? INLINE_SUFFIX : ""); + } + + public static Optional dataSchema(SerializationFormat format, SchemaWithVersion schemaWithVersion) + { + // it is intentional that nothing is returned for Custom + // pass schema to row decoders. refer to PravegaRecordSetProvider + switch (format) { + case Protobuf: + return Optional.of(encodeSchema(schemaWithVersion)); + case Avro: + return Optional.of(new String(schemaWithVersion.getSchemaInfo().getSchemaData().array(), StandardCharsets.UTF_8)); + default: + return Optional.empty(); + } + } +} diff --git a/src/main/java/io/pravega/connectors/presto/schemamangement/SchemaRegistry.java b/src/main/java/io/pravega/connectors/presto/schemamangement/SchemaRegistry.java new file mode 100644 index 0000000..36be87f --- /dev/null +++ b/src/main/java/io/pravega/connectors/presto/schemamangement/SchemaRegistry.java @@ -0,0 +1,29 @@ +/* + * Copyright (c) Pravega Authors. + * + * 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.pravega.connectors.presto.schemamangement; + +import com.facebook.presto.spi.SchemaTableName; +import io.pravega.connectors.presto.PravegaStreamDescription; +import io.pravega.connectors.presto.PravegaStreamFieldGroup; + +import java.util.List; + +public interface SchemaRegistry +{ + PravegaStreamDescription getTable(SchemaTableName schemaTableName); + + List getSchema(SchemaTableName schemaTableName); +} diff --git a/src/main/java/io/pravega/connectors/presto/schemamangement/SchemaSupplier.java b/src/main/java/io/pravega/connectors/presto/schemamangement/SchemaSupplier.java new file mode 100644 index 0000000..21746c3 --- /dev/null +++ b/src/main/java/io/pravega/connectors/presto/schemamangement/SchemaSupplier.java @@ -0,0 +1,27 @@ +/* + * Copyright (c) Pravega Authors. + * + * 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.pravega.connectors.presto.schemamangement; + +import io.pravega.connectors.presto.PravegaTableHandle; + +import java.util.List; + +public interface SchemaSupplier +{ + List listSchemas(); + + List listTables(String schema); +} diff --git a/src/main/java/io/pravega/connectors/presto/util/PravegaNameUtils.java b/src/main/java/io/pravega/connectors/presto/util/PravegaNameUtils.java index c4ba35b..b19f03c 100644 --- a/src/main/java/io/pravega/connectors/presto/util/PravegaNameUtils.java +++ b/src/main/java/io/pravega/connectors/presto/util/PravegaNameUtils.java @@ -16,6 +16,7 @@ package io.pravega.connectors.presto.util; +import io.pravega.client.stream.Stream; import io.pravega.connectors.presto.ObjectType; import io.pravega.connectors.presto.PravegaStreamDescription; import io.pravega.connectors.presto.PravegaTableHandle; @@ -64,7 +65,7 @@ public static boolean multiSourceStream(PravegaTableHandle object) multiSourceStream(object.getObjectName()); } - private static boolean multiSourceStream(String stream) + public static boolean multiSourceStream(String stream) { try { // test pattern for stream names pravega will allow @@ -108,4 +109,15 @@ public static String streamCutName(String stream) { return stream + STREAM_CUT_PREFIX; } + + public static boolean internalStream(Stream stream) + { + return internalObject(stream.getStreamName()); + } + + public static boolean internalObject(String object) + { + return object.startsWith("_") /* pravega internal */ || + object.endsWith("-SC") /* application internal - stream cuts */; + } } diff --git a/src/main/java/io/pravega/connectors/presto/util/PravegaStreamDescUtils.java b/src/main/java/io/pravega/connectors/presto/util/PravegaStreamDescUtils.java index adfed8c..38f4c35 100644 --- a/src/main/java/io/pravega/connectors/presto/util/PravegaStreamDescUtils.java +++ b/src/main/java/io/pravega/connectors/presto/util/PravegaStreamDescUtils.java @@ -18,6 +18,7 @@ import com.facebook.presto.common.type.Type; import com.google.protobuf.Descriptors; import io.pravega.connectors.presto.PravegaStreamFieldDescription; +import io.pravega.connectors.presto.PravegaStreamFieldGroup; import io.pravega.connectors.presto.ProtobufCommon; import io.pravega.schemaregistry.contract.data.SchemaWithVersion; import io.pravega.schemaregistry.contract.data.SerializationFormat; @@ -32,6 +33,7 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; import static com.facebook.presto.common.type.BigintType.BIGINT; import static com.facebook.presto.common.type.BooleanType.BOOLEAN; @@ -40,6 +42,7 @@ import static com.facebook.presto.common.type.VarcharType.createUnboundedVarcharType; import static io.pravega.connectors.presto.util.PravegaSchemaUtils.AVRO; import static io.pravega.connectors.presto.util.PravegaSchemaUtils.NESTED_RECORD_SEPARATOR; +import static io.pravega.connectors.presto.util.PravegaSchemaUtils.readSchema; import static org.apache.avro.Schema.Type.RECORD; /** @@ -343,4 +346,37 @@ private static String nestedPrefixFor(String prefix, String name) ? name : prefix + NESTED_RECORD_SEPARATOR + name; } + + /** + * PravegaStreamFieldGroup may contain pointer to schema (local file, or url) + * for these, read the schema and build the field definitions + * + * @param fieldGroups fieldGroups to look through + * @param columnPrefix function to return columnPrefix to be used for the fields in the group + * @return list of PravegaStreamFieldGroup with all schemas resolved + */ + public static List resolveAllSchemas(List fieldGroups, + Function columnPrefix) + { + // fields already defined + if (fieldGroups.stream().noneMatch( + schema -> schema.getDataSchema().isPresent())) { + return fieldGroups; + } + + // at least 1 schema for a fieldGroup must be resolved. read schema from local file or url + List finalSchemas = new ArrayList<>(fieldGroups.size()); + for (int i = 0; i < fieldGroups.size(); i++) { + PravegaStreamFieldGroup fieldGroup = fieldGroups.get(i); + if (fieldGroup.getDataSchema().isPresent()) { + String dataSchema = readSchema(fieldGroup.getDataSchema().get()); + List fields = + mapFieldsFromSchema(columnPrefix.apply(i), fieldGroup.getDataFormat(), dataSchema); + finalSchemas.add(new PravegaStreamFieldGroup(fieldGroup, dataSchema, fields)); + } else { + finalSchemas.add(fieldGroup); + } + } + return finalSchemas; + } } diff --git a/src/test/java/io/pravega/connectors/presto/integration/PravegaQueryRunner.java b/src/test/java/io/pravega/connectors/presto/integration/PravegaQueryRunner.java index 51ce818..83c63e7 100644 --- a/src/test/java/io/pravega/connectors/presto/integration/PravegaQueryRunner.java +++ b/src/test/java/io/pravega/connectors/presto/integration/PravegaQueryRunner.java @@ -144,7 +144,7 @@ private static PravegaTableDescriptionSupplier createSchemas(Metadata metadata, } // all schemas + tables will be served from these provided caches - return new PravegaTableDescriptionSupplier(null, schemaCache, tableCache); + return new PravegaTableDescriptionSupplier(schemaCache, tableCache); } public static Session createSession() From 61d0624ec98fb9479beb869d260b485f0acd583d Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Wed, 24 Mar 2021 06:55:53 -0700 Subject: [PATCH 09/32] working example with confluent Signed-off-by: Andrew Robertson --- .../PravegaTableDescriptionSupplier.java | 6 +-- .../ConfluentSchemaRegistry.java | 42 +++++++++++++++---- 2 files changed, 38 insertions(+), 10 deletions(-) diff --git a/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java b/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java index cfec988..2df1477 100644 --- a/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java +++ b/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java @@ -103,7 +103,7 @@ public List listSchemas() }); } else { - log.info("serving listSchemas() from cache"); + log.debug("serving listSchemas() from cache"); } return schemas; } @@ -143,7 +143,7 @@ public List listTables(Optional schema) }); } else { - log.info("serving listTables(%s) from cache", s); + log.debug("serving listTables(%s) from cache", s); } tableList.addAll(tableCache.asMap().keySet().stream() @@ -159,7 +159,7 @@ public PravegaStreamDescription getTable(SchemaTableName schemaTableName) PravegaTableName pravegaTableName = new PravegaTableName(schemaTableName); Optional cachedTable = tableCache.getIfPresent(pravegaTableName); if (cachedTable != null && cachedTable.isPresent()) { - log.info("serving getTable(%s) from cache", schemaTableName); + log.debug("serving getTable(%s) from cache", schemaTableName); return cachedTable.get(); } diff --git a/src/main/java/io/pravega/connectors/presto/schemamangement/ConfluentSchemaRegistry.java b/src/main/java/io/pravega/connectors/presto/schemamangement/ConfluentSchemaRegistry.java index 6b53a0d..ab5a607 100644 --- a/src/main/java/io/pravega/connectors/presto/schemamangement/ConfluentSchemaRegistry.java +++ b/src/main/java/io/pravega/connectors/presto/schemamangement/ConfluentSchemaRegistry.java @@ -16,28 +16,44 @@ package io.pravega.connectors.presto.schemamangement; import com.facebook.presto.spi.SchemaTableName; +import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; +import io.confluent.kafka.schemaregistry.client.SchemaMetadata; +import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; +import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; +import io.pravega.connectors.presto.ObjectType; import io.pravega.connectors.presto.PravegaStreamDescription; +import io.pravega.connectors.presto.PravegaStreamFieldDescription; import io.pravega.connectors.presto.PravegaStreamFieldGroup; +import java.io.IOException; +import java.io.UncheckedIOException; import java.net.URI; +import java.util.Collections; import java.util.List; +import java.util.Optional; + +import static io.pravega.connectors.presto.util.PravegaNameUtils.temp_tableNameToStreamName; +import static io.pravega.connectors.presto.util.PravegaSchemaUtils.AVRO; +import static io.pravega.connectors.presto.util.PravegaStreamDescUtils.mapFieldsFromSchema; public class ConfluentSchemaRegistry implements SchemaRegistry { - //private final SchemaRegistryClient schemaRegistryClient; + private final SchemaRegistryClient schemaRegistryClient; public ConfluentSchemaRegistry(URI registryURI) { - //this.schemaRegistryClient = new CachedSchemaRegistryClient(registryURI.toASCIIString(), Integer.MAX_VALUE); + this.schemaRegistryClient = new CachedSchemaRegistryClient(registryURI.toASCIIString(), Integer.MAX_VALUE); } @Override public List getSchema(SchemaTableName schemaTableName) { - /* try { SchemaMetadata metadata = schemaRegistryClient.getLatestSchemaMetadata(format(schemaTableName)); + if (!metadata.getSchemaType().equalsIgnoreCase(AVRO)) { + throw new UnsupportedOperationException("schema type '" + metadata.getSchemaType() + "' is not supported"); + } List fields = mapFieldsFromSchema("", AVRO, metadata.getSchema()); @@ -51,17 +67,29 @@ public List getSchema(SchemaTableName schemaTableName) catch (IOException e) { throw new UncheckedIOException(e); } - catch (Exception e) { + catch (RestClientException e) { + if (e.getStatus() == 404) { + return null; + } throw new RuntimeException(e); } - */ - return null; } @Override public PravegaStreamDescription getTable(SchemaTableName schemaTableName) { - return null; + List schema = getSchema(schemaTableName); + if (schema == null) { + return null; + } + + return new PravegaStreamDescription( + schemaTableName.getTableName(), + Optional.of(schemaTableName.getSchemaName()), + temp_tableNameToStreamName(schemaTableName.getTableName()), + Optional.of(ObjectType.STREAM), + Optional.empty() /* args */, + Optional.of(schema)); } static String format(SchemaTableName schemaTableName) From 7fc93b47e56fdce06bf496e654e595c17cb535a5 Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Wed, 24 Mar 2021 07:55:45 -0700 Subject: [PATCH 10/32] allow list of component streams in multi source stream definition Signed-off-by: Andrew Robertson --- .../connectors/presto/PravegaSplitManager.java | 8 ++++---- .../presto/PravegaTableDescriptionSupplier.java | 16 ++++++++++++++-- .../connectors/presto/PravegaTableHandle.java | 7 ++++++- .../schemamangement/LocalSchemaRegistry.java | 4 ++++ .../connectors/presto/util/PravegaNameUtils.java | 10 +++++++--- 5 files changed, 35 insertions(+), 10 deletions(-) diff --git a/src/main/java/io/pravega/connectors/presto/PravegaSplitManager.java b/src/main/java/io/pravega/connectors/presto/PravegaSplitManager.java index 4f4a7d0..81f4630 100644 --- a/src/main/java/io/pravega/connectors/presto/PravegaSplitManager.java +++ b/src/main/java/io/pravega/connectors/presto/PravegaSplitManager.java @@ -116,10 +116,10 @@ private static ReaderType readerType(PravegaProperties properties) private void buildKVSplits(PravegaTableHandle pravegaTableHandle, ImmutableList.Builder splits) { - pravegaTableHandle.getOjectArgs().orElseThrow(() -> + pravegaTableHandle.getObjectArgs().orElseThrow(() -> new IllegalArgumentException("no KF defined for " + pravegaTableHandle)); - for (String kf : pravegaTableHandle.getOjectArgs().get()) { + for (String kf : pravegaTableHandle.getObjectArgs().get()) { PravegaSplit split = new PravegaSplit(connectorId, ObjectType.KV_TABLE, @@ -130,7 +130,7 @@ private void buildKVSplits(PravegaTableHandle pravegaTableHandle, ImmutableList. splits.add(split); } - log.info("created " + pravegaTableHandle.getOjectArgs().get().size() + " kv splits"); + log.info("created " + pravegaTableHandle.getObjectArgs().get().size() + " kv splits"); } private void buildStreamSplits(final PravegaProperties properties, @@ -139,7 +139,7 @@ private void buildStreamSplits(final PravegaProperties properties, { // TODO: Enable begin and end cuts to be configurable: https://github.com/pravega/pravega-sql/issues/24 List sourceStreams = multiSourceStream(pravegaTableHandle) - ? pravegaTableHandle.getOjectArgs().orElseThrow( + ? pravegaTableHandle.getObjectArgs().orElseThrow( () -> new IllegalArgumentException("no args for multi source table found")) : Collections.singletonList(pravegaTableHandle.getObjectName()); diff --git a/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java b/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java index 2df1477..ebf5cda 100644 --- a/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java +++ b/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java @@ -138,7 +138,16 @@ public List listTables(Optional schema) } if (multiSourceStream(table)) { - compositeStreams.add(Pattern.compile(table.getObjectName())); + // if component streams specified look for exact match when hiding + if (table.getObjectArgs().isPresent()) { + table.getObjectArgs().get().forEach(arg -> { + compositeStreams.add(Pattern.compile("^" + arg + "$")); + }); + } + else { + // regex, fuzzy match + compositeStreams.add(Pattern.compile(table.getObjectName())); + } } }); } @@ -166,7 +175,10 @@ public PravegaStreamDescription getTable(SchemaTableName schemaTableName) PravegaStreamDescription table = schemaRegistry.getTable(schemaTableName); if (multiSourceStream(table)) { - List objectArgs = multiSourceStreamComponents(schemaTableName, table.getObjectName()); + // if component streams not already specified, look them up from pravega based on regex + List objectArgs = table.getObjectArgs().isPresent() + ? table.getObjectArgs().get() + : multiSourceStreamComponents(schemaTableName, table.getObjectName()); if (objectArgs.isEmpty()) { throw new IllegalArgumentException("could not get component streams for " + schemaTableName); } diff --git a/src/main/java/io/pravega/connectors/presto/PravegaTableHandle.java b/src/main/java/io/pravega/connectors/presto/PravegaTableHandle.java index 78bb238..c7456c6 100644 --- a/src/main/java/io/pravega/connectors/presto/PravegaTableHandle.java +++ b/src/main/java/io/pravega/connectors/presto/PravegaTableHandle.java @@ -51,6 +51,11 @@ public final class PravegaTableHandle */ private final String objectName; + /** + * optional + * for ObjectType.STREAM, this is list of composite streams in a multi source stream + * for ObjectType.KV_TABLE this is list of key families + */ private final Optional> objectArgs; private final List schema; @@ -117,7 +122,7 @@ public String getObjectName() } @JsonProperty - public Optional> getOjectArgs() + public Optional> getObjectArgs() { return objectArgs; } diff --git a/src/main/java/io/pravega/connectors/presto/schemamangement/LocalSchemaRegistry.java b/src/main/java/io/pravega/connectors/presto/schemamangement/LocalSchemaRegistry.java index a64eeb9..414d4f2 100644 --- a/src/main/java/io/pravega/connectors/presto/schemamangement/LocalSchemaRegistry.java +++ b/src/main/java/io/pravega/connectors/presto/schemamangement/LocalSchemaRegistry.java @@ -50,6 +50,7 @@ public LocalSchemaRegistry(File localTableDir, JsonCodec listSchemas() { + // file name format: {schema}.{table}.json return localSchemaStream() .map(file -> file.getName().split("\\.")[0]) .collect(Collectors.toList()); @@ -60,9 +61,12 @@ public List listTables(String schema) { final List tables = new ArrayList<>(); + // file name format: {schema}.{table}.json localSchemaStream() .filter(file -> file.getName().startsWith(schema)) + // ensures matches full file name format .filter(file -> file.getName().split("\\.").length == 3) + // {table} from file name .map(file -> file.getName().split("\\.")[1]) .map(file -> getLocalTable(new SchemaTableName(schema, file))) .forEach(table -> { diff --git a/src/main/java/io/pravega/connectors/presto/util/PravegaNameUtils.java b/src/main/java/io/pravega/connectors/presto/util/PravegaNameUtils.java index b19f03c..2ea7a5c 100644 --- a/src/main/java/io/pravega/connectors/presto/util/PravegaNameUtils.java +++ b/src/main/java/io/pravega/connectors/presto/util/PravegaNameUtils.java @@ -52,19 +52,23 @@ public static String groupId(String scope, String stream) return scope + "." + stream; } - // test stream name - if not valid pravega stream name assume it is regex for multi source public static boolean multiSourceStream(PravegaStreamDescription object) { + // if stream name is a regex, or if there are object args + // (objectArgs for stream are comma sep list of component streams) return object.getObjectType() == ObjectType.STREAM && - multiSourceStream(object.getObjectName()); + (multiSourceStream(object.getObjectName()) || object.getObjectArgs().isPresent()); } public static boolean multiSourceStream(PravegaTableHandle object) { + // if stream name is a regex, or if there are object args + // (objectArgs for stream are comma sep list of component streams) return object.getObjectType() == ObjectType.STREAM && - multiSourceStream(object.getObjectName()); + (multiSourceStream(object.getObjectName()) || object.getObjectArgs().isPresent()); } + // test stream name - if not valid pravega stream name assume it is regex for multi source public static boolean multiSourceStream(String stream) { try { From c8d8267775aac11502b9dbb1d7b00f2bf65288ba Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Wed, 24 Mar 2021 08:00:59 -0700 Subject: [PATCH 11/32] add confluent lib, remove stale/test aws key config Signed-off-by: Andrew Robertson --- build.gradle | 5 +- gradle.properties | 1 + .../presto/PravegaConnectorConfig.java | 50 ------------------- 3 files changed, 5 insertions(+), 51 deletions(-) diff --git a/build.gradle b/build.gradle index e507721..f2f427c 100644 --- a/build.gradle +++ b/build.gradle @@ -54,6 +54,8 @@ dependencies { compile "com.facebook.presto:presto-spi:${prestoVersion}" compile "com.facebook.presto:presto-common:${prestoVersion}" + compile "io.confluent:kafka-schema-registry-client:${confluentVersion}" + runtimeOnly "io.airlift:joda-to-java-time-bridge:3" runtimeOnly "com.facebook.airlift:log-manager:${airliftLogManagerVersion}" runtimeOnly "org.apache.zookeeper:zookeeper:${zookeeperVersion}" @@ -107,4 +109,5 @@ test { suites "src/test/integration-test.xml" } } -} \ No newline at end of file +} + diff --git a/gradle.properties b/gradle.properties index 0b5d456..6aba223 100644 --- a/gradle.properties +++ b/gradle.properties @@ -18,6 +18,7 @@ airliftTestingVersion=0.191 airliftUnitsVersion=1.3 avroVersion=1.8.1 commonsVersion=3.7 +confluentVersion=6.1.0 checkstyleToolVersion=8.23 everitJsonSchemaVersion=1.12.1 guavaVersion=26.0-jre diff --git a/src/main/java/io/pravega/connectors/presto/PravegaConnectorConfig.java b/src/main/java/io/pravega/connectors/presto/PravegaConnectorConfig.java index 941a47b..8ce0a97 100644 --- a/src/main/java/io/pravega/connectors/presto/PravegaConnectorConfig.java +++ b/src/main/java/io/pravega/connectors/presto/PravegaConnectorConfig.java @@ -17,7 +17,6 @@ package io.pravega.connectors.presto; import com.facebook.airlift.configuration.Config; -import com.facebook.airlift.configuration.ConfigSecuritySensitive; import javax.validation.constraints.NotNull; @@ -51,14 +50,6 @@ public class PravegaConnectorConfig { private File tableDescriptionDir = new File("etc/pravega/"); - private String awsAccessKey; - - private String awsSecretKey; - - private String awsRegion = "us-east-1"; - - private String awsGlueSchemaRegistry; - private URI confluentSchemaRegistry; @NotNull @@ -108,47 +99,6 @@ public PravegaConnectorConfig setHideInternalColumns(boolean hideInternalColumns return this; } - @Config("pravega.aws.accessKey") - @ConfigSecuritySensitive - public PravegaConnectorConfig setAwsAccessKey(String awsAccessKey) { - this.awsAccessKey = awsAccessKey; - return this; - } - - public String getAwsAccessKey() { - return awsAccessKey; - } - - @Config("pravega.aws.secretKey") - @ConfigSecuritySensitive - public PravegaConnectorConfig setAwsSecretKey(String awsSecretKey) { - this.awsSecretKey = awsSecretKey; - return this; - } - - public String getAwsSecretKey() { - return awsSecretKey; - } - - @Config("pravega.aws.region") - public PravegaConnectorConfig setAwsRegion(String awsRegion) { - this.awsRegion = awsRegion; - return this; - } - - public String getAwsRegion() { - return awsRegion; - } - - @Config("pravega.aws.glue.schema-registry") - public PravegaConnectorConfig setAwsGlueSchemaRegistry(String awsGlueSchemaRegistry) { - this.awsGlueSchemaRegistry = awsGlueSchemaRegistry; - return this; - } - - public String getAwsGlueSchemaRegistry() { - return awsGlueSchemaRegistry; - } @Config("pravega.confluentSchemaRegistry") public PravegaConnectorConfig setConfluentSchemaRegistry(URI confluentSchemaRegistry) { From a948bc1ac555146ab523a5ba261e455b0556b500 Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Wed, 24 Mar 2021 12:31:22 -0700 Subject: [PATCH 12/32] add package to unit-test Signed-off-by: Andrew Robertson --- src/test/unit-test.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/src/test/unit-test.xml b/src/test/unit-test.xml index 9f6b1d9..1737cd5 100644 --- a/src/test/unit-test.xml +++ b/src/test/unit-test.xml @@ -3,6 +3,7 @@ + From 4b553317a4417728b6daeec9828106c5eca36db8 Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Thu, 25 Mar 2021 06:35:18 -0700 Subject: [PATCH 13/32] add git rev to class header/borrowed code Signed-off-by: Andrew Robertson --- .../io/pravega/connectors/presto/integration/CodecSupplier.java | 1 + .../pravega/connectors/presto/integration/EmbeddedPravega.java | 1 + .../io/pravega/connectors/presto/integration/PravegaLoader.java | 1 + 3 files changed, 3 insertions(+) diff --git a/src/test/java/io/pravega/connectors/presto/integration/CodecSupplier.java b/src/test/java/io/pravega/connectors/presto/integration/CodecSupplier.java index 9d65e58..2334f1e 100644 --- a/src/test/java/io/pravega/connectors/presto/integration/CodecSupplier.java +++ b/src/test/java/io/pravega/connectors/presto/integration/CodecSupplier.java @@ -12,6 +12,7 @@ * limitations under the License. * * Note: This class file is from PrestoDb + * (rev a8968160e1840ac67a5f63def27d31c0ef0acde7) * https://github.com/prestodb/presto/blob/0.247/presto-kafka/src/test/java/com/facebook/presto/kafka/util/CodecSupplier.java */ package io.pravega.connectors.presto.integration; diff --git a/src/test/java/io/pravega/connectors/presto/integration/EmbeddedPravega.java b/src/test/java/io/pravega/connectors/presto/integration/EmbeddedPravega.java index 4161034..24db193 100644 --- a/src/test/java/io/pravega/connectors/presto/integration/EmbeddedPravega.java +++ b/src/test/java/io/pravega/connectors/presto/integration/EmbeddedPravega.java @@ -8,6 +8,7 @@ * http://www.apache.org/licenses/LICENSE-2.0 * * Note: this class is based on SetupUtils from pravega/flink-connectors + * (rev 9332ad67e520c03c7122de1d3b90c6cafbf97634) * https://github.com/pravega/flink-connectors/blob/v0.9.0/src/test/java/io/pravega/connectors/flink/utils/SetupUtils.java */ package io.pravega.connectors.presto.integration; diff --git a/src/test/java/io/pravega/connectors/presto/integration/PravegaLoader.java b/src/test/java/io/pravega/connectors/presto/integration/PravegaLoader.java index 36b5c17..8e591a6 100644 --- a/src/test/java/io/pravega/connectors/presto/integration/PravegaLoader.java +++ b/src/test/java/io/pravega/connectors/presto/integration/PravegaLoader.java @@ -14,6 +14,7 @@ * limitations under the License. * * Note: This file contains some code from PrestoDb (majority of the "ResultsSession" implementation) + * (rev a8968160e1840ac67a5f63def27d31c0ef0acde7) * https://github.com/prestodb/presto/blob/0.247/presto-kafka/src/test/java/com/facebook/presto/kafka/util/KafkaLoader.java */ package io.pravega.connectors.presto.integration; From 9341a092f5e8f40db959f60c682e3cc0f63db5b5 Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Thu, 25 Mar 2021 07:16:33 -0700 Subject: [PATCH 14/32] update README Signed-off-by: Andrew Robertson --- README.md | 21 ++++++++++++++++++--- 1 file changed, 18 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index f315b55..2aabf1b 100644 --- a/README.md +++ b/README.md @@ -12,7 +12,8 @@ See the [User Manual](https://prestodb.github.io/docs/current/) for Presto deplo To build and run the Pravega Presto connector, you must meet the following requirements: * Linux -* Java 8 Update 151 or higher (8u151+), 64-bit. Both Oracle JDK and OpenJDK are supported. +* To build: Java 11+ 64-bit. Both Oracle JDK and OpenJDK are supported (we build using Java 11 JDK but with Java 8 compatibility) +* To run: Java 8 Update 151 or higher (8u151+), 64-bit. Both Oracle JDK and OpenJDK are supported. * Gradle 6.5.1+ (for building) * Python 2.7+ (for running with the launcher script) * Pravega version 0.9.0 or higher @@ -46,7 +47,10 @@ Make a directory for the Presto configuration files [root@lrmk226 ~]# mkdir $PRESTO_HOME/etc -Now follow the directions to create the neccesary configuration files for configuring Presto found in the PrestoDB documentation. +Now follow the directions to create the necessary configuration files for configuring Presto found in the PrestoDB documentation. + +Note that if you are also running with Java 11, you may have to add the following to your etc/jvm.config +-Djdk.attach.allowAttachSelf=true ## Installing and Configuring Pravega Connector @@ -74,7 +78,7 @@ After building Presto for the first time, you can load the project into your IDE After opening the project in IntelliJ, double check that the Java SDK is properly configured for the project: * Open the File menu and select Project Structure -* In the SDKs section, ensure that a 1.8 JDK is selected (create one if none exist) +* In the SDKs section, ensure that a Java 11+ JDK is selected (create one if none exist) * In the Project section, ensure the Project language level is set to 8.0 as Presto makes use of several Java 8 language features Use the following options to create a run configuration that runs the Presto server using the Pravega Presto connector: @@ -85,8 +89,11 @@ Use the following options to create a run configuration that runs the Presto ser * Use classpath of module: 'pravega.main' * Add a 'Before Launch' task - Add a gradle task to run the 'jar' task for the 'presto-connector' Gradle project. +Please note that some versions of Intellij do not display VM Options by default. For this, enable them with 'Modify options' + Modify the pravega.properties file in etc/catalog as previously described to point to a running Pravega controller, and a running Schema Registry. + ## Schema Definitions Currently, you must manually create schema definitions using a JSON file. In future releases, the 'CREATE TABLE' Presto command will be available. The JSON configuration files are read at server startup, and should be located in etc/pravega directory. In the JSON schema example below, "customer" is a stream name in the tpch pravega scope. @@ -142,3 +149,11 @@ Currently, you must manually create schema definitions using a JSON file. In fut }] } +## Tests +The pravega presto connector has 2 types of tests +* unit tests + * all unit tests run during developer builds +* integration tests + * by default only run on CI server + * use '-Pintegration' flag to run: ./gradlew test -Pintegration + From 6d63a4fa48ab4af40fb2d3bb7be5de3615b2b3b6 Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Thu, 25 Mar 2021 11:41:53 -0700 Subject: [PATCH 15/32] bad class? Signed-off-by: Andrew Robertson --- .../CompositeSchemaRegistry.java | 102 ------------------ 1 file changed, 102 deletions(-) delete mode 100644 src/main/java/io/pravega/connectors/presto/schemamangement/CompositeSchemaRegistry.java diff --git a/src/main/java/io/pravega/connectors/presto/schemamangement/CompositeSchemaRegistry.java b/src/main/java/io/pravega/connectors/presto/schemamangement/CompositeSchemaRegistry.java deleted file mode 100644 index 33c84c3..0000000 --- a/src/main/java/io/pravega/connectors/presto/schemamangement/CompositeSchemaRegistry.java +++ /dev/null @@ -1,102 +0,0 @@ -/* - * Copyright (c) Pravega Authors. - * - * 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.pravega.connectors.presto.schemamangement; - -import com.facebook.airlift.json.JsonCodec; -import com.facebook.presto.spi.SchemaTableName; -import io.pravega.connectors.presto.PravegaConnectorConfig; -import io.pravega.connectors.presto.PravegaStreamDescription; -import io.pravega.connectors.presto.PravegaStreamFieldGroup; -import io.pravega.connectors.presto.PravegaTableHandle; - -import java.util.ArrayList; -import java.util.List; - -public class CompositeSchemaRegistry - implements SchemaSupplier, SchemaRegistry -{ - private final List schemaSuppliers; - - private final List schemaRegistries; - - public CompositeSchemaRegistry(PravegaConnectorConfig config, JsonCodec streamDescriptionCodec) - { - schemaSuppliers = new ArrayList<>(); - schemaRegistries = new ArrayList<>(); - - // local will override, always add first - if (config.getTableDescriptionDir() != null && - config.getTableDescriptionDir().exists() && - config.getTableDescriptionDir().isDirectory()) { - LocalSchemaRegistry schemaRegistry = - new LocalSchemaRegistry(config.getTableDescriptionDir(), streamDescriptionCodec); - schemaSuppliers.add(schemaRegistry); - schemaRegistries.add(schemaRegistry); - } - - if (config.getSchemaRegistryURI() != null) { - PravegaSchemaRegistry schemaRegistry = - new PravegaSchemaRegistry(config.getControllerURI(), config.getSchemaRegistryURI()); - schemaSuppliers.add(schemaRegistry); - schemaRegistries.add(schemaRegistry); - } - - if (config.getConfluentSchemaRegistry() != null) { - ConfluentSchemaRegistry schemaRegistry = - new ConfluentSchemaRegistry(config.getConfluentSchemaRegistry()); - schemaRegistries.add(schemaRegistry); - } - } - - @Override - public List listSchemas() - { - final List schemas = new ArrayList<>(); - schemaSuppliers.forEach(p -> schemas.addAll(p.listSchemas())); - return schemas; - } - - @Override - public List listTables(String schema) - { - final List tables = new ArrayList<>(); - schemaSuppliers.forEach(p -> tables.addAll(p.listTables(schema))); - return tables; - } - - @Override - public List getSchema(SchemaTableName schemaTableName) { - for (SchemaRegistry schemaRegistry : schemaRegistries) { - List schema = schemaRegistry.getSchema(schemaTableName); - if (schema != null) { - return schema; - } - } - return null; - } - - @Override - public PravegaStreamDescription getTable(SchemaTableName schemaTableName) - { - for (SchemaRegistry schemaRegistry : schemaRegistries) { - PravegaStreamDescription streamDescription = schemaRegistry.getTable(schemaTableName); - if (streamDescription != null) { - return streamDescription; - } - } - return null; - } -} From 3ec71e343b67d60a0315cf150b97c615ba3f643e Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Thu, 25 Mar 2021 11:44:25 -0700 Subject: [PATCH 16/32] re-add same file Signed-off-by: Andrew Robertson --- .../PravegaTableDescriptionSupplier.java | 1 + .../CompositeSchemaRegistry.java | 87 +++++++++++++++++++ 2 files changed, 88 insertions(+) create mode 100644 src/main/java/io/pravega/connectors/presto/schemamangement/CompositeSchemaRegistry.java diff --git a/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java b/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java index ebf5cda..e67d732 100644 --- a/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java +++ b/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java @@ -21,6 +21,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.cache.Cache; import com.google.common.cache.CacheBuilder; + import io.pravega.connectors.presto.schemamangement.CompositeSchemaRegistry; import javax.inject.Inject; diff --git a/src/main/java/io/pravega/connectors/presto/schemamangement/CompositeSchemaRegistry.java b/src/main/java/io/pravega/connectors/presto/schemamangement/CompositeSchemaRegistry.java new file mode 100644 index 0000000..8f5dd7f --- /dev/null +++ b/src/main/java/io/pravega/connectors/presto/schemamangement/CompositeSchemaRegistry.java @@ -0,0 +1,87 @@ +package io.pravega.connectors.presto.schemamangement; + +import com.facebook.airlift.json.JsonCodec; +import com.facebook.presto.spi.SchemaTableName; +import io.pravega.connectors.presto.PravegaConnectorConfig; +import io.pravega.connectors.presto.PravegaStreamDescription; +import io.pravega.connectors.presto.PravegaStreamFieldGroup; +import io.pravega.connectors.presto.PravegaTableHandle; + +import java.util.ArrayList; +import java.util.List; + +public class CompositeSchemaRegistry + implements SchemaSupplier, SchemaRegistry +{ + private final List schemaSuppliers; + + private final List schemaRegistries; + + public CompositeSchemaRegistry(PravegaConnectorConfig config, JsonCodec streamDescriptionCodec) + { + schemaSuppliers = new ArrayList<>(); + schemaRegistries = new ArrayList<>(); + + // local will override, always add first + if (config.getTableDescriptionDir() != null && + config.getTableDescriptionDir().exists() && + config.getTableDescriptionDir().isDirectory()) { + LocalSchemaRegistry schemaRegistry = + new LocalSchemaRegistry(config.getTableDescriptionDir(), streamDescriptionCodec); + schemaSuppliers.add(schemaRegistry); + schemaRegistries.add(schemaRegistry); + } + + if (config.getSchemaRegistryURI() != null) { + PravegaSchemaRegistry schemaRegistry = + new PravegaSchemaRegistry(config.getControllerURI(), config.getSchemaRegistryURI()); + schemaSuppliers.add(schemaRegistry); + schemaRegistries.add(schemaRegistry); + } + + if (config.getConfluentSchemaRegistry() != null) { + ConfluentSchemaRegistry schemaRegistry = + new ConfluentSchemaRegistry(config.getConfluentSchemaRegistry()); + schemaRegistries.add(schemaRegistry); + } + } + + @Override + public List listSchemas() + { + final List schemas = new ArrayList<>(); + schemaSuppliers.forEach(p -> schemas.addAll(p.listSchemas())); + return schemas; + } + + @Override + public List listTables(String schema) + { + final List tables = new ArrayList<>(); + schemaSuppliers.forEach(p -> tables.addAll(p.listTables(schema))); + return tables; + } + + @Override + public List getSchema(SchemaTableName schemaTableName) { + for (SchemaRegistry schemaRegistry : schemaRegistries) { + List schema = schemaRegistry.getSchema(schemaTableName); + if (schema != null) { + return schema; + } + } + return null; + } + + @Override + public PravegaStreamDescription getTable(SchemaTableName schemaTableName) + { + for (SchemaRegistry schemaRegistry : schemaRegistries) { + PravegaStreamDescription streamDescription = schemaRegistry.getTable(schemaTableName); + if (streamDescription != null) { + return streamDescription; + } + } + return null; + } +} From 534fa5b704fb5eae23b779e5c2cfaf6a0f5e59af Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Thu, 25 Mar 2021 13:03:39 -0700 Subject: [PATCH 17/32] add unit test, rename some files Signed-off-by: Andrew Robertson --- .../PravegaTableDescriptionSupplier.java | 2 +- .../CompositeSchemaRegistry.java | 31 ++++- .../ConfluentSchemaRegistry.java | 2 +- .../LocalSchemaRegistry.java | 3 +- .../PravegaSchemaRegistry.java | 2 +- .../SchemaRegistry.java | 2 +- .../SchemaSupplier.java | 2 +- .../integration/PravegaKeyValueTest.java | 5 +- .../integration/PravegaQueryRunner.java | 8 +- .../CompositeSchemaRegistryTest.java | 63 ++++++++++ .../LocalSchemaRegistryTest.java | 110 ++++++++++++++++++ .../{integration => util}/CodecSupplier.java | 12 +- .../PravegaTestUtils.java | 30 +++-- .../resources/{kv => etc}/employee-id.avsc | 0 .../resources/{kv => etc}/employee-value.avsc | 0 .../resources/{kv => etc}/employee.records | 0 .../employee.json => etc/kv.employee.json} | 0 .../customer.json => etc/tpch.customer.json} | 0 .../lineitem.json => etc/tpch.lineitem.json} | 0 .../nation.json => etc/tpch.nation.json} | 0 .../orders.json => etc/tpch.orders.json} | 0 .../{tpch/part.json => etc/tpch.part.json} | 0 .../partsupp.json => etc/tpch.partsupp.json} | 0 .../region.json => etc/tpch.region.json} | 0 .../supplier.json => etc/tpch.supplier.json} | 0 src/test/unit-test.xml | 1 + 26 files changed, 243 insertions(+), 30 deletions(-) rename src/main/java/io/pravega/connectors/presto/{schemamangement => schemamanagement}/CompositeSchemaRegistry.java (73%) rename src/main/java/io/pravega/connectors/presto/{schemamangement => schemamanagement}/ConfluentSchemaRegistry.java (98%) rename src/main/java/io/pravega/connectors/presto/{schemamangement => schemamanagement}/LocalSchemaRegistry.java (98%) rename src/main/java/io/pravega/connectors/presto/{schemamangement => schemamanagement}/PravegaSchemaRegistry.java (99%) rename src/main/java/io/pravega/connectors/presto/{schemamangement => schemamanagement}/SchemaRegistry.java (94%) rename src/main/java/io/pravega/connectors/presto/{schemamangement => schemamanagement}/SchemaSupplier.java (93%) create mode 100644 src/test/java/io/pravega/connectors/presto/schemamanagement/CompositeSchemaRegistryTest.java create mode 100644 src/test/java/io/pravega/connectors/presto/schemamanagement/LocalSchemaRegistryTest.java rename src/test/java/io/pravega/connectors/presto/{integration => util}/CodecSupplier.java (87%) rename src/test/java/io/pravega/connectors/presto/{integration => util}/PravegaTestUtils.java (83%) rename src/test/resources/{kv => etc}/employee-id.avsc (100%) rename src/test/resources/{kv => etc}/employee-value.avsc (100%) rename src/test/resources/{kv => etc}/employee.records (100%) rename src/test/resources/{kv/employee.json => etc/kv.employee.json} (100%) rename src/test/resources/{tpch/customer.json => etc/tpch.customer.json} (100%) rename src/test/resources/{tpch/lineitem.json => etc/tpch.lineitem.json} (100%) rename src/test/resources/{tpch/nation.json => etc/tpch.nation.json} (100%) rename src/test/resources/{tpch/orders.json => etc/tpch.orders.json} (100%) rename src/test/resources/{tpch/part.json => etc/tpch.part.json} (100%) rename src/test/resources/{tpch/partsupp.json => etc/tpch.partsupp.json} (100%) rename src/test/resources/{tpch/region.json => etc/tpch.region.json} (100%) rename src/test/resources/{tpch/supplier.json => etc/tpch.supplier.json} (100%) diff --git a/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java b/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java index e67d732..7af70a6 100644 --- a/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java +++ b/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java @@ -22,7 +22,7 @@ import com.google.common.cache.Cache; import com.google.common.cache.CacheBuilder; -import io.pravega.connectors.presto.schemamangement.CompositeSchemaRegistry; +import io.pravega.connectors.presto.schemamanagement.CompositeSchemaRegistry; import javax.inject.Inject; diff --git a/src/main/java/io/pravega/connectors/presto/schemamangement/CompositeSchemaRegistry.java b/src/main/java/io/pravega/connectors/presto/schemamanagement/CompositeSchemaRegistry.java similarity index 73% rename from src/main/java/io/pravega/connectors/presto/schemamangement/CompositeSchemaRegistry.java rename to src/main/java/io/pravega/connectors/presto/schemamanagement/CompositeSchemaRegistry.java index 8f5dd7f..82322c4 100644 --- a/src/main/java/io/pravega/connectors/presto/schemamangement/CompositeSchemaRegistry.java +++ b/src/main/java/io/pravega/connectors/presto/schemamanagement/CompositeSchemaRegistry.java @@ -1,7 +1,23 @@ -package io.pravega.connectors.presto.schemamangement; +/* + * Copyright (c) Pravega Authors. + * + * 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.pravega.connectors.presto.schemamanagement; import com.facebook.airlift.json.JsonCodec; import com.facebook.presto.spi.SchemaTableName; +import com.google.common.annotations.VisibleForTesting; import io.pravega.connectors.presto.PravegaConnectorConfig; import io.pravega.connectors.presto.PravegaStreamDescription; import io.pravega.connectors.presto.PravegaStreamFieldGroup; @@ -11,14 +27,12 @@ import java.util.List; public class CompositeSchemaRegistry - implements SchemaSupplier, SchemaRegistry -{ + implements SchemaSupplier, SchemaRegistry { private final List schemaSuppliers; private final List schemaRegistries; - public CompositeSchemaRegistry(PravegaConnectorConfig config, JsonCodec streamDescriptionCodec) - { + public CompositeSchemaRegistry(PravegaConnectorConfig config, JsonCodec streamDescriptionCodec) { schemaSuppliers = new ArrayList<>(); schemaRegistries = new ArrayList<>(); @@ -46,6 +60,13 @@ public CompositeSchemaRegistry(PravegaConnectorConfig config, JsonCodec schemaSuppliers, List schemaRegistries) + { + this.schemaSuppliers = schemaSuppliers; + this.schemaRegistries = schemaRegistries; + } + @Override public List listSchemas() { diff --git a/src/main/java/io/pravega/connectors/presto/schemamangement/ConfluentSchemaRegistry.java b/src/main/java/io/pravega/connectors/presto/schemamanagement/ConfluentSchemaRegistry.java similarity index 98% rename from src/main/java/io/pravega/connectors/presto/schemamangement/ConfluentSchemaRegistry.java rename to src/main/java/io/pravega/connectors/presto/schemamanagement/ConfluentSchemaRegistry.java index ab5a607..c8b55bc 100644 --- a/src/main/java/io/pravega/connectors/presto/schemamangement/ConfluentSchemaRegistry.java +++ b/src/main/java/io/pravega/connectors/presto/schemamanagement/ConfluentSchemaRegistry.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.pravega.connectors.presto.schemamangement; +package io.pravega.connectors.presto.schemamanagement; import com.facebook.presto.spi.SchemaTableName; import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; diff --git a/src/main/java/io/pravega/connectors/presto/schemamangement/LocalSchemaRegistry.java b/src/main/java/io/pravega/connectors/presto/schemamanagement/LocalSchemaRegistry.java similarity index 98% rename from src/main/java/io/pravega/connectors/presto/schemamangement/LocalSchemaRegistry.java rename to src/main/java/io/pravega/connectors/presto/schemamanagement/LocalSchemaRegistry.java index 414d4f2..067938f 100644 --- a/src/main/java/io/pravega/connectors/presto/schemamangement/LocalSchemaRegistry.java +++ b/src/main/java/io/pravega/connectors/presto/schemamanagement/LocalSchemaRegistry.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.pravega.connectors.presto.schemamangement; +package io.pravega.connectors.presto.schemamanagement; import com.facebook.airlift.json.JsonCodec; import com.facebook.presto.spi.SchemaTableName; @@ -53,6 +53,7 @@ public List listSchemas() // file name format: {schema}.{table}.json return localSchemaStream() .map(file -> file.getName().split("\\.")[0]) + .distinct() .collect(Collectors.toList()); } diff --git a/src/main/java/io/pravega/connectors/presto/schemamangement/PravegaSchemaRegistry.java b/src/main/java/io/pravega/connectors/presto/schemamanagement/PravegaSchemaRegistry.java similarity index 99% rename from src/main/java/io/pravega/connectors/presto/schemamangement/PravegaSchemaRegistry.java rename to src/main/java/io/pravega/connectors/presto/schemamanagement/PravegaSchemaRegistry.java index 8fc67e6..555d362 100644 --- a/src/main/java/io/pravega/connectors/presto/schemamangement/PravegaSchemaRegistry.java +++ b/src/main/java/io/pravega/connectors/presto/schemamanagement/PravegaSchemaRegistry.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.pravega.connectors.presto.schemamangement; +package io.pravega.connectors.presto.schemamanagement; import com.facebook.presto.spi.SchemaTableName; import com.google.common.collect.ImmutableMap; diff --git a/src/main/java/io/pravega/connectors/presto/schemamangement/SchemaRegistry.java b/src/main/java/io/pravega/connectors/presto/schemamanagement/SchemaRegistry.java similarity index 94% rename from src/main/java/io/pravega/connectors/presto/schemamangement/SchemaRegistry.java rename to src/main/java/io/pravega/connectors/presto/schemamanagement/SchemaRegistry.java index 36be87f..451de5f 100644 --- a/src/main/java/io/pravega/connectors/presto/schemamangement/SchemaRegistry.java +++ b/src/main/java/io/pravega/connectors/presto/schemamanagement/SchemaRegistry.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.pravega.connectors.presto.schemamangement; +package io.pravega.connectors.presto.schemamanagement; import com.facebook.presto.spi.SchemaTableName; import io.pravega.connectors.presto.PravegaStreamDescription; diff --git a/src/main/java/io/pravega/connectors/presto/schemamangement/SchemaSupplier.java b/src/main/java/io/pravega/connectors/presto/schemamanagement/SchemaSupplier.java similarity index 93% rename from src/main/java/io/pravega/connectors/presto/schemamangement/SchemaSupplier.java rename to src/main/java/io/pravega/connectors/presto/schemamanagement/SchemaSupplier.java index 21746c3..20bc1b2 100644 --- a/src/main/java/io/pravega/connectors/presto/schemamangement/SchemaSupplier.java +++ b/src/main/java/io/pravega/connectors/presto/schemamanagement/SchemaSupplier.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.pravega.connectors.presto.schemamangement; +package io.pravega.connectors.presto.schemamanagement; import io.pravega.connectors.presto.PravegaTableHandle; diff --git a/src/test/java/io/pravega/connectors/presto/integration/PravegaKeyValueTest.java b/src/test/java/io/pravega/connectors/presto/integration/PravegaKeyValueTest.java index 1091887..30f2f8e 100644 --- a/src/test/java/io/pravega/connectors/presto/integration/PravegaKeyValueTest.java +++ b/src/test/java/io/pravega/connectors/presto/integration/PravegaKeyValueTest.java @@ -18,6 +18,7 @@ import com.facebook.presto.Session; import com.facebook.presto.testing.MaterializedResult; import com.facebook.presto.tests.DistributedQueryRunner; +import io.airlift.tpch.TpchTable; import io.pravega.connectors.presto.PravegaStreamDescription; import io.pravega.connectors.presto.PravegaStreamFieldDescription; import org.testng.annotations.AfterClass; @@ -26,7 +27,7 @@ import java.util.List; import static com.facebook.presto.testing.TestingSession.testSessionBuilder; -import static io.pravega.connectors.presto.integration.PravegaTestUtils.getKvStreamDesc; +import static io.pravega.connectors.presto.util.PravegaTestUtils.getKvStreamDesc; import static org.testng.Assert.assertEquals; @Test @@ -40,7 +41,7 @@ public PravegaKeyValueTest() throws Exception { this.pravega = new EmbeddedPravega(); - this.queryRunner = PravegaQueryRunner.createQueryRunner(pravega.getController(), java.util.Collections.emptyList(), KeyValueTable.getTables()); + this.queryRunner = PravegaQueryRunner.createQueryRunner(pravega.getController(), TpchTable.getTables(), KeyValueTable.getTables()); } @Test diff --git a/src/test/java/io/pravega/connectors/presto/integration/PravegaQueryRunner.java b/src/test/java/io/pravega/connectors/presto/integration/PravegaQueryRunner.java index 83c63e7..3559353 100644 --- a/src/test/java/io/pravega/connectors/presto/integration/PravegaQueryRunner.java +++ b/src/test/java/io/pravega/connectors/presto/integration/PravegaQueryRunner.java @@ -35,6 +35,8 @@ import io.pravega.connectors.presto.PravegaStreamDescription; import io.pravega.connectors.presto.PravegaTableDescriptionSupplier; import io.pravega.connectors.presto.PravegaTableName; +import io.pravega.connectors.presto.util.CodecSupplier; +import io.pravega.connectors.presto.util.PravegaTestUtils; import java.net.URI; import java.util.Map; @@ -44,8 +46,8 @@ import static com.facebook.presto.testing.TestingSession.testSessionBuilder; import static com.facebook.presto.tpch.TpchMetadata.TINY_SCHEMA_NAME; import static io.airlift.units.Duration.nanosSince; -import static io.pravega.connectors.presto.integration.PravegaTestUtils.getKvStreamDesc; -import static io.pravega.connectors.presto.integration.PravegaTestUtils.getStreamDesc; +import static io.pravega.connectors.presto.util.PravegaTestUtils.getKvStreamDesc; +import static io.pravega.connectors.presto.util.PravegaTestUtils.getStreamDesc; import static java.util.Locale.ENGLISH; import static java.util.concurrent.TimeUnit.SECONDS; @@ -122,7 +124,7 @@ private static void loadPravegaKVTable(URI controller, StreamManager streamManag private static PravegaTableDescriptionSupplier createSchemas(Metadata metadata, Iterable> tables, Iterable keyValueTables) { - JsonCodec streamDescCodec = new CodecSupplier<>(PravegaStreamDescription.class, metadata).get(); + JsonCodec streamDescCodec = new CodecSupplier<>(PravegaStreamDescription.class, metadata.getFunctionAndTypeManager()).get(); Cache schemaCache = CacheBuilder.newBuilder().build(); Cache> tableCache = CacheBuilder.newBuilder().build(); diff --git a/src/test/java/io/pravega/connectors/presto/schemamanagement/CompositeSchemaRegistryTest.java b/src/test/java/io/pravega/connectors/presto/schemamanagement/CompositeSchemaRegistryTest.java new file mode 100644 index 0000000..f7a656f --- /dev/null +++ b/src/test/java/io/pravega/connectors/presto/schemamanagement/CompositeSchemaRegistryTest.java @@ -0,0 +1,63 @@ +/* + * Copyright (c) Pravega Authors. + * + * 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.pravega.connectors.presto.schemamanagement; + +import com.facebook.presto.spi.SchemaTableName; +import io.pravega.connectors.presto.ObjectType; +import io.pravega.connectors.presto.PravegaStreamDescription; +import io.pravega.connectors.presto.PravegaStreamFieldGroup; +import io.pravega.connectors.presto.PravegaTableHandle; +import org.testng.annotations.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Optional; + +@Test +public class CompositeSchemaRegistryTest +{ + @Test + public void testOne() + { + List schemaSuppliers = new ArrayList<>(); + List schemaRegistries = new ArrayList<>(); + + schemaSuppliers.add(new SchemaSupplier() { + @Override + public List listSchemas() { + return Collections.singletonList("schema1"); + } + + @Override + public List listTables(String schema) { + return Collections.singletonList(new PravegaTableHandle("schema1", "table1", "table1", ObjectType.STREAM, Optional.empty())); + } + }); + + schemaRegistries.add(new SchemaRegistry() { + @Override + public PravegaStreamDescription getTable(SchemaTableName schemaTableName) { + return null; + } + + @Override + public List getSchema(SchemaTableName schemaTableName) { + return null; + } + }); + } +} diff --git a/src/test/java/io/pravega/connectors/presto/schemamanagement/LocalSchemaRegistryTest.java b/src/test/java/io/pravega/connectors/presto/schemamanagement/LocalSchemaRegistryTest.java new file mode 100644 index 0000000..fd37124 --- /dev/null +++ b/src/test/java/io/pravega/connectors/presto/schemamanagement/LocalSchemaRegistryTest.java @@ -0,0 +1,110 @@ +/* + * Copyright (c) Pravega Authors. + * + * 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.pravega.connectors.presto.schemamanagement; + +import com.facebook.presto.common.type.BigintType; +import com.facebook.presto.common.type.VarcharType; +import com.facebook.presto.spi.SchemaTableName; +import io.pravega.connectors.presto.PravegaStreamDescription; +import io.pravega.connectors.presto.PravegaStreamFieldDescription; +import io.pravega.connectors.presto.PravegaStreamFieldGroup; +import io.pravega.connectors.presto.PravegaTableHandle; +import io.pravega.connectors.presto.util.PravegaTestUtils; +import org.testng.annotations.Test; + +import java.util.List; +import java.util.stream.Collectors; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; + +@Test +public class LocalSchemaRegistryTest +{ + @Test + public void testListSchemas() + { + LocalSchemaRegistry schemaRegistry = PravegaTestUtils.localSchemaRegistry(); + + List schemas = schemaRegistry.listSchemas(); + + assertEquals(schemas.size(), 2); + + schemas = schemas.stream().sorted().collect(Collectors.toList()); + assertEquals(schemas.get(0), "kv"); + assertEquals(schemas.get(1), "tpch"); + } + + @Test + public void testListTables() + { + LocalSchemaRegistry schemaRegistry = PravegaTestUtils.localSchemaRegistry(); + + List tables = schemaRegistry.listTables("tpch"); + assertEquals(tables.size(), 8); + + PravegaTableHandle customerTableHandle = + tables.stream().filter(h -> h.getTableName().equals("customer")).findFirst().get(); + assertEquals(customerTableHandle.getSchemaName(), "tpch"); + assertEquals(customerTableHandle.getTableName(), "customer"); + } + + @Test + public void testGetSchema() + { + LocalSchemaRegistry schemaRegistry = PravegaTestUtils.localSchemaRegistry(); + + List schema = + schemaRegistry.getSchema(new SchemaTableName("tpch", "customer")); + + assertNotNull(schema); + assertEquals(1, schema.size()); + + validateCustomerSchema(schema.get(0)); + } + + @Test + public void testGetTable() + { + LocalSchemaRegistry schemaRegistry = PravegaTestUtils.localSchemaRegistry(); + + PravegaStreamDescription table = + schemaRegistry.getTable(new SchemaTableName("tpch", "customer")); + assertNotNull(table); + + assertTrue(table.getEvent().isPresent()); + assertEquals(1, table.getEvent().get().size()); + + validateCustomerSchema(table.getEvent().get().get(0)); + } + + private void validateCustomerSchema(PravegaStreamFieldGroup fieldGroup) + { + assertEquals(fieldGroup.getDataFormat(), "json"); + assertEquals(fieldGroup.getFields().size(), 8); + + PravegaStreamFieldDescription field = fieldGroup.getFields().get(0); + assertEquals(field.getName(), "custkey"); + assertEquals(field.getMapping(), "custkey"); + assertTrue(field.getType() instanceof BigintType); + + field = fieldGroup.getFields().get(6); + assertEquals(field.getName(), "mktsegment"); + assertEquals(field.getMapping(), "mktsegment"); + assertTrue(field.getType() instanceof VarcharType); + } +} diff --git a/src/test/java/io/pravega/connectors/presto/integration/CodecSupplier.java b/src/test/java/io/pravega/connectors/presto/util/CodecSupplier.java similarity index 87% rename from src/test/java/io/pravega/connectors/presto/integration/CodecSupplier.java rename to src/test/java/io/pravega/connectors/presto/util/CodecSupplier.java index 2334f1e..884188e 100644 --- a/src/test/java/io/pravega/connectors/presto/integration/CodecSupplier.java +++ b/src/test/java/io/pravega/connectors/presto/util/CodecSupplier.java @@ -15,13 +15,13 @@ * (rev a8968160e1840ac67a5f63def27d31c0ef0acde7) * https://github.com/prestodb/presto/blob/0.247/presto-kafka/src/test/java/com/facebook/presto/kafka/util/CodecSupplier.java */ -package io.pravega.connectors.presto.integration; +package io.pravega.connectors.presto.util; import com.facebook.airlift.json.JsonCodec; import com.facebook.airlift.json.JsonCodecFactory; import com.facebook.airlift.json.ObjectMapperProvider; import com.facebook.presto.common.type.Type; -import com.facebook.presto.metadata.Metadata; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.fasterxml.jackson.databind.DeserializationContext; import com.fasterxml.jackson.databind.deser.std.FromStringDeserializer; import com.google.common.collect.ImmutableMap; @@ -33,14 +33,14 @@ public final class CodecSupplier implements Supplier> { - private final Metadata metadata; + private final FunctionAndTypeManager typeManager; private final JsonCodecFactory codecFactory; private final Class clazz; - public CodecSupplier(Class clazz, Metadata metadata) + public CodecSupplier(Class clazz, FunctionAndTypeManager typeManager) { this.clazz = clazz; - this.metadata = metadata; + this.typeManager = typeManager; ObjectMapperProvider objectMapperProvider = new ObjectMapperProvider(); objectMapperProvider.setJsonDeserializers(ImmutableMap.of(Type.class, new TypeDeserializer())); this.codecFactory = new JsonCodecFactory(objectMapperProvider); @@ -65,7 +65,7 @@ public TypeDeserializer() @Override protected Type _deserialize(String value, DeserializationContext context) { - Type type = metadata.getType(parseTypeSignature(value)); + Type type = typeManager.getType(parseTypeSignature(value)); if (type == null) { throw new IllegalArgumentException(String.valueOf("Unknown type " + value)); } diff --git a/src/test/java/io/pravega/connectors/presto/integration/PravegaTestUtils.java b/src/test/java/io/pravega/connectors/presto/util/PravegaTestUtils.java similarity index 83% rename from src/test/java/io/pravega/connectors/presto/integration/PravegaTestUtils.java rename to src/test/java/io/pravega/connectors/presto/util/PravegaTestUtils.java index 2452608..bde846f 100644 --- a/src/test/java/io/pravega/connectors/presto/integration/PravegaTestUtils.java +++ b/src/test/java/io/pravega/connectors/presto/util/PravegaTestUtils.java @@ -13,10 +13,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.pravega.connectors.presto.integration; +package io.pravega.connectors.presto.util; import com.facebook.airlift.json.JsonCodec; import com.facebook.presto.common.QualifiedObjectName; +import com.facebook.presto.metadata.FunctionAndTypeManager; import com.facebook.presto.tests.TestingPrestoClient; import com.google.common.io.ByteStreams; import com.google.common.io.CharStreams; @@ -24,10 +25,14 @@ import io.pravega.connectors.presto.PravegaStreamDescription; import io.pravega.connectors.presto.PravegaStreamFieldDescription; import io.pravega.connectors.presto.PravegaStreamFieldGroup; +import io.pravega.connectors.presto.integration.PravegaKeyValueLoader; +import io.pravega.connectors.presto.integration.PravegaLoader; +import io.pravega.connectors.presto.schemamanagement.LocalSchemaRegistry; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecordBuilder; import java.io.BufferedReader; +import java.io.File; import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; @@ -43,9 +48,17 @@ public final class PravegaTestUtils { private PravegaTestUtils() {} - public static PravegaStreamDescription getStreamDesc(JsonCodec streamDescriptionCodec, String directory, String table) + public static LocalSchemaRegistry localSchemaRegistry() { - try (InputStream inputStream = PravegaTestUtils.class.getResourceAsStream(String.format("/%s/%s.json", directory, table))) { + JsonCodec streamDescCodec = new CodecSupplier<>( + PravegaStreamDescription.class, + FunctionAndTypeManager.createTestFunctionAndTypeManager()).get(); + return new LocalSchemaRegistry(new File("src/test/resources/etc").getAbsoluteFile(), streamDescCodec); + } + + public static PravegaStreamDescription getStreamDesc(JsonCodec streamDescriptionCodec, String schema, String table) + { + try (InputStream inputStream = PravegaTestUtils.class.getResourceAsStream(String.format("/etc/%s.%s.json", schema, table))) { return streamDescriptionCodec.fromJson(ByteStreams.toByteArray(inputStream)); } catch (IOException e) { @@ -53,9 +66,9 @@ public static PravegaStreamDescription getStreamDesc(JsonCodec jsonCodec = JsonCodec.jsonCodec(PravegaStreamDescription.class); PravegaStreamDescription streamDescription = getStreamDesc(jsonCodec, "kv", table); + streamDescription = getStreamDesc(jsonCodec, "tpch", "customer"); streamDescription.getEvent().orElseThrow(IllegalArgumentException::new); PravegaStreamFieldGroup keyEvent = streamDescription.getEvent().get().get(0); keyEvent.getDataSchema().orElseThrow(IllegalArgumentException::new); - String keySchema = PravegaTestUtils.readSchema("kv", keyEvent.getDataSchema().get()); + String keySchema = PravegaTestUtils.readSchema(keyEvent.getDataSchema().get()); PravegaStreamFieldGroup valueEvent = streamDescription.getEvent().get().get(1); valueEvent.getDataSchema().orElseThrow(IllegalArgumentException::new); - String valueSchema = PravegaTestUtils.readSchema("kv", valueEvent.getDataSchema().get()); + String valueSchema = PravegaTestUtils.readSchema(valueEvent.getDataSchema().get()); List keyFields = mapFieldsFromSchema("key", keyEvent.getDataFormat(), keySchema); diff --git a/src/test/resources/kv/employee-id.avsc b/src/test/resources/etc/employee-id.avsc similarity index 100% rename from src/test/resources/kv/employee-id.avsc rename to src/test/resources/etc/employee-id.avsc diff --git a/src/test/resources/kv/employee-value.avsc b/src/test/resources/etc/employee-value.avsc similarity index 100% rename from src/test/resources/kv/employee-value.avsc rename to src/test/resources/etc/employee-value.avsc diff --git a/src/test/resources/kv/employee.records b/src/test/resources/etc/employee.records similarity index 100% rename from src/test/resources/kv/employee.records rename to src/test/resources/etc/employee.records diff --git a/src/test/resources/kv/employee.json b/src/test/resources/etc/kv.employee.json similarity index 100% rename from src/test/resources/kv/employee.json rename to src/test/resources/etc/kv.employee.json diff --git a/src/test/resources/tpch/customer.json b/src/test/resources/etc/tpch.customer.json similarity index 100% rename from src/test/resources/tpch/customer.json rename to src/test/resources/etc/tpch.customer.json diff --git a/src/test/resources/tpch/lineitem.json b/src/test/resources/etc/tpch.lineitem.json similarity index 100% rename from src/test/resources/tpch/lineitem.json rename to src/test/resources/etc/tpch.lineitem.json diff --git a/src/test/resources/tpch/nation.json b/src/test/resources/etc/tpch.nation.json similarity index 100% rename from src/test/resources/tpch/nation.json rename to src/test/resources/etc/tpch.nation.json diff --git a/src/test/resources/tpch/orders.json b/src/test/resources/etc/tpch.orders.json similarity index 100% rename from src/test/resources/tpch/orders.json rename to src/test/resources/etc/tpch.orders.json diff --git a/src/test/resources/tpch/part.json b/src/test/resources/etc/tpch.part.json similarity index 100% rename from src/test/resources/tpch/part.json rename to src/test/resources/etc/tpch.part.json diff --git a/src/test/resources/tpch/partsupp.json b/src/test/resources/etc/tpch.partsupp.json similarity index 100% rename from src/test/resources/tpch/partsupp.json rename to src/test/resources/etc/tpch.partsupp.json diff --git a/src/test/resources/tpch/region.json b/src/test/resources/etc/tpch.region.json similarity index 100% rename from src/test/resources/tpch/region.json rename to src/test/resources/etc/tpch.region.json diff --git a/src/test/resources/tpch/supplier.json b/src/test/resources/etc/tpch.supplier.json similarity index 100% rename from src/test/resources/tpch/supplier.json rename to src/test/resources/etc/tpch.supplier.json diff --git a/src/test/unit-test.xml b/src/test/unit-test.xml index 1737cd5..eca6ae0 100644 --- a/src/test/unit-test.xml +++ b/src/test/unit-test.xml @@ -4,6 +4,7 @@ + From 7b2a4e089a27ed3df7f2ee012b70a357b5bdfb8a Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Thu, 25 Mar 2021 14:15:55 -0700 Subject: [PATCH 18/32] separate schemas from different tests Signed-off-by: Andrew Robertson --- .../PravegaTableDescriptionSupplier.java | 13 ++-- .../schemamanagement/LocalSchemaRegistry.java | 2 +- .../schemamanagement/SchemaRegistry.java | 3 + .../schemamanagement/SchemaSupplier.java | 4 ++ .../presto/util/PravegaSchemaUtils.java | 5 +- .../presto/util/PravegaStreamDescUtils.java | 7 ++- .../integration/PravegaKeyValueTest.java | 3 +- .../integration/PravegaQueryRunner.java | 52 ++++++--------- .../CompositeSchemaRegistryTest.java | 63 ------------------- .../LocalSchemaRegistryTest.java | 28 ++++++--- .../presto/util/PravegaTestUtils.java | 62 ++---------------- .../resources/{etc => kv}/employee-id.avsc | 0 .../resources/{etc => kv}/employee-value.avsc | 0 .../resources/{etc => kv}/employee.records | 0 .../resources/{etc => kv}/kv.employee.json | 0 .../{etc => tpch}/tpch.customer.json | 0 .../{etc => tpch}/tpch.lineitem.json | 0 .../resources/{etc => tpch}/tpch.nation.json | 0 .../resources/{etc => tpch}/tpch.orders.json | 0 .../resources/{etc => tpch}/tpch.part.json | 0 .../{etc => tpch}/tpch.partsupp.json | 0 .../resources/{etc => tpch}/tpch.region.json | 0 .../{etc => tpch}/tpch.supplier.json | 0 23 files changed, 69 insertions(+), 173 deletions(-) delete mode 100644 src/test/java/io/pravega/connectors/presto/schemamanagement/CompositeSchemaRegistryTest.java rename src/test/resources/{etc => kv}/employee-id.avsc (100%) rename src/test/resources/{etc => kv}/employee-value.avsc (100%) rename src/test/resources/{etc => kv}/employee.records (100%) rename src/test/resources/{etc => kv}/kv.employee.json (100%) rename src/test/resources/{etc => tpch}/tpch.customer.json (100%) rename src/test/resources/{etc => tpch}/tpch.lineitem.json (100%) rename src/test/resources/{etc => tpch}/tpch.nation.json (100%) rename src/test/resources/{etc => tpch}/tpch.orders.json (100%) rename src/test/resources/{etc => tpch}/tpch.part.json (100%) rename src/test/resources/{etc => tpch}/tpch.partsupp.json (100%) rename src/test/resources/{etc => tpch}/tpch.region.json (100%) rename src/test/resources/{etc => tpch}/tpch.supplier.json (100%) diff --git a/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java b/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java index 7af70a6..f18d4d3 100644 --- a/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java +++ b/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java @@ -84,12 +84,12 @@ public class PravegaTableDescriptionSupplier } @VisibleForTesting - public PravegaTableDescriptionSupplier(Cache schemaCache, - Cache> tableCache) + public PravegaTableDescriptionSupplier(CompositeSchemaRegistry schemaRegistry) { - this.schemaCache = schemaCache; - this.tableCache = tableCache; - this.schemaRegistry = null; + this.schemaRegistry = schemaRegistry; + + this.schemaCache = CacheBuilder.newBuilder().build(); + this.tableCache = CacheBuilder.newBuilder().build(); } public List listSchemas() @@ -174,6 +174,9 @@ public PravegaStreamDescription getTable(SchemaTableName schemaTableName) } PravegaStreamDescription table = schemaRegistry.getTable(schemaTableName); + if (table == null) { + return null; + } if (multiSourceStream(table)) { // if component streams not already specified, look them up from pravega based on regex diff --git a/src/main/java/io/pravega/connectors/presto/schemamanagement/LocalSchemaRegistry.java b/src/main/java/io/pravega/connectors/presto/schemamanagement/LocalSchemaRegistry.java index 067938f..7da4b49 100644 --- a/src/main/java/io/pravega/connectors/presto/schemamanagement/LocalSchemaRegistry.java +++ b/src/main/java/io/pravega/connectors/presto/schemamanagement/LocalSchemaRegistry.java @@ -108,7 +108,7 @@ public PravegaStreamDescription getTable(SchemaTableName schemaTableName) // at least 1 schema for a fieldGroup must be resolved. read schema from local file or url List finalSchemas = - resolveAllSchemas(table.getEvent().get(), (i) -> columnPrefix(table, i)); + resolveAllSchemas(localTableDir, table.getEvent().get(), (i) -> columnPrefix(table, i)); return new PravegaStreamDescription(table, finalSchemas); } diff --git a/src/main/java/io/pravega/connectors/presto/schemamanagement/SchemaRegistry.java b/src/main/java/io/pravega/connectors/presto/schemamanagement/SchemaRegistry.java index 451de5f..0a5c28c 100644 --- a/src/main/java/io/pravega/connectors/presto/schemamanagement/SchemaRegistry.java +++ b/src/main/java/io/pravega/connectors/presto/schemamanagement/SchemaRegistry.java @@ -21,6 +21,9 @@ import java.util.List; +/** + * return schema for the given schema.table + */ public interface SchemaRegistry { PravegaStreamDescription getTable(SchemaTableName schemaTableName); diff --git a/src/main/java/io/pravega/connectors/presto/schemamanagement/SchemaSupplier.java b/src/main/java/io/pravega/connectors/presto/schemamanagement/SchemaSupplier.java index 20bc1b2..c73bcea 100644 --- a/src/main/java/io/pravega/connectors/presto/schemamanagement/SchemaSupplier.java +++ b/src/main/java/io/pravega/connectors/presto/schemamanagement/SchemaSupplier.java @@ -19,6 +19,10 @@ import java.util.List; +/** + * a source of schemas, and tables + * this may be reading files on local disk or listing scopes+streams from pravega + */ public interface SchemaSupplier { List listSchemas(); diff --git a/src/main/java/io/pravega/connectors/presto/util/PravegaSchemaUtils.java b/src/main/java/io/pravega/connectors/presto/util/PravegaSchemaUtils.java index 1938c6b..4fbfb71 100644 --- a/src/main/java/io/pravega/connectors/presto/util/PravegaSchemaUtils.java +++ b/src/main/java/io/pravega/connectors/presto/util/PravegaSchemaUtils.java @@ -20,6 +20,7 @@ import com.facebook.presto.spi.PrestoException; import com.google.common.io.CharStreams; +import java.io.File; import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; @@ -54,7 +55,7 @@ private PravegaSchemaUtils() public static final String NESTED_RECORD_SEPARATOR = "/"; - public static String readSchema(String dataSchemaLocation) + public static String readSchema(File schemaDir, String dataSchemaLocation) { InputStream inputStream = null; try { @@ -65,7 +66,7 @@ public static String readSchema(String dataSchemaLocation) catch (MalformedURLException e) { // try again before failing log.warn("invalid URL: " + dataSchemaLocation); - inputStream = new FileInputStream(dataSchemaLocation); + inputStream = new FileInputStream(new File(schemaDir, dataSchemaLocation)); } } else { diff --git a/src/main/java/io/pravega/connectors/presto/util/PravegaStreamDescUtils.java b/src/main/java/io/pravega/connectors/presto/util/PravegaStreamDescUtils.java index 38f4c35..96c435c 100644 --- a/src/main/java/io/pravega/connectors/presto/util/PravegaStreamDescUtils.java +++ b/src/main/java/io/pravega/connectors/presto/util/PravegaStreamDescUtils.java @@ -29,6 +29,7 @@ import org.everit.json.schema.Schema; import org.everit.json.schema.StringSchema; +import java.io.File; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; @@ -351,11 +352,13 @@ private static String nestedPrefixFor(String prefix, String name) * PravegaStreamFieldGroup may contain pointer to schema (local file, or url) * for these, read the schema and build the field definitions * + * @param schemaDir directory where we can find the schema * @param fieldGroups fieldGroups to look through * @param columnPrefix function to return columnPrefix to be used for the fields in the group * @return list of PravegaStreamFieldGroup with all schemas resolved */ - public static List resolveAllSchemas(List fieldGroups, + public static List resolveAllSchemas(File schemaDir, + List fieldGroups, Function columnPrefix) { // fields already defined @@ -369,7 +372,7 @@ public static List resolveAllSchemas(List fields = mapFieldsFromSchema(columnPrefix.apply(i), fieldGroup.getDataFormat(), dataSchema); finalSchemas.add(new PravegaStreamFieldGroup(fieldGroup, dataSchema, fields)); diff --git a/src/test/java/io/pravega/connectors/presto/integration/PravegaKeyValueTest.java b/src/test/java/io/pravega/connectors/presto/integration/PravegaKeyValueTest.java index 30f2f8e..99307e3 100644 --- a/src/test/java/io/pravega/connectors/presto/integration/PravegaKeyValueTest.java +++ b/src/test/java/io/pravega/connectors/presto/integration/PravegaKeyValueTest.java @@ -18,7 +18,6 @@ import com.facebook.presto.Session; import com.facebook.presto.testing.MaterializedResult; import com.facebook.presto.tests.DistributedQueryRunner; -import io.airlift.tpch.TpchTable; import io.pravega.connectors.presto.PravegaStreamDescription; import io.pravega.connectors.presto.PravegaStreamFieldDescription; import org.testng.annotations.AfterClass; @@ -41,7 +40,7 @@ public PravegaKeyValueTest() throws Exception { this.pravega = new EmbeddedPravega(); - this.queryRunner = PravegaQueryRunner.createQueryRunner(pravega.getController(), TpchTable.getTables(), KeyValueTable.getTables()); + this.queryRunner = PravegaQueryRunner.createQueryRunner(pravega.getController(), java.util.Collections.emptyList(), KeyValueTable.getTables()); } @Test diff --git a/src/test/java/io/pravega/connectors/presto/integration/PravegaQueryRunner.java b/src/test/java/io/pravega/connectors/presto/integration/PravegaQueryRunner.java index 3559353..78ad2b9 100644 --- a/src/test/java/io/pravega/connectors/presto/integration/PravegaQueryRunner.java +++ b/src/test/java/io/pravega/connectors/presto/integration/PravegaQueryRunner.java @@ -15,39 +15,35 @@ */ package io.pravega.connectors.presto.integration; -import com.facebook.airlift.json.JsonCodec; import com.facebook.airlift.log.Logger; import com.facebook.airlift.log.Logging; import com.facebook.presto.Session; import com.facebook.presto.common.QualifiedObjectName; -import com.facebook.presto.metadata.Metadata; -import com.facebook.presto.spi.SchemaTableName; import com.facebook.presto.testing.QueryRunner; import com.facebook.presto.tests.DistributedQueryRunner; import com.facebook.presto.tests.TestingPrestoClient; import com.facebook.presto.tpch.TpchPlugin; -import com.google.common.cache.Cache; -import com.google.common.cache.CacheBuilder; import com.google.common.collect.ImmutableMap; import io.airlift.tpch.TpchTable; import io.pravega.client.admin.StreamManager; import io.pravega.connectors.presto.PravegaPlugin; -import io.pravega.connectors.presto.PravegaStreamDescription; import io.pravega.connectors.presto.PravegaTableDescriptionSupplier; -import io.pravega.connectors.presto.PravegaTableName; -import io.pravega.connectors.presto.util.CodecSupplier; +import io.pravega.connectors.presto.schemamanagement.CompositeSchemaRegistry; +import io.pravega.connectors.presto.schemamanagement.LocalSchemaRegistry; +import io.pravega.connectors.presto.schemamanagement.SchemaRegistry; +import io.pravega.connectors.presto.schemamanagement.SchemaSupplier; import io.pravega.connectors.presto.util.PravegaTestUtils; import java.net.URI; +import java.util.ArrayList; +import java.util.List; import java.util.Map; -import java.util.Optional; import static com.facebook.airlift.testing.Closeables.closeAllSuppress; import static com.facebook.presto.testing.TestingSession.testSessionBuilder; import static com.facebook.presto.tpch.TpchMetadata.TINY_SCHEMA_NAME; import static io.airlift.units.Duration.nanosSince; -import static io.pravega.connectors.presto.util.PravegaTestUtils.getKvStreamDesc; -import static io.pravega.connectors.presto.util.PravegaTestUtils.getStreamDesc; + import static java.util.Locale.ENGLISH; import static java.util.concurrent.TimeUnit.SECONDS; @@ -73,8 +69,7 @@ public static DistributedQueryRunner createQueryRunner(URI controller, Iterable< queryRunner.installPlugin(new TpchPlugin()); queryRunner.createCatalog("tpch", "tpch"); - PravegaTableDescriptionSupplier tableDescriptionSupplier = - createSchemas(queryRunner.getCoordinator().getMetadata(), tpchTables, keyValueTables); + PravegaTableDescriptionSupplier tableDescriptionSupplier = createTableDescriptionSupplier(tpchTables, keyValueTables); installPlugin(controller, queryRunner, tableDescriptionSupplier); @@ -122,31 +117,24 @@ private static void loadPravegaKVTable(URI controller, StreamManager streamManag log.info("Imported %s in %s", 0, table, nanosSince(start).convertToMostSuccinctTimeUnit()); } - private static PravegaTableDescriptionSupplier createSchemas(Metadata metadata, Iterable> tables, Iterable keyValueTables) + private static PravegaTableDescriptionSupplier createTableDescriptionSupplier(Iterable> tpchTables, Iterable keyValueTables) { - JsonCodec streamDescCodec = new CodecSupplier<>(PravegaStreamDescription.class, metadata.getFunctionAndTypeManager()).get(); - - Cache schemaCache = CacheBuilder.newBuilder().build(); - Cache> tableCache = CacheBuilder.newBuilder().build(); + List schemaSuppliers = new ArrayList<>(); + List schemaRegistries = new ArrayList<>(); - for (TpchTable table : tables) { - SchemaTableName schemaTableName = new SchemaTableName(TPCH_SCHEMA, table.getTableName()); - PravegaTableName pravegaTableName = new PravegaTableName(schemaTableName); - - schemaCache.put(schemaTableName.getSchemaName(), new Object()); - tableCache.put(pravegaTableName, Optional.of(getStreamDesc(streamDescCodec, "tpch", table.getTableName()))); + if (tpchTables.iterator().hasNext()) { + LocalSchemaRegistry tpch = PravegaTestUtils.localSchemaRegistry("tpch"); + schemaSuppliers.add(tpch); + schemaRegistries.add(tpch); } - for (String table : keyValueTables) { - SchemaTableName schemaTableName = new SchemaTableName(KV_SCHEMA, table); - PravegaTableName pravegaTableName = new PravegaTableName(schemaTableName); - - schemaCache.put(schemaTableName.getSchemaName(), new Object()); - tableCache.put(pravegaTableName, Optional.of(getKvStreamDesc(table))); + if (keyValueTables.iterator().hasNext()) { + LocalSchemaRegistry kv = PravegaTestUtils.localSchemaRegistry("kv"); + schemaSuppliers.add(kv); + schemaRegistries.add(kv); } - // all schemas + tables will be served from these provided caches - return new PravegaTableDescriptionSupplier(schemaCache, tableCache); + return new PravegaTableDescriptionSupplier(new CompositeSchemaRegistry(schemaSuppliers, schemaRegistries)); } public static Session createSession() diff --git a/src/test/java/io/pravega/connectors/presto/schemamanagement/CompositeSchemaRegistryTest.java b/src/test/java/io/pravega/connectors/presto/schemamanagement/CompositeSchemaRegistryTest.java deleted file mode 100644 index f7a656f..0000000 --- a/src/test/java/io/pravega/connectors/presto/schemamanagement/CompositeSchemaRegistryTest.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Copyright (c) Pravega Authors. - * - * 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.pravega.connectors.presto.schemamanagement; - -import com.facebook.presto.spi.SchemaTableName; -import io.pravega.connectors.presto.ObjectType; -import io.pravega.connectors.presto.PravegaStreamDescription; -import io.pravega.connectors.presto.PravegaStreamFieldGroup; -import io.pravega.connectors.presto.PravegaTableHandle; -import org.testng.annotations.Test; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Optional; - -@Test -public class CompositeSchemaRegistryTest -{ - @Test - public void testOne() - { - List schemaSuppliers = new ArrayList<>(); - List schemaRegistries = new ArrayList<>(); - - schemaSuppliers.add(new SchemaSupplier() { - @Override - public List listSchemas() { - return Collections.singletonList("schema1"); - } - - @Override - public List listTables(String schema) { - return Collections.singletonList(new PravegaTableHandle("schema1", "table1", "table1", ObjectType.STREAM, Optional.empty())); - } - }); - - schemaRegistries.add(new SchemaRegistry() { - @Override - public PravegaStreamDescription getTable(SchemaTableName schemaTableName) { - return null; - } - - @Override - public List getSchema(SchemaTableName schemaTableName) { - return null; - } - }); - } -} diff --git a/src/test/java/io/pravega/connectors/presto/schemamanagement/LocalSchemaRegistryTest.java b/src/test/java/io/pravega/connectors/presto/schemamanagement/LocalSchemaRegistryTest.java index fd37124..724b66e 100644 --- a/src/test/java/io/pravega/connectors/presto/schemamanagement/LocalSchemaRegistryTest.java +++ b/src/test/java/io/pravega/connectors/presto/schemamanagement/LocalSchemaRegistryTest.java @@ -26,33 +26,31 @@ import org.testng.annotations.Test; import java.util.List; -import java.util.stream.Collectors; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; @Test public class LocalSchemaRegistryTest { + // uses resources/tpch for table description dir + @Test public void testListSchemas() { - LocalSchemaRegistry schemaRegistry = PravegaTestUtils.localSchemaRegistry(); + LocalSchemaRegistry schemaRegistry = PravegaTestUtils.localSchemaRegistry("tpch"); List schemas = schemaRegistry.listSchemas(); - - assertEquals(schemas.size(), 2); - - schemas = schemas.stream().sorted().collect(Collectors.toList()); - assertEquals(schemas.get(0), "kv"); + assertEquals(schemas.size(), 1); assertEquals(schemas.get(1), "tpch"); } @Test public void testListTables() { - LocalSchemaRegistry schemaRegistry = PravegaTestUtils.localSchemaRegistry(); + LocalSchemaRegistry schemaRegistry = PravegaTestUtils.localSchemaRegistry("tpch"); List tables = schemaRegistry.listTables("tpch"); assertEquals(tables.size(), 8); @@ -61,12 +59,13 @@ public void testListTables() tables.stream().filter(h -> h.getTableName().equals("customer")).findFirst().get(); assertEquals(customerTableHandle.getSchemaName(), "tpch"); assertEquals(customerTableHandle.getTableName(), "customer"); + assertEquals(customerTableHandle.getObjectName(), "customer"); } @Test public void testGetSchema() { - LocalSchemaRegistry schemaRegistry = PravegaTestUtils.localSchemaRegistry(); + LocalSchemaRegistry schemaRegistry = PravegaTestUtils.localSchemaRegistry("tpch"); List schema = schemaRegistry.getSchema(new SchemaTableName("tpch", "customer")); @@ -80,7 +79,7 @@ public void testGetSchema() @Test public void testGetTable() { - LocalSchemaRegistry schemaRegistry = PravegaTestUtils.localSchemaRegistry(); + LocalSchemaRegistry schemaRegistry = PravegaTestUtils.localSchemaRegistry("tpch"); PravegaStreamDescription table = schemaRegistry.getTable(new SchemaTableName("tpch", "customer")); @@ -92,8 +91,17 @@ public void testGetTable() validateCustomerSchema(table.getEvent().get().get(0)); } + @Test + public void testTableDoesNotExist() + { + LocalSchemaRegistry schemaRegistry = PravegaTestUtils.localSchemaRegistry("tpch"); + assertNull(schemaRegistry.getTable(new SchemaTableName("tpch", "abcxyz123"))); + } + private void validateCustomerSchema(PravegaStreamFieldGroup fieldGroup) { + // spot check a fiew fields + assertEquals(fieldGroup.getDataFormat(), "json"); assertEquals(fieldGroup.getFields().size(), 8); diff --git a/src/test/java/io/pravega/connectors/presto/util/PravegaTestUtils.java b/src/test/java/io/pravega/connectors/presto/util/PravegaTestUtils.java index bde846f..ecab9c1 100644 --- a/src/test/java/io/pravega/connectors/presto/util/PravegaTestUtils.java +++ b/src/test/java/io/pravega/connectors/presto/util/PravegaTestUtils.java @@ -18,13 +18,10 @@ import com.facebook.airlift.json.JsonCodec; import com.facebook.presto.common.QualifiedObjectName; import com.facebook.presto.metadata.FunctionAndTypeManager; +import com.facebook.presto.spi.SchemaTableName; import com.facebook.presto.tests.TestingPrestoClient; -import com.google.common.io.ByteStreams; -import com.google.common.io.CharStreams; import io.pravega.client.admin.StreamManager; import io.pravega.connectors.presto.PravegaStreamDescription; -import io.pravega.connectors.presto.PravegaStreamFieldDescription; -import io.pravega.connectors.presto.PravegaStreamFieldGroup; import io.pravega.connectors.presto.integration.PravegaKeyValueLoader; import io.pravega.connectors.presto.integration.PravegaLoader; import io.pravega.connectors.presto.schemamanagement.LocalSchemaRegistry; @@ -38,42 +35,19 @@ import java.io.InputStreamReader; import java.io.UncheckedIOException; import java.net.URI; -import java.util.ArrayList; -import java.util.List; -import static io.pravega.connectors.presto.util.PravegaStreamDescUtils.mapFieldsFromSchema; import static java.lang.String.format; public final class PravegaTestUtils { private PravegaTestUtils() {} - public static LocalSchemaRegistry localSchemaRegistry() + public static LocalSchemaRegistry localSchemaRegistry(String dir) { JsonCodec streamDescCodec = new CodecSupplier<>( PravegaStreamDescription.class, FunctionAndTypeManager.createTestFunctionAndTypeManager()).get(); - return new LocalSchemaRegistry(new File("src/test/resources/etc").getAbsoluteFile(), streamDescCodec); - } - - public static PravegaStreamDescription getStreamDesc(JsonCodec streamDescriptionCodec, String schema, String table) - { - try (InputStream inputStream = PravegaTestUtils.class.getResourceAsStream(String.format("/etc/%s.%s.json", schema, table))) { - return streamDescriptionCodec.fromJson(ByteStreams.toByteArray(inputStream)); - } - catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - public static String readSchema(String schema) - { - try (InputStreamReader reader = new InputStreamReader(PravegaTestUtils.class.getResourceAsStream(String.format("/etc/%s", schema)))) { - return CharStreams.toString(reader); - } - catch (IOException e) { - throw new UncheckedIOException(e); - } + return new LocalSchemaRegistry(new File("src/test/resources/" + dir).getAbsoluteFile(), streamDescCodec); } public static void loadTpchStream(URI controller, StreamManager streamManager, TestingPrestoClient prestoClient, String schema, String stream, QualifiedObjectName tpchTableName) @@ -98,8 +72,8 @@ public static void loadKeyValueTable(URI controller, StreamManager streamManager streamManager, schema, table, avroSchema(tableDesc, 0), avroSchema(tableDesc, 1))) { - try (InputStream inputStream = PravegaTestUtils.class.getResourceAsStream(String.format("/etc/%s.records", table)); - BufferedReader reader = new BufferedReader(new InputStreamReader(inputStream))) { + try (InputStream inputStream = PravegaTestUtils.class.getResourceAsStream(String.format("/kv/%s.records", table)); + BufferedReader reader = new BufferedReader(new InputStreamReader(inputStream))) { // each line in file is a record, key + value // '|' character separates key fields from values fields // fields separated by ',' @@ -152,31 +126,7 @@ static void setAvroValue(GenericRecordBuilder builder, Schema.Field field, Strin public static PravegaStreamDescription getKvStreamDesc(String table) { - JsonCodec jsonCodec = JsonCodec.jsonCodec(PravegaStreamDescription.class); - - PravegaStreamDescription streamDescription = getStreamDesc(jsonCodec, "kv", table); - streamDescription = getStreamDesc(jsonCodec, "tpch", "customer"); - streamDescription.getEvent().orElseThrow(IllegalArgumentException::new); - - PravegaStreamFieldGroup keyEvent = streamDescription.getEvent().get().get(0); - keyEvent.getDataSchema().orElseThrow(IllegalArgumentException::new); - String keySchema = PravegaTestUtils.readSchema(keyEvent.getDataSchema().get()); - - PravegaStreamFieldGroup valueEvent = streamDescription.getEvent().get().get(1); - valueEvent.getDataSchema().orElseThrow(IllegalArgumentException::new); - String valueSchema = PravegaTestUtils.readSchema(valueEvent.getDataSchema().get()); - - List keyFields = - mapFieldsFromSchema("key", keyEvent.getDataFormat(), keySchema); - - List valueFields = - mapFieldsFromSchema("value", valueEvent.getDataFormat(), valueSchema); - - List newFieldGroups = new ArrayList<>(2); - newFieldGroups.add(new PravegaStreamFieldGroup(keyEvent, keySchema, keyFields)); - newFieldGroups.add(new PravegaStreamFieldGroup(valueEvent, valueSchema, valueFields)); - - return new PravegaStreamDescription(streamDescription, newFieldGroups); + return localSchemaRegistry("kv").getTable(new SchemaTableName("kv", table)); } public static Schema avroSchema(PravegaStreamDescription streamDescription, int event) diff --git a/src/test/resources/etc/employee-id.avsc b/src/test/resources/kv/employee-id.avsc similarity index 100% rename from src/test/resources/etc/employee-id.avsc rename to src/test/resources/kv/employee-id.avsc diff --git a/src/test/resources/etc/employee-value.avsc b/src/test/resources/kv/employee-value.avsc similarity index 100% rename from src/test/resources/etc/employee-value.avsc rename to src/test/resources/kv/employee-value.avsc diff --git a/src/test/resources/etc/employee.records b/src/test/resources/kv/employee.records similarity index 100% rename from src/test/resources/etc/employee.records rename to src/test/resources/kv/employee.records diff --git a/src/test/resources/etc/kv.employee.json b/src/test/resources/kv/kv.employee.json similarity index 100% rename from src/test/resources/etc/kv.employee.json rename to src/test/resources/kv/kv.employee.json diff --git a/src/test/resources/etc/tpch.customer.json b/src/test/resources/tpch/tpch.customer.json similarity index 100% rename from src/test/resources/etc/tpch.customer.json rename to src/test/resources/tpch/tpch.customer.json diff --git a/src/test/resources/etc/tpch.lineitem.json b/src/test/resources/tpch/tpch.lineitem.json similarity index 100% rename from src/test/resources/etc/tpch.lineitem.json rename to src/test/resources/tpch/tpch.lineitem.json diff --git a/src/test/resources/etc/tpch.nation.json b/src/test/resources/tpch/tpch.nation.json similarity index 100% rename from src/test/resources/etc/tpch.nation.json rename to src/test/resources/tpch/tpch.nation.json diff --git a/src/test/resources/etc/tpch.orders.json b/src/test/resources/tpch/tpch.orders.json similarity index 100% rename from src/test/resources/etc/tpch.orders.json rename to src/test/resources/tpch/tpch.orders.json diff --git a/src/test/resources/etc/tpch.part.json b/src/test/resources/tpch/tpch.part.json similarity index 100% rename from src/test/resources/etc/tpch.part.json rename to src/test/resources/tpch/tpch.part.json diff --git a/src/test/resources/etc/tpch.partsupp.json b/src/test/resources/tpch/tpch.partsupp.json similarity index 100% rename from src/test/resources/etc/tpch.partsupp.json rename to src/test/resources/tpch/tpch.partsupp.json diff --git a/src/test/resources/etc/tpch.region.json b/src/test/resources/tpch/tpch.region.json similarity index 100% rename from src/test/resources/etc/tpch.region.json rename to src/test/resources/tpch/tpch.region.json diff --git a/src/test/resources/etc/tpch.supplier.json b/src/test/resources/tpch/tpch.supplier.json similarity index 100% rename from src/test/resources/etc/tpch.supplier.json rename to src/test/resources/tpch/tpch.supplier.json From cd037a9e3c2b32276ba2a5c4625284e9b38f0a11 Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Thu, 25 Mar 2021 14:54:12 -0700 Subject: [PATCH 19/32] add unit test for pravega schema registry Signed-off-by: Andrew Robertson --- .../PravegaSchemaRegistry.java | 8 + .../PravegaSchemaRegistryTest.java | 124 ++++++++++++ .../presto/util/MockSchemaRegistryClient.java | 181 ++++++++++++++++++ .../presto/util/MockStreamManager.java | 123 ++++++++++++ .../presto/util/PravegaTestUtils.java | 5 + 5 files changed, 441 insertions(+) create mode 100644 src/test/java/io/pravega/connectors/presto/schemamanagement/PravegaSchemaRegistryTest.java create mode 100644 src/test/java/io/pravega/connectors/presto/util/MockSchemaRegistryClient.java create mode 100644 src/test/java/io/pravega/connectors/presto/util/MockStreamManager.java diff --git a/src/main/java/io/pravega/connectors/presto/schemamanagement/PravegaSchemaRegistry.java b/src/main/java/io/pravega/connectors/presto/schemamanagement/PravegaSchemaRegistry.java index 555d362..fece340 100644 --- a/src/main/java/io/pravega/connectors/presto/schemamanagement/PravegaSchemaRegistry.java +++ b/src/main/java/io/pravega/connectors/presto/schemamanagement/PravegaSchemaRegistry.java @@ -16,6 +16,7 @@ package io.pravega.connectors.presto.schemamanagement; import com.facebook.presto.spi.SchemaTableName; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Streams; import io.pravega.client.ClientConfig; @@ -67,6 +68,13 @@ public PravegaSchemaRegistry(URI controllerURI, URI schemaRegistryURI) this.streamManager = StreamManager.create(ClientConfig.builder().controllerURI(controllerURI).build()); } + @VisibleForTesting + public PravegaSchemaRegistry(SchemaRegistryClient registryClient, StreamManager streamManager) + { + this.registryClient = registryClient; + this.streamManager = streamManager; + } + @Override public List listSchemas() { diff --git a/src/test/java/io/pravega/connectors/presto/schemamanagement/PravegaSchemaRegistryTest.java b/src/test/java/io/pravega/connectors/presto/schemamanagement/PravegaSchemaRegistryTest.java new file mode 100644 index 0000000..79537d8 --- /dev/null +++ b/src/test/java/io/pravega/connectors/presto/schemamanagement/PravegaSchemaRegistryTest.java @@ -0,0 +1,124 @@ +/* + * Copyright (c) Pravega Authors. + * + * 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.pravega.connectors.presto.schemamanagement; + +import com.facebook.presto.common.type.VarcharType; +import com.facebook.presto.spi.SchemaTableName; +import com.google.common.collect.ImmutableMap; +import io.pravega.client.admin.StreamManager; +import io.pravega.connectors.presto.PravegaStreamDescription; +import io.pravega.connectors.presto.PravegaStreamFieldDescription; +import io.pravega.connectors.presto.PravegaStreamFieldGroup; +import io.pravega.connectors.presto.PravegaTableHandle; +import io.pravega.connectors.presto.util.MockSchemaRegistryClient; +import io.pravega.connectors.presto.util.MockStreamManager; +import io.pravega.connectors.presto.util.PravegaNameUtils; +import io.pravega.schemaregistry.client.SchemaRegistryClient; +import io.pravega.schemaregistry.contract.data.Compatibility; +import io.pravega.schemaregistry.contract.data.GroupProperties; +import io.pravega.schemaregistry.contract.data.SerializationFormat; +import io.pravega.schemaregistry.serializer.avro.schemas.AvroSchema; +import org.testng.annotations.Test; + +import java.util.List; + +import static io.pravega.connectors.presto.util.PravegaTestUtils.avroSchema; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; + +@Test +public class PravegaSchemaRegistryTest +{ + private static final String testSchema = + "{\"namespace\": \"io.pravega.avro\",\"type\": \"record\",\"name\": \"Employee\",\"fields\": [{\"name\": \"first\", \"type\": \"string\"},{\"name\": \"last\", \"type\": \"string\"}]}"; + @Test + public void testListSchemas() + { + StreamManager streamManager = new MockStreamManager(); + + streamManager.createScope("schema1"); + streamManager.createScope("schema2"); + + PravegaSchemaRegistry schemaRegistry = new PravegaSchemaRegistry(new MockSchemaRegistryClient(), streamManager); + + List schemas = schemaRegistry.listSchemas(); + assertEquals(schemas.size(), 2); + assertEquals("schema1", schemas.get(0)); + assertEquals("schema2", schemas.get(1)); + } + + @Test + public void testListTables() + { + StreamManager streamManager = new MockStreamManager(); + + streamManager.createStream("schema1", "stream1", null); + streamManager.createStream("schema2", "stream2", null); + // stream starting with '_' is internal/hidden + streamManager.createStream("schema2", "_markStream2", null); + + PravegaSchemaRegistry schemaRegistry = new PravegaSchemaRegistry(new MockSchemaRegistryClient(), streamManager); + List tables = schemaRegistry.listTables("schema2"); + assertEquals(tables.size(), 1); + assertEquals("stream2", tables.get(0).getObjectName()); + } + + @Test + public void testGetTable() + { + SchemaTableName schemaTableName = new SchemaTableName("hr", "employee"); + + SchemaRegistryClient schemaRegistryClient = new MockSchemaRegistryClient(); + schemaRegistryClient.addGroup(groupId(schemaTableName), groupProperties(false)); + schemaRegistryClient.addSchema(groupId(schemaTableName), AvroSchema.of(avroSchema(testSchema)).getSchemaInfo()); + + + PravegaSchemaRegistry schemaRegistry = new PravegaSchemaRegistry(schemaRegistryClient, new MockStreamManager()); + + PravegaStreamDescription table = schemaRegistry.getTable(schemaTableName); + + assertNotNull(table); + assertTrue(table.getEvent().isPresent()); + assertEquals(table.getEvent().get().size(), 1); + + PravegaStreamFieldGroup fieldGroup = table.getEvent().get().get(0); + assertEquals(fieldGroup.getFields().size(), 2); + + PravegaStreamFieldDescription field = fieldGroup.getFields().get(0); + assertEquals(field.getName(), "first"); + assertTrue(field.getType() instanceof VarcharType); + + field = fieldGroup.getFields().get(1); + assertEquals(field.getName(), "last"); + assertTrue(field.getType() instanceof VarcharType); + } + + private static GroupProperties groupProperties(boolean inline) + { + return new GroupProperties( + SerializationFormat.Avro, + Compatibility.allowAny(), + false, + ImmutableMap.builder().put(inline ? "inline" : "", "").build()); + } + + private static String groupId(SchemaTableName schemaTableName) + { + return PravegaNameUtils.groupId(schemaTableName.getSchemaName(), schemaTableName.getTableName()); + } + +} diff --git a/src/test/java/io/pravega/connectors/presto/util/MockSchemaRegistryClient.java b/src/test/java/io/pravega/connectors/presto/util/MockSchemaRegistryClient.java new file mode 100644 index 0000000..2faac1d --- /dev/null +++ b/src/test/java/io/pravega/connectors/presto/util/MockSchemaRegistryClient.java @@ -0,0 +1,181 @@ +/* + * Copyright (c) Pravega Authors. + * + * 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.pravega.connectors.presto.util; + +import io.pravega.schemaregistry.client.SchemaRegistryClient; +import io.pravega.schemaregistry.client.exceptions.RegistryExceptions; +import io.pravega.schemaregistry.contract.data.CodecType; +import io.pravega.schemaregistry.contract.data.Compatibility; +import io.pravega.schemaregistry.contract.data.EncodingId; +import io.pravega.schemaregistry.contract.data.EncodingInfo; +import io.pravega.schemaregistry.contract.data.GroupHistoryRecord; +import io.pravega.schemaregistry.contract.data.GroupProperties; +import io.pravega.schemaregistry.contract.data.SchemaInfo; +import io.pravega.schemaregistry.contract.data.SchemaWithVersion; +import io.pravega.schemaregistry.contract.data.VersionInfo; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + * this class has a very limited use in unit testing, so many methods not implemented + */ +public class MockSchemaRegistryClient + implements SchemaRegistryClient +{ + private final Map groups = new HashMap<>(); + + private final Map> schemas = new HashMap<>(); + + @Override + public boolean addGroup(String s, GroupProperties groupProperties) throws RegistryExceptions.BadArgumentException, RegistryExceptions.UnauthorizedException + { + return groups.putIfAbsent(s, groupProperties) == null; + } + + @Override + public void removeGroup(String s) throws RegistryExceptions.UnauthorizedException + { + groups.remove(s); + } + + @Override + public Iterator> listGroups() throws RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public GroupProperties getGroupProperties(String s) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + return groups.get(s); + } + + @Override + public boolean updateCompatibility(String s, Compatibility compatibility, @Nullable Compatibility compatibility1) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public List getSchemas(String s) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + return schemas.get(s); + } + + @Override + public VersionInfo addSchema(String s, SchemaInfo schemaInfo) throws RegistryExceptions.SchemaValidationFailedException, RegistryExceptions.SerializationMismatchException, RegistryExceptions.MalformedSchemaException, RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + List list = schemas.computeIfAbsent(s, k -> new ArrayList<>()); + VersionInfo versionInfo = new VersionInfo("type", "avro", list.size() + 1, list.size() + 1); + list.add(new SchemaWithVersion(schemaInfo, versionInfo)); + return versionInfo; + } + + @Override + public void deleteSchemaVersion(String s, VersionInfo versionInfo) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public SchemaInfo getSchemaForVersion(String s, VersionInfo versionInfo) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public EncodingInfo getEncodingInfo(String s, EncodingId encodingId) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public EncodingId getEncodingId(String s, VersionInfo versionInfo, String s1) throws RegistryExceptions.CodecTypeNotRegisteredException, RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public SchemaWithVersion getLatestSchemaVersion(String s, @Nullable String s1) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public VersionInfo getVersionForSchema(String s, SchemaInfo schemaInfo) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public List getSchemaVersions(String s, @Nullable String s1) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean validateSchema(String s, SchemaInfo schemaInfo) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean canReadUsing(String s, SchemaInfo schemaInfo) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public List getCodecTypes(String s) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public void addCodecType(String s, CodecType codecType) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public List getGroupHistory(String s) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public Map getSchemaReferences(SchemaInfo schemaInfo) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public String getNamespace() + { + throw new UnsupportedOperationException(); + } + + @Override + public void close() throws Exception + { + + } +} diff --git a/src/test/java/io/pravega/connectors/presto/util/MockStreamManager.java b/src/test/java/io/pravega/connectors/presto/util/MockStreamManager.java new file mode 100644 index 0000000..dae528a --- /dev/null +++ b/src/test/java/io/pravega/connectors/presto/util/MockStreamManager.java @@ -0,0 +1,123 @@ +/* + * Copyright (c) Pravega Authors. + * + * 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.pravega.connectors.presto.util; + +import io.pravega.client.admin.StreamInfo; +import io.pravega.client.admin.StreamManager; +import io.pravega.client.stream.DeleteScopeFailedException; +import io.pravega.client.stream.Stream; +import io.pravega.client.stream.StreamConfiguration; +import io.pravega.client.stream.StreamCut; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +/** + * this class has a very limited use in unit testing, so many methods not implemented + */ +public class MockStreamManager + implements StreamManager { + + private final List scopes = new ArrayList<>(); + + private final List streams = new ArrayList<>(); + + @Override + public boolean createStream(String s, String s1, StreamConfiguration streamConfiguration) + { + Stream stream = Stream.of(s, s1); + return !streams.contains(stream) && streams.add(stream); + } + + @Override + public boolean updateStream(String s, String s1, StreamConfiguration streamConfiguration) + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean truncateStream(String s, String s1, StreamCut streamCut) + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean sealStream(String s, String s1) + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean deleteStream(String s, String s1) + { + throw new UnsupportedOperationException(); + } + + @Override + public Iterator listScopes() + { + return scopes.iterator(); + } + + @Override + public boolean createScope(String s) + { + return !scopes.contains(s) && scopes.add(s); + } + + @Override + public boolean checkScopeExists(String s) + { + return scopes.contains(s); + } + + @Override + public Iterator listStreams(String s) + { + return streams.stream().filter(stream -> stream.getScope().equals(s)).iterator(); + } + + @Override + public boolean checkStreamExists(String s, String s1) + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean deleteScope(String s) + { + return scopes.remove(s); + } + + @Override + public boolean deleteScope(String s, boolean b) throws DeleteScopeFailedException + { + throw new UnsupportedOperationException(); + } + + @Override + public StreamInfo getStreamInfo(String s, String s1) + { + throw new UnsupportedOperationException(); + } + + @Override + public void close() + { + + } +} diff --git a/src/test/java/io/pravega/connectors/presto/util/PravegaTestUtils.java b/src/test/java/io/pravega/connectors/presto/util/PravegaTestUtils.java index ecab9c1..1209a2c 100644 --- a/src/test/java/io/pravega/connectors/presto/util/PravegaTestUtils.java +++ b/src/test/java/io/pravega/connectors/presto/util/PravegaTestUtils.java @@ -133,4 +133,9 @@ public static Schema avroSchema(PravegaStreamDescription streamDescription, int { return new Schema.Parser().parse(streamDescription.getEvent().get().get(event).getDataSchema().get()); } + + public static Schema avroSchema(String avroSchemaString) + { + return new Schema.Parser().parse(avroSchemaString); + } } From f9f31c940c54b5bf4c56c3c7eb3d29a6d766cf49 Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Fri, 26 Mar 2021 06:28:22 -0700 Subject: [PATCH 20/32] unit test for table supplier, test multi source Signed-off-by: Andrew Robertson --- .../PravegaTableDescriptionSupplierTest.java | 94 ++++++++++++++++ .../PravegaSchemaRegistryTest.java | 6 +- .../presto/util/SchemaRegistryUtil.java | 100 ++++++++++++++++++ .../connectors/presto/util/TestSchemas.java | 10 ++ src/test/resources/ut/ut.multiexplicit.json | 6 ++ src/test/resources/ut/ut.multiregex.json | 5 + 6 files changed, 217 insertions(+), 4 deletions(-) create mode 100644 src/test/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplierTest.java create mode 100644 src/test/java/io/pravega/connectors/presto/util/SchemaRegistryUtil.java create mode 100644 src/test/java/io/pravega/connectors/presto/util/TestSchemas.java create mode 100644 src/test/resources/ut/ut.multiexplicit.json create mode 100644 src/test/resources/ut/ut.multiregex.json diff --git a/src/test/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplierTest.java b/src/test/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplierTest.java new file mode 100644 index 0000000..8b97b92 --- /dev/null +++ b/src/test/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplierTest.java @@ -0,0 +1,94 @@ +/* + * Copyright (c) Pravega Authors. + * + * 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.pravega.connectors.presto; + +import com.facebook.presto.spi.SchemaTableName; +import io.pravega.connectors.presto.util.SchemaRegistryUtil; +import org.testng.annotations.Test; + +import java.util.List; + +import static io.pravega.connectors.presto.util.TestSchemas.EMPLOYEE_AVSC; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; + +@Test +public class PravegaTableDescriptionSupplierTest +{ + private static final String SCHEMA = "ut"; + + @Test + public void testTableDoesNotExist() + { + PravegaTableDescriptionSupplier tableSupplier = + new PravegaTableDescriptionSupplier(new SchemaRegistryUtil().getSchemaRegistry()); + assertNull(tableSupplier.getTable(new SchemaTableName(SCHEMA, "stream1"))); + } + + @Test + public void testMultiSourceStreamRegex() + { + SchemaRegistryUtil schemaRegistryUtil = new SchemaRegistryUtil(); + schemaRegistryUtil.addLocalSchema(SCHEMA); + + PravegaTableDescriptionSupplier tableSupplier = + new PravegaTableDescriptionSupplier(schemaRegistryUtil.getSchemaRegistry()); + + + schemaRegistryUtil.addPravegaSchema(SCHEMA); + schemaRegistryUtil.addPravegaTable(new SchemaTableName(SCHEMA, "stream1"), EMPLOYEE_AVSC); + schemaRegistryUtil.addPravegaTable(new SchemaTableName(SCHEMA, "stream2"), EMPLOYEE_AVSC); + schemaRegistryUtil.addPravegaTable(new SchemaTableName(SCHEMA, "stream3"), EMPLOYEE_AVSC); + + PravegaStreamDescription table = + tableSupplier.getTable(new SchemaTableName(SCHEMA, "multiregex")); + + assertNotNull(table); + assertTrue(table.getObjectArgs().isPresent()); + + List components = table.getObjectArgs().get(); + assertEquals(components.size(), 3); + assertEquals(components.stream().sorted().toArray(), new String[]{"stream1", "stream2", "stream3"}); + } + + @Test + public void testMultiSourceStreamExplicit() + { + // same setup as regex. but multi source def. only has 2 component streams. + SchemaRegistryUtil schemaRegistryUtil = new SchemaRegistryUtil(); + schemaRegistryUtil.addLocalSchema(SCHEMA); + + PravegaTableDescriptionSupplier tableSupplier = + new PravegaTableDescriptionSupplier(schemaRegistryUtil.getSchemaRegistry()); + + schemaRegistryUtil.addPravegaSchema(SCHEMA); + schemaRegistryUtil.addPravegaTable(new SchemaTableName(SCHEMA, "stream1"), EMPLOYEE_AVSC); + schemaRegistryUtil.addPravegaTable(new SchemaTableName(SCHEMA, "stream2"), EMPLOYEE_AVSC); + schemaRegistryUtil.addPravegaTable(new SchemaTableName(SCHEMA, "stream3"), EMPLOYEE_AVSC); + + PravegaStreamDescription table = + tableSupplier.getTable(new SchemaTableName(SCHEMA, "multiexplicit")); + + assertNotNull(table); + assertTrue(table.getObjectArgs().isPresent()); + + List components = table.getObjectArgs().get(); + assertEquals(components.size(), 2); + assertEquals(components.stream().sorted().toArray(), new String[]{"stream1", "stream3"}); + } +} diff --git a/src/test/java/io/pravega/connectors/presto/schemamanagement/PravegaSchemaRegistryTest.java b/src/test/java/io/pravega/connectors/presto/schemamanagement/PravegaSchemaRegistryTest.java index 79537d8..e85bf65 100644 --- a/src/test/java/io/pravega/connectors/presto/schemamanagement/PravegaSchemaRegistryTest.java +++ b/src/test/java/io/pravega/connectors/presto/schemamanagement/PravegaSchemaRegistryTest.java @@ -36,6 +36,7 @@ import java.util.List; import static io.pravega.connectors.presto.util.PravegaTestUtils.avroSchema; +import static io.pravega.connectors.presto.util.TestSchemas.EMPLOYEE_AVSC; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; @@ -43,8 +44,6 @@ @Test public class PravegaSchemaRegistryTest { - private static final String testSchema = - "{\"namespace\": \"io.pravega.avro\",\"type\": \"record\",\"name\": \"Employee\",\"fields\": [{\"name\": \"first\", \"type\": \"string\"},{\"name\": \"last\", \"type\": \"string\"}]}"; @Test public void testListSchemas() { @@ -84,7 +83,7 @@ public void testGetTable() SchemaRegistryClient schemaRegistryClient = new MockSchemaRegistryClient(); schemaRegistryClient.addGroup(groupId(schemaTableName), groupProperties(false)); - schemaRegistryClient.addSchema(groupId(schemaTableName), AvroSchema.of(avroSchema(testSchema)).getSchemaInfo()); + schemaRegistryClient.addSchema(groupId(schemaTableName), AvroSchema.of(avroSchema(EMPLOYEE_AVSC)).getSchemaInfo()); PravegaSchemaRegistry schemaRegistry = new PravegaSchemaRegistry(schemaRegistryClient, new MockStreamManager()); @@ -120,5 +119,4 @@ private static String groupId(SchemaTableName schemaTableName) { return PravegaNameUtils.groupId(schemaTableName.getSchemaName(), schemaTableName.getTableName()); } - } diff --git a/src/test/java/io/pravega/connectors/presto/util/SchemaRegistryUtil.java b/src/test/java/io/pravega/connectors/presto/util/SchemaRegistryUtil.java new file mode 100644 index 0000000..f3cbc45 --- /dev/null +++ b/src/test/java/io/pravega/connectors/presto/util/SchemaRegistryUtil.java @@ -0,0 +1,100 @@ +package io.pravega.connectors.presto.util; + +import com.facebook.presto.spi.SchemaTableName; +import com.google.common.collect.ImmutableMap; +import io.pravega.client.admin.StreamManager; +import io.pravega.connectors.presto.schemamanagement.CompositeSchemaRegistry; +import io.pravega.connectors.presto.schemamanagement.LocalSchemaRegistry; +import io.pravega.connectors.presto.schemamanagement.PravegaSchemaRegistry; +import io.pravega.connectors.presto.schemamanagement.SchemaRegistry; +import io.pravega.connectors.presto.schemamanagement.SchemaSupplier; +import io.pravega.schemaregistry.client.SchemaRegistryClient; +import io.pravega.schemaregistry.contract.data.Compatibility; +import io.pravega.schemaregistry.contract.data.GroupProperties; +import io.pravega.schemaregistry.contract.data.SerializationFormat; +import io.pravega.schemaregistry.serializer.avro.schemas.AvroSchema; + +import java.util.ArrayList; +import java.util.List; + +import static io.pravega.connectors.presto.util.PravegaTestUtils.avroSchema; +import static io.pravega.connectors.presto.util.TestSchemas.EMPLOYEE_AVSC; + +public class SchemaRegistryUtil +{ + private final StreamManager streamManager; + private final SchemaRegistryClient schemaRegistryClient; + private final PravegaSchemaRegistry pravegaSchemaRegistry; + + private final List localSchemaRegistryList; + + public SchemaRegistryUtil() + { + this.streamManager = new MockStreamManager(); + this.schemaRegistryClient = new MockSchemaRegistryClient(); + + this.pravegaSchemaRegistry = new PravegaSchemaRegistry(schemaRegistryClient, streamManager); + + this.localSchemaRegistryList = new ArrayList<>(); + } + + public CompositeSchemaRegistry getSchemaRegistry() + { + List schemaSuppliers = new ArrayList<>(); + List schemaRegistries = new ArrayList<>(); + + localSchemaRegistryList.forEach(lsr -> { + schemaSuppliers.add(lsr); + schemaRegistries.add(lsr); + }); + + schemaSuppliers.add(pravegaSchemaRegistry); + schemaRegistries.add(pravegaSchemaRegistry); + + return new CompositeSchemaRegistry(schemaSuppliers, schemaRegistries); + } + + public void addLocalSchema(String dir) + { + localSchemaRegistryList.add(PravegaTestUtils.localSchemaRegistry(dir)); + } + + public boolean addPravegaSchema(String schema) + { + return streamManager.createScope(schema); + } + + public boolean addPravegaTable(SchemaTableName schemaTableName) + { + return streamManager.createStream(schemaTableName.getSchemaName(), + schemaTableName.getTableName(), + null); + } + + public boolean addPravegaTable(SchemaTableName schemaTableName, String schema) + { + addPravegaTable(schemaTableName); + addAvroSchema(schemaTableName, schema); + return true; + } + + public void addAvroSchema(SchemaTableName schemaTableName, String schema) + { + schemaRegistryClient.addGroup(groupId(schemaTableName), groupProperties(false)); + schemaRegistryClient.addSchema(groupId(schemaTableName), AvroSchema.of(avroSchema(schema)).getSchemaInfo()); + } + + private static GroupProperties groupProperties(boolean inline) + { + return new GroupProperties( + SerializationFormat.Avro, + Compatibility.allowAny(), + false, + ImmutableMap.builder().put(inline ? "inline" : "", "").build()); + } + + private static String groupId(SchemaTableName schemaTableName) + { + return PravegaNameUtils.groupId(schemaTableName.getSchemaName(), schemaTableName.getTableName()); + } +} diff --git a/src/test/java/io/pravega/connectors/presto/util/TestSchemas.java b/src/test/java/io/pravega/connectors/presto/util/TestSchemas.java new file mode 100644 index 0000000..29a316f --- /dev/null +++ b/src/test/java/io/pravega/connectors/presto/util/TestSchemas.java @@ -0,0 +1,10 @@ +package io.pravega.connectors.presto.util; + +public class TestSchemas +{ + + private TestSchemas() {} + + public static final String EMPLOYEE_AVSC = + "{\"namespace\": \"io.pravega.avro\",\"type\": \"record\",\"name\": \"Employee\",\"fields\": [{\"name\": \"first\", \"type\": \"string\"},{\"name\": \"last\", \"type\": \"string\"}]}"; +} diff --git a/src/test/resources/ut/ut.multiexplicit.json b/src/test/resources/ut/ut.multiexplicit.json new file mode 100644 index 0000000..bf6b50c --- /dev/null +++ b/src/test/resources/ut/ut.multiexplicit.json @@ -0,0 +1,6 @@ +{ + "schemaName": "ut", + "tableName": "multiexplicit", + "objectName": "multiexplicit", + "objectArgs": ["stream1", "stream3"] +} \ No newline at end of file diff --git a/src/test/resources/ut/ut.multiregex.json b/src/test/resources/ut/ut.multiregex.json new file mode 100644 index 0000000..911ee32 --- /dev/null +++ b/src/test/resources/ut/ut.multiregex.json @@ -0,0 +1,5 @@ +{ + "schemaName": "ut", + "tableName": "multiregex", + "objectName": "stream[0-9]" +} \ No newline at end of file From 0184eaba198a0c5a065efe10709fa0a9dd2e6a26 Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Fri, 26 Mar 2021 06:52:39 -0700 Subject: [PATCH 21/32] use new test util in UT Signed-off-by: Andrew Robertson --- .../PravegaTableDescriptionSupplierTest.java | 16 +++--- .../LocalSchemaRegistryTest.java | 2 +- .../PravegaSchemaRegistryTest.java | 57 +++++-------------- .../presto/util/MockSchemaRegistryClient.java | 6 +- .../presto/util/SchemaRegistryUtil.java | 34 +++++++++-- .../connectors/presto/util/TestSchemas.java | 16 +++++- 6 files changed, 72 insertions(+), 59 deletions(-) diff --git a/src/test/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplierTest.java b/src/test/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplierTest.java index 8b97b92..2c869d1 100644 --- a/src/test/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplierTest.java +++ b/src/test/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplierTest.java @@ -50,10 +50,10 @@ public void testMultiSourceStreamRegex() new PravegaTableDescriptionSupplier(schemaRegistryUtil.getSchemaRegistry()); - schemaRegistryUtil.addPravegaSchema(SCHEMA); - schemaRegistryUtil.addPravegaTable(new SchemaTableName(SCHEMA, "stream1"), EMPLOYEE_AVSC); - schemaRegistryUtil.addPravegaTable(new SchemaTableName(SCHEMA, "stream2"), EMPLOYEE_AVSC); - schemaRegistryUtil.addPravegaTable(new SchemaTableName(SCHEMA, "stream3"), EMPLOYEE_AVSC); + schemaRegistryUtil.addSchema(SCHEMA); + schemaRegistryUtil.addTable(new SchemaTableName(SCHEMA, "stream1"), EMPLOYEE_AVSC); + schemaRegistryUtil.addTable(new SchemaTableName(SCHEMA, "stream2"), EMPLOYEE_AVSC); + schemaRegistryUtil.addTable(new SchemaTableName(SCHEMA, "stream3"), EMPLOYEE_AVSC); PravegaStreamDescription table = tableSupplier.getTable(new SchemaTableName(SCHEMA, "multiregex")); @@ -76,10 +76,10 @@ public void testMultiSourceStreamExplicit() PravegaTableDescriptionSupplier tableSupplier = new PravegaTableDescriptionSupplier(schemaRegistryUtil.getSchemaRegistry()); - schemaRegistryUtil.addPravegaSchema(SCHEMA); - schemaRegistryUtil.addPravegaTable(new SchemaTableName(SCHEMA, "stream1"), EMPLOYEE_AVSC); - schemaRegistryUtil.addPravegaTable(new SchemaTableName(SCHEMA, "stream2"), EMPLOYEE_AVSC); - schemaRegistryUtil.addPravegaTable(new SchemaTableName(SCHEMA, "stream3"), EMPLOYEE_AVSC); + schemaRegistryUtil.addSchema(SCHEMA); + schemaRegistryUtil.addTable(new SchemaTableName(SCHEMA, "stream1"), EMPLOYEE_AVSC); + schemaRegistryUtil.addTable(new SchemaTableName(SCHEMA, "stream2"), EMPLOYEE_AVSC); + schemaRegistryUtil.addTable(new SchemaTableName(SCHEMA, "stream3"), EMPLOYEE_AVSC); PravegaStreamDescription table = tableSupplier.getTable(new SchemaTableName(SCHEMA, "multiexplicit")); diff --git a/src/test/java/io/pravega/connectors/presto/schemamanagement/LocalSchemaRegistryTest.java b/src/test/java/io/pravega/connectors/presto/schemamanagement/LocalSchemaRegistryTest.java index 724b66e..9580c29 100644 --- a/src/test/java/io/pravega/connectors/presto/schemamanagement/LocalSchemaRegistryTest.java +++ b/src/test/java/io/pravega/connectors/presto/schemamanagement/LocalSchemaRegistryTest.java @@ -44,7 +44,7 @@ public void testListSchemas() List schemas = schemaRegistry.listSchemas(); assertEquals(schemas.size(), 1); - assertEquals(schemas.get(1), "tpch"); + assertEquals(schemas.get(0), "tpch"); } @Test diff --git a/src/test/java/io/pravega/connectors/presto/schemamanagement/PravegaSchemaRegistryTest.java b/src/test/java/io/pravega/connectors/presto/schemamanagement/PravegaSchemaRegistryTest.java index e85bf65..f5ea023 100644 --- a/src/test/java/io/pravega/connectors/presto/schemamanagement/PravegaSchemaRegistryTest.java +++ b/src/test/java/io/pravega/connectors/presto/schemamanagement/PravegaSchemaRegistryTest.java @@ -17,25 +17,15 @@ import com.facebook.presto.common.type.VarcharType; import com.facebook.presto.spi.SchemaTableName; -import com.google.common.collect.ImmutableMap; -import io.pravega.client.admin.StreamManager; import io.pravega.connectors.presto.PravegaStreamDescription; import io.pravega.connectors.presto.PravegaStreamFieldDescription; import io.pravega.connectors.presto.PravegaStreamFieldGroup; import io.pravega.connectors.presto.PravegaTableHandle; -import io.pravega.connectors.presto.util.MockSchemaRegistryClient; -import io.pravega.connectors.presto.util.MockStreamManager; -import io.pravega.connectors.presto.util.PravegaNameUtils; -import io.pravega.schemaregistry.client.SchemaRegistryClient; -import io.pravega.schemaregistry.contract.data.Compatibility; -import io.pravega.schemaregistry.contract.data.GroupProperties; -import io.pravega.schemaregistry.contract.data.SerializationFormat; -import io.pravega.schemaregistry.serializer.avro.schemas.AvroSchema; +import io.pravega.connectors.presto.util.SchemaRegistryUtil; import org.testng.annotations.Test; import java.util.List; -import static io.pravega.connectors.presto.util.PravegaTestUtils.avroSchema; import static io.pravega.connectors.presto.util.TestSchemas.EMPLOYEE_AVSC; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; @@ -47,14 +37,12 @@ public class PravegaSchemaRegistryTest @Test public void testListSchemas() { - StreamManager streamManager = new MockStreamManager(); + SchemaRegistryUtil schemaRegistryUtil = new SchemaRegistryUtil(); - streamManager.createScope("schema1"); - streamManager.createScope("schema2"); + schemaRegistryUtil.addSchema("schema1"); + schemaRegistryUtil.addSchema("schema2"); - PravegaSchemaRegistry schemaRegistry = new PravegaSchemaRegistry(new MockSchemaRegistryClient(), streamManager); - - List schemas = schemaRegistry.listSchemas(); + List schemas = schemaRegistryUtil.getSchemaRegistry().listSchemas(); assertEquals(schemas.size(), 2); assertEquals("schema1", schemas.get(0)); assertEquals("schema2", schemas.get(1)); @@ -63,15 +51,14 @@ public void testListSchemas() @Test public void testListTables() { - StreamManager streamManager = new MockStreamManager(); + SchemaRegistryUtil schemaRegistryUtil = new SchemaRegistryUtil(); - streamManager.createStream("schema1", "stream1", null); - streamManager.createStream("schema2", "stream2", null); + schemaRegistryUtil.addTable("schema1", "stream1"); + schemaRegistryUtil.addTable("schema2", "stream2"); // stream starting with '_' is internal/hidden - streamManager.createStream("schema2", "_markStream2", null); + schemaRegistryUtil.addTable("schema2", "_markStream2"); - PravegaSchemaRegistry schemaRegistry = new PravegaSchemaRegistry(new MockSchemaRegistryClient(), streamManager); - List tables = schemaRegistry.listTables("schema2"); + List tables = schemaRegistryUtil.getSchemaRegistry().listTables("schema2"); assertEquals(tables.size(), 1); assertEquals("stream2", tables.get(0).getObjectName()); } @@ -79,14 +66,12 @@ public void testListTables() @Test public void testGetTable() { - SchemaTableName schemaTableName = new SchemaTableName("hr", "employee"); - - SchemaRegistryClient schemaRegistryClient = new MockSchemaRegistryClient(); - schemaRegistryClient.addGroup(groupId(schemaTableName), groupProperties(false)); - schemaRegistryClient.addSchema(groupId(schemaTableName), AvroSchema.of(avroSchema(EMPLOYEE_AVSC)).getSchemaInfo()); + SchemaRegistryUtil schemaRegistryUtil = new SchemaRegistryUtil(); + SchemaTableName schemaTableName = new SchemaTableName("hr", "employee"); + schemaRegistryUtil.addAvroSchema(schemaTableName, EMPLOYEE_AVSC); - PravegaSchemaRegistry schemaRegistry = new PravegaSchemaRegistry(schemaRegistryClient, new MockStreamManager()); + SchemaRegistry schemaRegistry = schemaRegistryUtil.getSchemaRegistry(); PravegaStreamDescription table = schemaRegistry.getTable(schemaTableName); @@ -105,18 +90,4 @@ public void testGetTable() assertEquals(field.getName(), "last"); assertTrue(field.getType() instanceof VarcharType); } - - private static GroupProperties groupProperties(boolean inline) - { - return new GroupProperties( - SerializationFormat.Avro, - Compatibility.allowAny(), - false, - ImmutableMap.builder().put(inline ? "inline" : "", "").build()); - } - - private static String groupId(SchemaTableName schemaTableName) - { - return PravegaNameUtils.groupId(schemaTableName.getSchemaName(), schemaTableName.getTableName()); - } } diff --git a/src/test/java/io/pravega/connectors/presto/util/MockSchemaRegistryClient.java b/src/test/java/io/pravega/connectors/presto/util/MockSchemaRegistryClient.java index 2faac1d..9a47ce8 100644 --- a/src/test/java/io/pravega/connectors/presto/util/MockSchemaRegistryClient.java +++ b/src/test/java/io/pravega/connectors/presto/util/MockSchemaRegistryClient.java @@ -65,7 +65,11 @@ public Iterator> listGroups() throws Registry @Override public GroupProperties getGroupProperties(String s) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException { - return groups.get(s); + GroupProperties groupProperties = groups.get(s); + if (groupProperties == null) { + throw new RegistryExceptions.ResourceNotFoundException(s); + } + return groupProperties; } @Override diff --git a/src/test/java/io/pravega/connectors/presto/util/SchemaRegistryUtil.java b/src/test/java/io/pravega/connectors/presto/util/SchemaRegistryUtil.java index f3cbc45..b4882c1 100644 --- a/src/test/java/io/pravega/connectors/presto/util/SchemaRegistryUtil.java +++ b/src/test/java/io/pravega/connectors/presto/util/SchemaRegistryUtil.java @@ -1,3 +1,18 @@ +/* + * Copyright (c) Pravega Authors. + * + * 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.pravega.connectors.presto.util; import com.facebook.presto.spi.SchemaTableName; @@ -18,8 +33,10 @@ import java.util.List; import static io.pravega.connectors.presto.util.PravegaTestUtils.avroSchema; -import static io.pravega.connectors.presto.util.TestSchemas.EMPLOYEE_AVSC; +/** + * build CompositeSchemaRegistry for use in unit tests + */ public class SchemaRegistryUtil { private final StreamManager streamManager; @@ -59,21 +76,28 @@ public void addLocalSchema(String dir) localSchemaRegistryList.add(PravegaTestUtils.localSchemaRegistry(dir)); } - public boolean addPravegaSchema(String schema) + public boolean addSchema(String schema) { return streamManager.createScope(schema); } - public boolean addPravegaTable(SchemaTableName schemaTableName) + public boolean addTable(SchemaTableName schemaTableName) { return streamManager.createStream(schemaTableName.getSchemaName(), schemaTableName.getTableName(), null); } - public boolean addPravegaTable(SchemaTableName schemaTableName, String schema) + public boolean addTable(String schema, String stream) { - addPravegaTable(schemaTableName); + return addTable(new SchemaTableName(schema, stream)); + } + + public boolean addTable(SchemaTableName schemaTableName, String schema) + { + if (!addTable(schemaTableName)) { + return false; + } addAvroSchema(schemaTableName, schema); return true; } diff --git a/src/test/java/io/pravega/connectors/presto/util/TestSchemas.java b/src/test/java/io/pravega/connectors/presto/util/TestSchemas.java index 29a316f..116cbcd 100644 --- a/src/test/java/io/pravega/connectors/presto/util/TestSchemas.java +++ b/src/test/java/io/pravega/connectors/presto/util/TestSchemas.java @@ -1,8 +1,22 @@ +/* + * Copyright (c) Pravega Authors. + * + * 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.pravega.connectors.presto.util; public class TestSchemas { - private TestSchemas() {} public static final String EMPLOYEE_AVSC = From 067a3e8471bbd446ff6dcd4f6394a9fd1af5a475 Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Fri, 26 Mar 2021 08:21:46 -0700 Subject: [PATCH 22/32] fix issue in integration test. now that we are listing vs. seeding cache for test. Signed-off-by: Andrew Robertson --- .../presto/PravegaTableDescriptionSupplier.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java b/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java index f18d4d3..e1dd32a 100644 --- a/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java +++ b/src/main/java/io/pravega/connectors/presto/PravegaTableDescriptionSupplier.java @@ -28,6 +28,7 @@ import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Optional; import java.util.concurrent.TimeUnit; @@ -59,6 +60,9 @@ public class PravegaTableDescriptionSupplier private Cache> tableCache; + // whether we have listed tables from this schema or not + private final HashMap tableListMap = new HashMap<>(); + private JsonCodec streamDescriptionCodec; @Inject @@ -71,7 +75,7 @@ public class PravegaTableDescriptionSupplier // there will be many successive calls to listSchemas + listTables in short time period // do not reach out to pravega each time as it is unlikely things would have changed // enhancement issue - can we determine if there are changes/removals and selectively update? - // https://github.com/StreamingDataPlatform/pravega-sql/issues/101 + // https://github.com/pravega/presto-connector/issues/30 this.schemaCache = CacheBuilder.newBuilder() .expireAfterWrite(pravegaConnectorConfig.getTableCacheExpireSecs(), TimeUnit.SECONDS) .build(); @@ -121,7 +125,7 @@ public List listTables(Optional schema) .filter(streamDesc -> streamDesc.getSchemaTableName().getSchemaName().startsWith(s)) .collect(Collectors.toList()); - if (tableListForSchema.isEmpty()) { + if (tableListForSchema.isEmpty() || tableListMap.get(s) == null) { List compositeStreams = new ArrayList<>(); @@ -151,6 +155,7 @@ public List listTables(Optional schema) } } }); + tableListMap.put(s, true); // we have now listed tables from the schema } else { log.debug("serving listTables(%s) from cache", s); From e78ccc70ddc885069cc1b61af56ce10c8acf6e28 Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Fri, 26 Mar 2021 08:22:45 -0700 Subject: [PATCH 23/32] restore formatting Signed-off-by: Andrew Robertson --- .../presto/PravegaConnectorConfig.java | 34 ++++++++++++------- 1 file changed, 22 insertions(+), 12 deletions(-) diff --git a/src/main/java/io/pravega/connectors/presto/PravegaConnectorConfig.java b/src/main/java/io/pravega/connectors/presto/PravegaConnectorConfig.java index 8ce0a97..e4b086c 100644 --- a/src/main/java/io/pravega/connectors/presto/PravegaConnectorConfig.java +++ b/src/main/java/io/pravega/connectors/presto/PravegaConnectorConfig.java @@ -23,7 +23,8 @@ import java.io.File; import java.net.URI; -public class PravegaConnectorConfig { +public class PravegaConnectorConfig +{ /** * Pravega Controller URI */ @@ -53,55 +54,64 @@ public class PravegaConnectorConfig { private URI confluentSchemaRegistry; @NotNull - public URI getControllerURI() { + public URI getControllerURI() + { return this.controllerURI; } @NotNull - public URI getSchemaRegistryURI() { + public URI getSchemaRegistryURI() + { return this.schemaRegistryURI; } @NotNull - public File getTableDescriptionDir() { + public File getTableDescriptionDir() + { return tableDescriptionDir; } @Config("pravega.table-description-dir") - public PravegaConnectorConfig setTableDescriptionDir(File tableDescriptionDir) { + public PravegaConnectorConfig setTableDescriptionDir(File tableDescriptionDir) + { this.tableDescriptionDir = tableDescriptionDir; return this; } - public int getTableCacheExpireSecs() { + public int getTableCacheExpireSecs() + { return this.tableCacheExpireSecs; } @Config("pravega.controller") - public PravegaConnectorConfig setControllerURI(URI controllerURI) { + public PravegaConnectorConfig setControllerURI(URI controllerURI) + { this.controllerURI = controllerURI; return this; } @Config("pravega.schema-registry") - public PravegaConnectorConfig setSchemaRegistryURI(URI schemaRegistryURI) { + public PravegaConnectorConfig setSchemaRegistryURI(URI schemaRegistryURI) + { this.schemaRegistryURI = schemaRegistryURI; return this; } - public boolean isHideInternalColumns() { + public boolean isHideInternalColumns() + { return hideInternalColumns; } @Config("pravega.hide-internal-columns") - public PravegaConnectorConfig setHideInternalColumns(boolean hideInternalColumns) { + public PravegaConnectorConfig setHideInternalColumns(boolean hideInternalColumns) + { this.hideInternalColumns = hideInternalColumns; return this; } - @Config("pravega.confluentSchemaRegistry") - public PravegaConnectorConfig setConfluentSchemaRegistry(URI confluentSchemaRegistry) { + public PravegaConnectorConfig setConfluentSchemaRegistry(URI confluentSchemaRegistry) + { this.confluentSchemaRegistry = confluentSchemaRegistry; return this; } From 6905745fa9d6c4be6bccd1edf1e8c49211b8e9bf Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Mon, 29 Mar 2021 08:05:19 -0700 Subject: [PATCH 24/32] add maven location for confluent sr lib Signed-off-by: Andrew Robertson --- build.gradle | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/build.gradle b/build.gradle index 7891773..6a0657f 100644 --- a/build.gradle +++ b/build.gradle @@ -24,6 +24,10 @@ repositories { maven { url = uri('https://repo.maven.apache.org/maven2') } + + maven { + url = uri('https://packages.confluent.io/maven/') + } } dependencies { From 5bbb17ecb4e4f5e8582be898ce372e632c415d58 Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Wed, 23 Jun 2021 07:34:02 -0700 Subject: [PATCH 25/32] remove confluent Signed-off-by: Andrew Robertson --- build.gradle | 6 -- gradle.properties | 1 - .../presto/PravegaConnectorConfig.java | 14 --- .../CompositeSchemaRegistry.java | 6 -- .../ConfluentSchemaRegistry.java | 99 ------------------- 5 files changed, 126 deletions(-) delete mode 100644 src/main/java/io/pravega/connectors/presto/schemamanagement/ConfluentSchemaRegistry.java diff --git a/build.gradle b/build.gradle index 9a26e81..bc15c5c 100644 --- a/build.gradle +++ b/build.gradle @@ -24,10 +24,6 @@ repositories { maven { url = uri('https://repo.maven.apache.org/maven2') } - - maven { - url = uri('https://packages.confluent.io/maven/') - } } dependencies { @@ -59,8 +55,6 @@ dependencies { compile "com.facebook.presto:presto-record-decoder:${prestoVersion}" compile "com.facebook.presto:presto-spi:${prestoVersion}" compile "com.facebook.presto:presto-common:${prestoVersion}" - - compile "io.confluent:kafka-schema-registry-client:${confluentVersion}" compile group: 'io.netty', name: 'netty-all', version:"{nettyVersion}" diff --git a/gradle.properties b/gradle.properties index cdb197a..4db7ff3 100644 --- a/gradle.properties +++ b/gradle.properties @@ -18,7 +18,6 @@ airliftTestingVersion=0.191 airliftUnitsVersion=1.3 avroVersion=1.8.1 commonsVersion=3.7 -confluentVersion=6.1.0 checkstyleToolVersion=8.23 everitJsonSchemaVersion=1.12.1 guavaVersion=26.0-jre diff --git a/src/main/java/io/pravega/connectors/presto/PravegaConnectorConfig.java b/src/main/java/io/pravega/connectors/presto/PravegaConnectorConfig.java index e4b086c..c938f5a 100644 --- a/src/main/java/io/pravega/connectors/presto/PravegaConnectorConfig.java +++ b/src/main/java/io/pravega/connectors/presto/PravegaConnectorConfig.java @@ -51,8 +51,6 @@ public class PravegaConnectorConfig private File tableDescriptionDir = new File("etc/pravega/"); - private URI confluentSchemaRegistry; - @NotNull public URI getControllerURI() { @@ -108,16 +106,4 @@ public PravegaConnectorConfig setHideInternalColumns(boolean hideInternalColumns this.hideInternalColumns = hideInternalColumns; return this; } - - @Config("pravega.confluentSchemaRegistry") - public PravegaConnectorConfig setConfluentSchemaRegistry(URI confluentSchemaRegistry) - { - this.confluentSchemaRegistry = confluentSchemaRegistry; - return this; - } - - public URI getConfluentSchemaRegistry() - { - return confluentSchemaRegistry; - } } diff --git a/src/main/java/io/pravega/connectors/presto/schemamanagement/CompositeSchemaRegistry.java b/src/main/java/io/pravega/connectors/presto/schemamanagement/CompositeSchemaRegistry.java index 82322c4..d4cd188 100644 --- a/src/main/java/io/pravega/connectors/presto/schemamanagement/CompositeSchemaRegistry.java +++ b/src/main/java/io/pravega/connectors/presto/schemamanagement/CompositeSchemaRegistry.java @@ -52,12 +52,6 @@ public CompositeSchemaRegistry(PravegaConnectorConfig config, JsonCodec getSchema(SchemaTableName schemaTableName) - { - try { - SchemaMetadata metadata = schemaRegistryClient.getLatestSchemaMetadata(format(schemaTableName)); - if (!metadata.getSchemaType().equalsIgnoreCase(AVRO)) { - throw new UnsupportedOperationException("schema type '" + metadata.getSchemaType() + "' is not supported"); - } - - List fields = - mapFieldsFromSchema("", AVRO, metadata.getSchema()); - - return Collections.singletonList( - new PravegaStreamFieldGroup(AVRO, - Optional.empty(), - Optional.of(metadata.getSchema()), - Optional.of(fields))); - } - catch (IOException e) { - throw new UncheckedIOException(e); - } - catch (RestClientException e) { - if (e.getStatus() == 404) { - return null; - } - throw new RuntimeException(e); - } - } - - @Override - public PravegaStreamDescription getTable(SchemaTableName schemaTableName) - { - List schema = getSchema(schemaTableName); - if (schema == null) { - return null; - } - - return new PravegaStreamDescription( - schemaTableName.getTableName(), - Optional.of(schemaTableName.getSchemaName()), - temp_tableNameToStreamName(schemaTableName.getTableName()), - Optional.of(ObjectType.STREAM), - Optional.empty() /* args */, - Optional.of(schema)); - } - - static String format(SchemaTableName schemaTableName) - { - return String.format("%s-%s", schemaTableName.getSchemaName(), schemaTableName.getTableName()); - } -} \ No newline at end of file From c4cd4bfd8a1e8b5d776a3d76af41dd4f7b03b8ee Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Fri, 10 Sep 2021 11:13:18 -0700 Subject: [PATCH 26/32] remove spurious changes Signed-off-by: Andrew Robertson --- build.gradle | 2 +- .../io/pravega/connectors/presto/PravegaConnectorConfig.java | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/build.gradle b/build.gradle index bc15c5c..bd33990 100644 --- a/build.gradle +++ b/build.gradle @@ -55,7 +55,7 @@ dependencies { compile "com.facebook.presto:presto-record-decoder:${prestoVersion}" compile "com.facebook.presto:presto-spi:${prestoVersion}" compile "com.facebook.presto:presto-common:${prestoVersion}" - + compile group: 'io.netty', name: 'netty-all', version:"{nettyVersion}" runtimeOnly "io.airlift:joda-to-java-time-bridge:3" diff --git a/src/main/java/io/pravega/connectors/presto/PravegaConnectorConfig.java b/src/main/java/io/pravega/connectors/presto/PravegaConnectorConfig.java index c938f5a..e209adc 100644 --- a/src/main/java/io/pravega/connectors/presto/PravegaConnectorConfig.java +++ b/src/main/java/io/pravega/connectors/presto/PravegaConnectorConfig.java @@ -50,7 +50,6 @@ public class PravegaConnectorConfig */ private File tableDescriptionDir = new File("etc/pravega/"); - @NotNull public URI getControllerURI() { From acd5cf4b53aea6d0b5d96d26853edfaf642de280 Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Mon, 27 Sep 2021 11:04:14 -0700 Subject: [PATCH 27/32] rename resource files to includce schema Signed-off-by: Andrew Robertson --- .../src/test/resources/kv/{employee.json => kv.employee.json} | 0 .../src/test/resources/tpch/{customer.json => tpch.customer.json} | 0 .../src/test/resources/tpch/{lineitem.json => tpch.lineitem.json} | 0 .../src/test/resources/tpch/{nation.json => tpch.nation.json} | 0 .../src/test/resources/tpch/{orders.json => tpch.orders.json} | 0 prestodb/src/test/resources/tpch/{part.json => tpch.part.json} | 0 .../src/test/resources/tpch/{partsupp.json => tpch.partsupp.json} | 0 .../src/test/resources/tpch/{region.json => tpch.region.json} | 0 .../src/test/resources/tpch/{supplier.json => tpch.supplier.json} | 0 trino/src/test/resources/kv/{employee.json => kv.employee.json} | 0 .../src/test/resources/tpch/{customer.json => tpch.customer.json} | 0 .../src/test/resources/tpch/{lineitem.json => tpch.lineitem.json} | 0 trino/src/test/resources/tpch/{nation.json => tpch.nation.json} | 0 trino/src/test/resources/tpch/{orders.json => tpch.orders.json} | 0 trino/src/test/resources/tpch/{part.json => tpch.part.json} | 0 .../src/test/resources/tpch/{partsupp.json => tpch.partsupp.json} | 0 trino/src/test/resources/tpch/{region.json => tpch.region.json} | 0 .../src/test/resources/tpch/{supplier.json => tpch.supplier.json} | 0 18 files changed, 0 insertions(+), 0 deletions(-) rename prestodb/src/test/resources/kv/{employee.json => kv.employee.json} (100%) rename prestodb/src/test/resources/tpch/{customer.json => tpch.customer.json} (100%) rename prestodb/src/test/resources/tpch/{lineitem.json => tpch.lineitem.json} (100%) rename prestodb/src/test/resources/tpch/{nation.json => tpch.nation.json} (100%) rename prestodb/src/test/resources/tpch/{orders.json => tpch.orders.json} (100%) rename prestodb/src/test/resources/tpch/{part.json => tpch.part.json} (100%) rename prestodb/src/test/resources/tpch/{partsupp.json => tpch.partsupp.json} (100%) rename prestodb/src/test/resources/tpch/{region.json => tpch.region.json} (100%) rename prestodb/src/test/resources/tpch/{supplier.json => tpch.supplier.json} (100%) rename trino/src/test/resources/kv/{employee.json => kv.employee.json} (100%) rename trino/src/test/resources/tpch/{customer.json => tpch.customer.json} (100%) rename trino/src/test/resources/tpch/{lineitem.json => tpch.lineitem.json} (100%) rename trino/src/test/resources/tpch/{nation.json => tpch.nation.json} (100%) rename trino/src/test/resources/tpch/{orders.json => tpch.orders.json} (100%) rename trino/src/test/resources/tpch/{part.json => tpch.part.json} (100%) rename trino/src/test/resources/tpch/{partsupp.json => tpch.partsupp.json} (100%) rename trino/src/test/resources/tpch/{region.json => tpch.region.json} (100%) rename trino/src/test/resources/tpch/{supplier.json => tpch.supplier.json} (100%) diff --git a/prestodb/src/test/resources/kv/employee.json b/prestodb/src/test/resources/kv/kv.employee.json similarity index 100% rename from prestodb/src/test/resources/kv/employee.json rename to prestodb/src/test/resources/kv/kv.employee.json diff --git a/prestodb/src/test/resources/tpch/customer.json b/prestodb/src/test/resources/tpch/tpch.customer.json similarity index 100% rename from prestodb/src/test/resources/tpch/customer.json rename to prestodb/src/test/resources/tpch/tpch.customer.json diff --git a/prestodb/src/test/resources/tpch/lineitem.json b/prestodb/src/test/resources/tpch/tpch.lineitem.json similarity index 100% rename from prestodb/src/test/resources/tpch/lineitem.json rename to prestodb/src/test/resources/tpch/tpch.lineitem.json diff --git a/prestodb/src/test/resources/tpch/nation.json b/prestodb/src/test/resources/tpch/tpch.nation.json similarity index 100% rename from prestodb/src/test/resources/tpch/nation.json rename to prestodb/src/test/resources/tpch/tpch.nation.json diff --git a/prestodb/src/test/resources/tpch/orders.json b/prestodb/src/test/resources/tpch/tpch.orders.json similarity index 100% rename from prestodb/src/test/resources/tpch/orders.json rename to prestodb/src/test/resources/tpch/tpch.orders.json diff --git a/prestodb/src/test/resources/tpch/part.json b/prestodb/src/test/resources/tpch/tpch.part.json similarity index 100% rename from prestodb/src/test/resources/tpch/part.json rename to prestodb/src/test/resources/tpch/tpch.part.json diff --git a/prestodb/src/test/resources/tpch/partsupp.json b/prestodb/src/test/resources/tpch/tpch.partsupp.json similarity index 100% rename from prestodb/src/test/resources/tpch/partsupp.json rename to prestodb/src/test/resources/tpch/tpch.partsupp.json diff --git a/prestodb/src/test/resources/tpch/region.json b/prestodb/src/test/resources/tpch/tpch.region.json similarity index 100% rename from prestodb/src/test/resources/tpch/region.json rename to prestodb/src/test/resources/tpch/tpch.region.json diff --git a/prestodb/src/test/resources/tpch/supplier.json b/prestodb/src/test/resources/tpch/tpch.supplier.json similarity index 100% rename from prestodb/src/test/resources/tpch/supplier.json rename to prestodb/src/test/resources/tpch/tpch.supplier.json diff --git a/trino/src/test/resources/kv/employee.json b/trino/src/test/resources/kv/kv.employee.json similarity index 100% rename from trino/src/test/resources/kv/employee.json rename to trino/src/test/resources/kv/kv.employee.json diff --git a/trino/src/test/resources/tpch/customer.json b/trino/src/test/resources/tpch/tpch.customer.json similarity index 100% rename from trino/src/test/resources/tpch/customer.json rename to trino/src/test/resources/tpch/tpch.customer.json diff --git a/trino/src/test/resources/tpch/lineitem.json b/trino/src/test/resources/tpch/tpch.lineitem.json similarity index 100% rename from trino/src/test/resources/tpch/lineitem.json rename to trino/src/test/resources/tpch/tpch.lineitem.json diff --git a/trino/src/test/resources/tpch/nation.json b/trino/src/test/resources/tpch/tpch.nation.json similarity index 100% rename from trino/src/test/resources/tpch/nation.json rename to trino/src/test/resources/tpch/tpch.nation.json diff --git a/trino/src/test/resources/tpch/orders.json b/trino/src/test/resources/tpch/tpch.orders.json similarity index 100% rename from trino/src/test/resources/tpch/orders.json rename to trino/src/test/resources/tpch/tpch.orders.json diff --git a/trino/src/test/resources/tpch/part.json b/trino/src/test/resources/tpch/tpch.part.json similarity index 100% rename from trino/src/test/resources/tpch/part.json rename to trino/src/test/resources/tpch/tpch.part.json diff --git a/trino/src/test/resources/tpch/partsupp.json b/trino/src/test/resources/tpch/tpch.partsupp.json similarity index 100% rename from trino/src/test/resources/tpch/partsupp.json rename to trino/src/test/resources/tpch/tpch.partsupp.json diff --git a/trino/src/test/resources/tpch/region.json b/trino/src/test/resources/tpch/tpch.region.json similarity index 100% rename from trino/src/test/resources/tpch/region.json rename to trino/src/test/resources/tpch/tpch.region.json diff --git a/trino/src/test/resources/tpch/supplier.json b/trino/src/test/resources/tpch/tpch.supplier.json similarity index 100% rename from trino/src/test/resources/tpch/supplier.json rename to trino/src/test/resources/tpch/tpch.supplier.json From 53330266de704c7f8425356ad31893599dbe532c Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Mon, 27 Sep 2021 11:08:24 -0700 Subject: [PATCH 28/32] relocate test util files Signed-off-by: Andrew Robertson --- .../connectors/presto/{integration => util}/CodecSupplier.java | 0 .../connectors/presto/{integration => util}/PravegaTestUtils.java | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename prestodb/src/test/java/io/pravega/connectors/presto/{integration => util}/CodecSupplier.java (100%) rename prestodb/src/test/java/io/pravega/connectors/presto/{integration => util}/PravegaTestUtils.java (100%) diff --git a/prestodb/src/test/java/io/pravega/connectors/presto/integration/CodecSupplier.java b/prestodb/src/test/java/io/pravega/connectors/presto/util/CodecSupplier.java similarity index 100% rename from prestodb/src/test/java/io/pravega/connectors/presto/integration/CodecSupplier.java rename to prestodb/src/test/java/io/pravega/connectors/presto/util/CodecSupplier.java diff --git a/prestodb/src/test/java/io/pravega/connectors/presto/integration/PravegaTestUtils.java b/prestodb/src/test/java/io/pravega/connectors/presto/util/PravegaTestUtils.java similarity index 100% rename from prestodb/src/test/java/io/pravega/connectors/presto/integration/PravegaTestUtils.java rename to prestodb/src/test/java/io/pravega/connectors/presto/util/PravegaTestUtils.java From 7a9fc481102932c1f8ef0b9f0e25c043ec616445 Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Mon, 27 Sep 2021 11:54:34 -0700 Subject: [PATCH 29/32] replicate changes to trino Signed-off-by: Andrew Robertson --- .../presto/util/PravegaSchemaUtils.java | 2 +- .../plugin/pravega/PravegaSplitManager.java | 8 +- .../pravega/PravegaStreamDescription.java | 11 + .../pravega/PravegaStreamFieldGroup.java | 2 +- .../PravegaTableDescriptionSupplier.java | 451 ++++-------------- .../plugin/pravega/PravegaTableHandle.java | 39 +- .../CompositeSchemaRegistry.java | 102 ++++ .../schemamanagement/LocalSchemaRegistry.java | 153 ++++++ .../PravegaSchemaRegistry.java | 196 ++++++++ .../schemamanagement/SchemaRegistry.java | 32 ++ .../schemamanagement/SchemaSupplier.java | 31 ++ .../plugin/pravega/util/PravegaNameUtils.java | 28 +- .../pravega/util/PravegaSchemaUtils.java | 11 +- .../pravega/util/PravegaStreamDescUtils.java | 41 +- .../PravegaTableDescriptionSupplierTest.java | 91 ++++ .../integration/PravegaQueryRunner.java | 53 +- .../integration/TestPravegaKeyValue.java | 2 +- .../LocalSchemaRegistryTest.java | 115 +++++ .../PravegaSchemaRegistryTest.java | 91 ++++ .../{integration => util}/CodecSupplier.java | 2 +- .../util/MockSchemaRegistryClient.java | 173 +++++++ .../pravega/util/MockStreamManager.java | 123 +++++ .../PravegaTestUtils.java | 68 +-- .../pravega/util/SchemaRegistryUtil.java | 120 +++++ .../plugin/pravega/util/TestSchemas.java | 24 + 25 files changed, 1489 insertions(+), 480 deletions(-) create mode 100644 trino/src/main/java/io/trino/plugin/pravega/schemamanagement/CompositeSchemaRegistry.java create mode 100644 trino/src/main/java/io/trino/plugin/pravega/schemamanagement/LocalSchemaRegistry.java create mode 100644 trino/src/main/java/io/trino/plugin/pravega/schemamanagement/PravegaSchemaRegistry.java create mode 100644 trino/src/main/java/io/trino/plugin/pravega/schemamanagement/SchemaRegistry.java create mode 100644 trino/src/main/java/io/trino/plugin/pravega/schemamanagement/SchemaSupplier.java create mode 100644 trino/src/test/java/io/trino/plugin/pravega/PravegaTableDescriptionSupplierTest.java create mode 100644 trino/src/test/java/io/trino/plugin/pravega/schemamanagement/LocalSchemaRegistryTest.java create mode 100644 trino/src/test/java/io/trino/plugin/pravega/schemamanagement/PravegaSchemaRegistryTest.java rename trino/src/test/java/io/trino/plugin/pravega/{integration => util}/CodecSupplier.java (98%) create mode 100644 trino/src/test/java/io/trino/plugin/pravega/util/MockSchemaRegistryClient.java create mode 100644 trino/src/test/java/io/trino/plugin/pravega/util/MockStreamManager.java rename trino/src/test/java/io/trino/plugin/pravega/{integration => util}/PravegaTestUtils.java (63%) create mode 100644 trino/src/test/java/io/trino/plugin/pravega/util/SchemaRegistryUtil.java create mode 100644 trino/src/test/java/io/trino/plugin/pravega/util/TestSchemas.java diff --git a/prestodb/src/main/java/io/pravega/connectors/presto/util/PravegaSchemaUtils.java b/prestodb/src/main/java/io/pravega/connectors/presto/util/PravegaSchemaUtils.java index 4fbfb71..cd3e140 100644 --- a/prestodb/src/main/java/io/pravega/connectors/presto/util/PravegaSchemaUtils.java +++ b/prestodb/src/main/java/io/pravega/connectors/presto/util/PravegaSchemaUtils.java @@ -70,7 +70,7 @@ public static String readSchema(File schemaDir, String dataSchemaLocation) } } else { - inputStream = new FileInputStream(dataSchemaLocation); + inputStream = new FileInputStream(new File(schemaDir, dataSchemaLocation)); } return CharStreams.toString(new InputStreamReader(inputStream, UTF_8)); } diff --git a/trino/src/main/java/io/trino/plugin/pravega/PravegaSplitManager.java b/trino/src/main/java/io/trino/plugin/pravega/PravegaSplitManager.java index f3f267e..4e45da4 100644 --- a/trino/src/main/java/io/trino/plugin/pravega/PravegaSplitManager.java +++ b/trino/src/main/java/io/trino/plugin/pravega/PravegaSplitManager.java @@ -128,10 +128,10 @@ private List getNodeAddresses() private void buildKVSplits(PravegaTableHandle pravegaTableHandle, ImmutableList.Builder splits) { - pravegaTableHandle.getOjectArgs().orElseThrow(() -> + pravegaTableHandle.getObjectArgs().orElseThrow(() -> new IllegalArgumentException("no KF defined for " + pravegaTableHandle)); - for (String kf : pravegaTableHandle.getOjectArgs().get()) { + for (String kf : pravegaTableHandle.getObjectArgs().get()) { PravegaSplit split = new PravegaSplit(connectorId, ObjectType.KV_TABLE, @@ -143,7 +143,7 @@ private void buildKVSplits(PravegaTableHandle pravegaTableHandle, ImmutableList. splits.add(split); } - log.info("created " + pravegaTableHandle.getOjectArgs().get().size() + " kv splits"); + log.info("created " + pravegaTableHandle.getObjectArgs().get().size() + " kv splits"); } private void buildStreamSplits(final PravegaProperties properties, @@ -151,7 +151,7 @@ private void buildStreamSplits(final PravegaProperties properties, ImmutableList.Builder splits) { List sourceStreams = multiSourceStream(pravegaTableHandle) - ? pravegaTableHandle.getOjectArgs().orElseThrow( + ? pravegaTableHandle.getObjectArgs().orElseThrow( () -> new IllegalArgumentException("no args for multi source table found")) : Collections.singletonList(pravegaTableHandle.getObjectName()); diff --git a/trino/src/main/java/io/trino/plugin/pravega/PravegaStreamDescription.java b/trino/src/main/java/io/trino/plugin/pravega/PravegaStreamDescription.java index 4b57e8b..5ef3111 100644 --- a/trino/src/main/java/io/trino/plugin/pravega/PravegaStreamDescription.java +++ b/trino/src/main/java/io/trino/plugin/pravega/PravegaStreamDescription.java @@ -67,6 +67,17 @@ public PravegaStreamDescription(PravegaStreamDescription streamDescription, List this.event = Optional.of(event); } + + public PravegaStreamDescription(PravegaStreamDescription streamDescription, List event, List objectArgs) + { + this.tableName = streamDescription.tableName; + this.schemaName = streamDescription.schemaName; + this.objectName = streamDescription.objectName; + this.objectType = streamDescription.objectType; + this.objectArgs = Optional.of(objectArgs); + this.event = Optional.of(event); + } + @JsonProperty public Optional getSchemaName() { diff --git a/trino/src/main/java/io/trino/plugin/pravega/PravegaStreamFieldGroup.java b/trino/src/main/java/io/trino/plugin/pravega/PravegaStreamFieldGroup.java index 2356229..53daf79 100644 --- a/trino/src/main/java/io/trino/plugin/pravega/PravegaStreamFieldGroup.java +++ b/trino/src/main/java/io/trino/plugin/pravega/PravegaStreamFieldGroup.java @@ -33,7 +33,7 @@ public class PravegaStreamFieldGroup private final String dataFormat; private final Optional dataSchema; private final Optional> fields; - private final Optional mapping; + private final Optional mapping; // column prefix @JsonCreator public PravegaStreamFieldGroup( diff --git a/trino/src/main/java/io/trino/plugin/pravega/PravegaTableDescriptionSupplier.java b/trino/src/main/java/io/trino/plugin/pravega/PravegaTableDescriptionSupplier.java index 2d8126d..f598929 100644 --- a/trino/src/main/java/io/trino/plugin/pravega/PravegaTableDescriptionSupplier.java +++ b/trino/src/main/java/io/trino/plugin/pravega/PravegaTableDescriptionSupplier.java @@ -13,59 +13,29 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package io.trino.plugin.pravega; import com.google.common.annotations.VisibleForTesting; import com.google.common.cache.Cache; import com.google.common.cache.CacheBuilder; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Streams; + import io.airlift.json.JsonCodec; import io.airlift.log.Logger; -import io.pravega.client.ClientConfig; -import io.pravega.client.admin.StreamManager; -import io.pravega.client.stream.Stream; -import io.pravega.schemaregistry.client.SchemaRegistryClient; -import io.pravega.schemaregistry.client.SchemaRegistryClientConfig; -import io.pravega.schemaregistry.client.SchemaRegistryClientFactory; -import io.pravega.schemaregistry.contract.data.GroupProperties; -import io.pravega.schemaregistry.contract.data.SchemaWithVersion; -import io.pravega.schemaregistry.contract.data.SerializationFormat; +import io.trino.plugin.pravega.schemamanagement.CompositeSchemaRegistry; import io.trino.spi.connector.SchemaTableName; import javax.inject.Inject; -import java.io.File; -import java.io.IOException; -import java.io.UncheckedIOException; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collections; -import java.util.LinkedList; +import java.util.HashMap; import java.util.List; -import java.util.Locale; import java.util.Optional; import java.util.concurrent.TimeUnit; import java.util.regex.Pattern; import java.util.stream.Collectors; -import static io.trino.plugin.pravega.ProtobufCommon.encodeSchema; -import static io.trino.plugin.pravega.util.PravegaNameUtils.groupId; -import static io.trino.plugin.pravega.util.PravegaNameUtils.kvFieldMapping; -import static io.trino.plugin.pravega.util.PravegaNameUtils.kvTable; import static io.trino.plugin.pravega.util.PravegaNameUtils.multiSourceStream; -import static io.trino.plugin.pravega.util.PravegaNameUtils.temp_streamNameToTableName; -import static io.trino.plugin.pravega.util.PravegaNameUtils.temp_tableNameToStreamName; -import static io.trino.plugin.pravega.util.PravegaSchemaUtils.GROUP_PROPERTIES_INLINE_KEY; -import static io.trino.plugin.pravega.util.PravegaSchemaUtils.GROUP_PROPERTIES_INLINE_KV_KEY; -import static io.trino.plugin.pravega.util.PravegaSchemaUtils.GROUP_PROPERTIES_INLINE_KV_VALUE; -import static io.trino.plugin.pravega.util.PravegaSchemaUtils.INLINE_SUFFIX; -import static io.trino.plugin.pravega.util.PravegaSchemaUtils.readSchema; -import static io.trino.plugin.pravega.util.PravegaStreamDescUtils.mapFieldsFromSchema; -import static java.nio.file.Files.readAllBytes; -import static java.util.Arrays.asList; import static java.util.Objects.requireNonNull; // pravega scope is a namespace for streams. stream is unique within scope. @@ -84,22 +54,22 @@ public class PravegaTableDescriptionSupplier { private static final Logger log = Logger.get(PravegaTableDescriptionSupplier.class); - private final PravegaConnectorConfig pravegaConnectorConfig; + private final CompositeSchemaRegistry schemaRegistry; private Cache schemaCache; private Cache> tableCache; + // whether we have listed tables from this schema or not + private final HashMap tableListMap = new HashMap<>(); + private JsonCodec streamDescriptionCodec; - // "inline" means that event was written using schema registry wrapped client and schema encoding id - // is within the raw event data in pravega @Inject PravegaTableDescriptionSupplier(PravegaConnectorConfig pravegaConnectorConfig, - JsonCodec streamDescriptionCodec) + JsonCodec streamDescriptionCodec) { requireNonNull(pravegaConnectorConfig, "pravegaConfig is null"); - this.pravegaConnectorConfig = pravegaConnectorConfig; this.streamDescriptionCodec = streamDescriptionCodec; // there will be many successive calls to listSchemas + listTables in short time period @@ -113,16 +83,17 @@ public class PravegaTableDescriptionSupplier this.tableCache = CacheBuilder.newBuilder() .expireAfterWrite(pravegaConnectorConfig.getTableCacheExpireSecs(), TimeUnit.SECONDS) .build(); + + this.schemaRegistry = new CompositeSchemaRegistry(pravegaConnectorConfig, streamDescriptionCodec); } @VisibleForTesting - public PravegaTableDescriptionSupplier(PravegaConnectorConfig pravegaConnectorConfig, - Cache schemaCache, - Cache> tableCache) + public PravegaTableDescriptionSupplier(CompositeSchemaRegistry schemaRegistry) { - this.pravegaConnectorConfig = pravegaConnectorConfig; - this.schemaCache = schemaCache; - this.tableCache = tableCache; + this.schemaRegistry = schemaRegistry; + + this.schemaCache = CacheBuilder.newBuilder().build(); + this.tableCache = CacheBuilder.newBuilder().build(); } public List listSchemas() @@ -131,94 +102,72 @@ public List listSchemas() // they are inserted to cache at same time so will all be same state final List schemas = schemaCache.asMap().keySet().stream().collect(Collectors.toList()); if (schemas.isEmpty()) { - listLocalSchemas().forEach(schema -> schemaCache.put(schema, new Object())); - - try (StreamManager streamManager = - StreamManager.create(ClientConfig.builder().controllerURI( - pravegaConnectorConfig.getControllerURI()).build())) { - Streams.stream(streamManager.listScopes()).filter(s -> !internalObject(s)).forEach(schema -> { - schemas.add(schema); - schemaCache.put(schema, new Object()); - }); - } + schemaRegistry.listSchemas().forEach(schema -> { + schemas.add(schema); + schemaCache.put(schema, new Object()); + }); } else { - log.info("serving listSchemas() from cache"); + log.debug("serving listSchemas() from cache"); } return schemas; } public List listTables(Optional schema) { - List schemas = schema.isPresent() ? Collections.singletonList(schema.get()) : listSchemas(); + List schemas = schema.map(Collections::singletonList).orElseGet(this::listSchemas); - StreamManager streamManager = null; + List tableList = new ArrayList<>(); - try { - List tableList = new ArrayList<>(); + for (String s : schemas) { + List tableListForSchema = + tableCache.asMap().keySet().stream() + .filter(streamDesc -> streamDesc.getSchemaTableName().getSchemaName().startsWith(s)) + .collect(Collectors.toList()); - for (String s : schemas) { - List tableListForSchema = - tableCache.asMap().keySet().stream() - .filter(streamDesc -> streamDesc.getSchemaTableName().getSchemaName().startsWith(s)) - .collect(Collectors.toList()); + if (tableListForSchema.isEmpty() || tableListMap.get(s) == null) { - // not all tables inserted to cache at same time - if (tableListForSchema.isEmpty()) { - if (streamManager == null) { - streamManager = StreamManager.create( - ClientConfig.builder().controllerURI(pravegaConnectorConfig.getControllerURI()).build()); - } + List compositeStreams = new ArrayList<>(); - List compositeStreams = new ArrayList<>(); + schemaRegistry.listTables(s).forEach(table -> { - // local takes precedence. list before pravega. ifAbsent used later to not clobber. - listLocalTables(s).forEach(table -> { - PravegaTableName pravegaTableName = new PravegaTableName(s, table); + // we hide component streams (components of multi-source streams) from view + boolean hidden = + compositeStreams.stream().anyMatch(p -> p.matcher(table.getTableName()).matches()); - // don't clobber existing entry - if (tableCache.getIfPresent(pravegaTableName) == null || - !tableCache.getIfPresent(pravegaTableName).isPresent()) { - tableCache.put(pravegaTableName, Optional.empty()); - } + PravegaTableName pravegaTableName = new PravegaTableName(s, table.getTableName(), hidden); - // load .json def to get stream name in order to determine type - PravegaStreamDescription localTable = getLocalTable(pravegaTableName.getSchemaTableName()); - if (multiSourceStream(localTable)) { - compositeStreams.add(Pattern.compile(localTable.getObjectName())); - } - }); - - // (underlying streams used by kv table are seen as internal and thus are skipped) - Streams.stream(streamManager.listStreams(s)) - .filter(stream -> !internalStream(stream)) - .forEach(stream -> { - boolean hidden = - compositeStreams.stream().anyMatch(p -> p.matcher(stream.getStreamName()).matches()); - // ifAbsent - don't clobber table description if we have it - PravegaTableName tableName = new PravegaTableName(s, temp_streamNameToTableName(stream.getStreamName()), hidden); - if (tableCache.getIfPresent(tableName) == null || - !tableCache.getIfPresent(tableName).isPresent()) { - tableCache.put(tableName, Optional.empty()); - } + // don't clobber existing entry + if (tableCache.getIfPresent(pravegaTableName) == null || + !tableCache.getIfPresent(pravegaTableName).isPresent()) { + tableCache.put(pravegaTableName, Optional.empty()); + } + + if (multiSourceStream(table)) { + // if component streams specified look for exact match when hiding + if (table.getObjectArgs().isPresent()) { + table.getObjectArgs().get().forEach(arg -> { + compositeStreams.add(Pattern.compile("^" + arg + "$")); }); - } - else { - log.info("serving listTables(%s) from cache", s); - } - - tableList.addAll(tableCache.asMap().keySet().stream() - .filter(pravegaStreamDescription -> - pravegaStreamDescription.getSchemaTableName().getSchemaName().startsWith(s)) - .collect(Collectors.toList())); + } + else { + // regex, fuzzy match + compositeStreams.add(Pattern.compile(table.getObjectName())); + } + } + }); + tableListMap.put(s, true); // we have now listed tables from the schema } - return tableList; - } - finally { - if (streamManager != null) { - streamManager.close(); + else { + log.debug("serving listTables(%s) from cache", s); } + + tableList.addAll(tableCache.asMap().keySet().stream() + .filter(pravegaStreamDescription -> + pravegaStreamDescription.getSchemaTableName().getSchemaName().startsWith(s)) + .collect(Collectors.toList())); } + return tableList; } public PravegaStreamDescription getTable(SchemaTableName schemaTableName) @@ -226,275 +175,63 @@ public PravegaStreamDescription getTable(SchemaTableName schemaTableName) PravegaTableName pravegaTableName = new PravegaTableName(schemaTableName); Optional cachedTable = tableCache.getIfPresent(pravegaTableName); if (cachedTable != null && cachedTable.isPresent()) { - log.info("serving getTable(%s) from cache", schemaTableName); + log.debug("serving getTable(%s) from cache", schemaTableName); return cachedTable.get(); } - PravegaStreamDescription table = getLocalTable(schemaTableName); - if (table != null) { - log.info("found local schema for '%s'", schemaTableName); - - // kv this is list of key family (defined in local schema file) - // for multi source stream this is list of composite streams (empty here, to be filled in later) - Optional> objectArgs = table.getObjectArgs(); - - // field definitions can come from 1 of 4 places - // (1) defined in local .json schema ("event/fields") - // (2) uri in "dataSchema" field - // (3) lookup from a source stream (if multi source stream) - // (4) lookup directly in schema registry (if kv table) - - Optional> fieldGroups = Optional.empty(); + PravegaStreamDescription table = schemaRegistry.getTable(schemaTableName); + if (table == null) { + return null; + } - if (fieldsDefined(table)) { - // case (1) - no-op - log.info("fields defined in schema file %s", schemaTableName); - fieldGroups = Optional.of(new LinkedList<>(table.getEvent().get())); - } - else if (table.getEvent().isPresent() && - table.getEvent().get().get(0).getDataSchema().isPresent()) { - fieldGroups = Optional.of(new LinkedList<>()); - - // case (2) uri containing schema - List finalFieldGroups = fieldGroups.get(); - for (int i = 0; i < table.getEvent().get().size(); i++) { - PravegaStreamFieldGroup event = table.getEvent().get().get(i); - String colPrefix = event.getMapping().orElse( - table.getEvent().get().size() > 1 ? kvFieldMapping(i) : ""); - Optional dataSchema = Optional.of(readSchema(event.getDataSchema().get())); - PravegaStreamFieldGroup fieldGroup = - new PravegaStreamFieldGroup(event.getDataFormat(), - Optional.empty(), - dataSchema, - Optional.of( - mapFieldsFromSchema(colPrefix, event.getDataFormat(), dataSchema.get()))); - finalFieldGroups.add(fieldGroup); - } - } - else if (kvTable(table)) { - fieldGroups = fieldGroupsFromSchemaRegistry(schemaTableName); + if (multiSourceStream(table)) { + // if component streams not already specified, look them up from pravega based on regex + List objectArgs = table.getObjectArgs().isPresent() + ? table.getObjectArgs().get() + : multiSourceStreamComponents(schemaTableName, table.getObjectName()); + if (objectArgs.isEmpty()) { + throw new IllegalArgumentException("could not get component streams for " + schemaTableName); } - if (multiSourceStream(table)) { - // stream name will be some regex. - // find all the possible source streams. - Pattern pattern = Pattern.compile(table.getObjectName()); - - List sourceTableNames = - listTables(Optional.of(schemaTableName.getSchemaName())).stream() - .filter(t -> pattern.matcher(t.getSchemaTableName().getTableName()).matches()) - .collect(Collectors.toList()); - - objectArgs = Optional.of(sourceTableNames.stream() - .map(PravegaTableName::getSchemaTableName) - .map(SchemaTableName::getTableName) - .collect(Collectors.toList())); - - if (!fieldGroups.isPresent()) { - // case (3) schema not already defined, look one up - // lookup actual schema from any of them - implies all sources are the same - PravegaStreamDescription sourceTable = sourceTableNames.isEmpty() - ? null - : getTable(sourceTableNames.get(0).getSchemaTableName()); - if (sourceTable == null) { - throw new IllegalArgumentException("no stream found for multi source"); - } - fieldGroups = Optional.of(new LinkedList<>()); - fieldGroups.get().add(new PravegaStreamFieldGroup( - sourceTable.getEvent().get().get(0).getDataFormat(), - Optional.empty(), - sourceTable.getEvent().get().get(0).getDataSchema(), - Optional.of(sourceTable.getEvent().get().get(0).getFields()))); - } + List fieldGroups = table.getEvent().orElse(new ArrayList<>(1)); + if (fieldGroups.isEmpty()) { + fieldGroups = schemaRegistry.getSchema(new SchemaTableName(schemaTableName.getSchemaName(), objectArgs.get(0))); } - fieldGroups.orElseThrow(() -> - new IllegalArgumentException("unable to determine schema for " + schemaTableName)); - - // our final table definition. use schema that we looked up, and set all source stream names here - table = new PravegaStreamDescription(schemaTableName.getTableName(), - Optional.of(schemaTableName.getSchemaName()), - table.getObjectName(), - Optional.of(table.getObjectType()), - objectArgs, - fieldGroups); - - tableCache.put(pravegaTableName, Optional.of(table)); - return table; + table = new PravegaStreamDescription(table, fieldGroups, objectArgs); + } + else if (!fieldsDefined(table)) { + table = new PravegaStreamDescription(table, schemaRegistry.getSchema(schemaTableName)); } - Optional> fieldGroups = fieldGroupsFromSchemaRegistry(schemaTableName); - - table = new PravegaStreamDescription( - schemaTableName.getTableName(), - Optional.of(schemaTableName.getSchemaName()), - temp_tableNameToStreamName(schemaTableName.getTableName()), - Optional.of(ObjectType.STREAM), - Optional.empty() /* args */, - fieldGroups); tableCache.put(pravegaTableName, Optional.of(table)); return table; } - /** - * construct PravegaStreamFieldGroup by looking up schema in schema registry - * - * @param schemaTableName - * @return - */ - private Optional> fieldGroupsFromSchemaRegistry(final SchemaTableName schemaTableName) - { - log.info("look up description of '%s' from pravega", schemaTableName); - String groupName = groupId(schemaTableName.getSchemaName(), temp_tableNameToStreamName(schemaTableName.getTableName())); - - SchemaRegistryClientConfig registryConfig = - SchemaRegistryClientConfig.builder() - .schemaRegistryUri(pravegaConnectorConfig.getSchemaRegistryURI()).build(); - SchemaRegistryClient registryClient = SchemaRegistryClientFactory.withDefaultNamespace(registryConfig); - - List fieldGroups = new ArrayList<>(2); - - GroupProperties properties = - registryClient.getGroupProperties(groupName); - - List schemas = registryClient.getSchemas(groupName); - if (schemas.size() == 0 || schemas.size() > 2) { - throw new IllegalStateException(schemaTableName + " has " + schemas.size() + " registered schemas. expecting either 1 or 2"); - } - - // kv table will have > 1 schema. key+value likely different types - boolean kv = schemas.size() > 1; - - for (int i = 0; i < schemas.size(); i++) { - // colPrefix used for display so can differentiate between fields from key or value - String colPrefix = kv ? kvFieldMapping(i) : ""; - - SerializationFormat format = schemas.get(i).getSchemaInfo().getSerializationFormat(); - fieldGroups.add(new PravegaStreamFieldGroup( - dataFormat(properties.getProperties(), format, kv, i), - Optional.of(colPrefix), - dataSchema(format, schemas.get(i)), - Optional.of(mapFieldsFromSchema(colPrefix, format, schemas.get(i))))); - } - - return Optional.of(fieldGroups); - } - - private static boolean fieldsDefined(PravegaStreamDescription table) - { - // event is optional, fields within event is also optional - // for kv table - 0 or 2 schemas. so fine to just check for 1. - return table.getEvent().isPresent() && (table.getEvent().get().get(0).getFields() != null); - } - - private List listLocalSchemas() + private List multiSourceStreamComponents(SchemaTableName schemaTableName, String sourcePattern) { - return localSchemaStream() - .map(file -> file.getName().split("\\.")[0]) - .collect(Collectors.toList()); - } + Pattern pattern = Pattern.compile(sourcePattern); - // scope.stream -> schema.table - private List listLocalTables(String schema) - { - return localSchemaStream() - .filter(file -> file.getName().endsWith(".json")) - .filter(file -> file.getName().startsWith(schema)) - .filter(file -> file.getName().split("\\.").length == 3) - .map(file -> file.getName().split("\\.")[1]) + return listTables(Optional.of(schemaTableName.getSchemaName())).stream() + .map(PravegaTableName::getSchemaTableName) + .map(SchemaTableName::getTableName) + .filter(tableName -> pattern.matcher(tableName).matches()) .collect(Collectors.toList()); } - private PravegaStreamDescription getLocalTable(SchemaTableName schemaTableName) + private static boolean fieldsDefined(PravegaStreamDescription table) { - try { - File file = new File(pravegaConnectorConfig.getTableDescriptionDir(), - String.format("%s.%s.json", schemaTableName.getSchemaName(), schemaTableName.getTableName())); - if (!file.exists()) { - return null; - } - return streamDescriptionCodec.fromJson(readAllBytes(file.toPath())); - } - catch (IOException e) { - log.error("%s", e); - throw new UncheckedIOException(e); + if (!table.getEvent().isPresent() || + table.getEvent().get().isEmpty()) { + return false; } - catch (RuntimeException e) { - log.error("%s", e); - throw e; - } - } - private java.util.stream.Stream localSchemaStream() - { - return listFiles(pravegaConnectorConfig.getTableDescriptionDir()).stream() - .filter(file -> file.isFile() && file.getName().endsWith(".json")); - } - - private static List listFiles(File dir) - { - if ((dir != null) && dir.isDirectory()) { - File[] files = dir.listFiles(); - if (files != null) { - log.debug("Considering files: %s", asList(files)); - return ImmutableList.copyOf(files); + for (PravegaStreamFieldGroup fieldGroup : table.getEvent().get()) { + if (fieldGroup.getFields() == null) { + return false; } - } - return ImmutableList.of(); - } - - private static String dataFormat(ImmutableMap groupProperties, - SerializationFormat format, - boolean kvTable, - int kvIdx) - { - /* - TODO: auto-detect https://github.com/pravega/presto-connector/issues/20 - (1) no schema registry. - (2) Register and evolve schemas in registry but do not use registry client while writing data - (3) Register schemas in the registry and use registry client to encode schema Id with payload - "inline" is for #3. for e.g. "avro" -> "avro-inline". PravegaRecordSetProvider is interested in this - hopefully this can all go away (see linked issue 58 above) - but for now the following is our convention - if "inline" exists in our properties, all data uses SR - else if it is a kv table key+value may be different. both, neither, or either may use SR - look for "inlinekey" / "inlinevalue" - */ - - String key = GROUP_PROPERTIES_INLINE_KEY; - - if (kvTable && !groupProperties.containsKey(key)) { - key = kvIdx == 0 ? GROUP_PROPERTIES_INLINE_KV_KEY : GROUP_PROPERTIES_INLINE_KV_VALUE; - } - String finalFormat = format == SerializationFormat.Custom - ? format.getFullTypeName().toLowerCase(Locale.ENGLISH) - : format.name().toLowerCase(Locale.ENGLISH); - return finalFormat + (groupProperties.containsKey(key) ? INLINE_SUFFIX : ""); - } - - private static Optional dataSchema(SerializationFormat format, SchemaWithVersion schemaWithVersion) - { - // it is intentional that nothing is returned for Custom - // pass schema to row decoders. refer to PravegaRecordSetProvider - switch (format) { - case Protobuf: - return Optional.of(encodeSchema(schemaWithVersion)); - case Avro: - return Optional.of(new String(schemaWithVersion.getSchemaInfo().getSchemaData().array(), StandardCharsets.UTF_8)); - default: - return Optional.empty(); } - } - - private static boolean internalStream(Stream stream) - { - return internalObject(stream.getStreamName()); - } - - private static boolean internalObject(String object) - { - return object.startsWith("_") /* pravega internal */ || - object.endsWith("-SC") /* application internal - stream cuts */; + return true; } } diff --git a/trino/src/main/java/io/trino/plugin/pravega/PravegaTableHandle.java b/trino/src/main/java/io/trino/plugin/pravega/PravegaTableHandle.java index de5fc0f..d487356 100644 --- a/trino/src/main/java/io/trino/plugin/pravega/PravegaTableHandle.java +++ b/trino/src/main/java/io/trino/plugin/pravega/PravegaTableHandle.java @@ -34,11 +34,6 @@ public final class PravegaTableHandle implements ConnectorTableHandle { - /** - * connector id - */ - private final String connectorId; - /** * The schema name for this table. Is set through configuration and read */ @@ -56,6 +51,11 @@ public final class PravegaTableHandle */ private final String objectName; + /** + * optional + * for ObjectType.STREAM, this is list of composite streams in a multi source stream + * for ObjectType.KV_TABLE this is list of key families + */ private final Optional> objectArgs; private final List schema; @@ -63,9 +63,7 @@ public final class PravegaTableHandle private final String schemaRegistryGroupId; @JsonCreator - public PravegaTableHandle( - @JsonProperty("connectorId") String connectorId, - @JsonProperty("schemaName") String schemaName, + public PravegaTableHandle(@JsonProperty("schemaName") String schemaName, @JsonProperty("tableName") String tableName, @JsonProperty("objectName") String objectName, @JsonProperty("objectType") ObjectType objectType, @@ -73,7 +71,6 @@ public PravegaTableHandle( @JsonProperty("schema") List schema, @JsonProperty("schemaRegistryGroupId") String schemaRegistryGroupId) { - this.connectorId = requireNonNull(connectorId, "connectorId is null"); this.schemaName = requireNonNull(schemaName, "schemaName is null"); this.tableName = requireNonNull(tableName, "tableName is null"); this.objectName = requireNonNull(objectName, "objectName is null"); @@ -83,10 +80,20 @@ public PravegaTableHandle( this.schemaRegistryGroupId = requireNonNull(schemaRegistryGroupId, "schemaRegistryGroupId is null"); } - @JsonProperty - public String getConnectorId() + public PravegaTableHandle( + @JsonProperty("schemaName") String schemaName, + @JsonProperty("tableName") String tableName, + @JsonProperty("objectName") String objectName, + @JsonProperty("objectType") ObjectType objectType, + @JsonProperty("objectArgs") Optional> objectArgs) { - return connectorId; + this.schemaName = requireNonNull(schemaName, "schemaName is null"); + this.tableName = requireNonNull(tableName, "tableName is null"); + this.objectName = requireNonNull(objectName, "objectName is null"); + this.objectType = requireNonNull(objectType, "objectType is null"); + this.objectArgs = objectArgs; + this.schema = null; + this.schemaRegistryGroupId = null; } @JsonProperty @@ -114,7 +121,7 @@ public String getObjectName() } @JsonProperty - public Optional> getOjectArgs() + public Optional> getObjectArgs() { return objectArgs; } @@ -139,7 +146,7 @@ public SchemaTableName toSchemaTableName() @Override public int hashCode() { - return Objects.hash(connectorId, schemaName, tableName, objectName, objectType, schema, schemaRegistryGroupId); + return Objects.hash(schemaName, tableName, objectName, objectType, schema, schemaRegistryGroupId); } @Override @@ -153,8 +160,7 @@ public boolean equals(Object obj) } PravegaTableHandle other = (PravegaTableHandle) obj; - return Objects.equals(this.connectorId, other.connectorId) - && Objects.equals(this.schemaName, other.schemaName) + return Objects.equals(this.schemaName, other.schemaName) && Objects.equals(this.tableName, other.tableName) && Objects.equals(this.objectName, other.objectName) && Objects.equals(this.objectType, other.objectType) @@ -166,7 +172,6 @@ public boolean equals(Object obj) public String toString() { return toStringHelper(this) - .add("connectorId", connectorId) .add("schemaName", schemaName) .add("tableName", tableName) .add("objectName", objectName) diff --git a/trino/src/main/java/io/trino/plugin/pravega/schemamanagement/CompositeSchemaRegistry.java b/trino/src/main/java/io/trino/plugin/pravega/schemamanagement/CompositeSchemaRegistry.java new file mode 100644 index 0000000..ec04e03 --- /dev/null +++ b/trino/src/main/java/io/trino/plugin/pravega/schemamanagement/CompositeSchemaRegistry.java @@ -0,0 +1,102 @@ +/* + * Copyright (c) Pravega Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.pravega.schemamanagement; + +import com.google.common.annotations.VisibleForTesting; +import io.airlift.json.JsonCodec; +import io.trino.plugin.pravega.PravegaConnectorConfig; +import io.trino.plugin.pravega.PravegaStreamDescription; +import io.trino.plugin.pravega.PravegaStreamFieldGroup; +import io.trino.plugin.pravega.PravegaTableHandle; +import io.trino.spi.connector.SchemaTableName; + +import java.util.ArrayList; +import java.util.List; + +public class CompositeSchemaRegistry + implements SchemaSupplier, SchemaRegistry { + private final List schemaSuppliers; + + private final List schemaRegistries; + + public CompositeSchemaRegistry(PravegaConnectorConfig config, JsonCodec streamDescriptionCodec) { + schemaSuppliers = new ArrayList<>(); + schemaRegistries = new ArrayList<>(); + + // local will override, always add first + if (config.getTableDescriptionDir() != null && + config.getTableDescriptionDir().exists() && + config.getTableDescriptionDir().isDirectory()) { + LocalSchemaRegistry schemaRegistry = + new LocalSchemaRegistry(config.getTableDescriptionDir(), streamDescriptionCodec); + schemaSuppliers.add(schemaRegistry); + schemaRegistries.add(schemaRegistry); + } + + if (config.getSchemaRegistryURI() != null) { + PravegaSchemaRegistry schemaRegistry = + new PravegaSchemaRegistry(config.getControllerURI(), config.getSchemaRegistryURI()); + schemaSuppliers.add(schemaRegistry); + schemaRegistries.add(schemaRegistry); + } + } + + @VisibleForTesting + public CompositeSchemaRegistry(List schemaSuppliers, List schemaRegistries) + { + this.schemaSuppliers = schemaSuppliers; + this.schemaRegistries = schemaRegistries; + } + + @Override + public List listSchemas() + { + final List schemas = new ArrayList<>(); + schemaSuppliers.forEach(p -> schemas.addAll(p.listSchemas())); + return schemas; + } + + @Override + public List listTables(String schema) + { + final List tables = new ArrayList<>(); + schemaSuppliers.forEach(p -> tables.addAll(p.listTables(schema))); + return tables; + } + + @Override + public List getSchema(SchemaTableName schemaTableName) { + for (SchemaRegistry schemaRegistry : schemaRegistries) { + List schema = schemaRegistry.getSchema(schemaTableName); + if (schema != null) { + return schema; + } + } + return null; + } + + @Override + public PravegaStreamDescription getTable(SchemaTableName schemaTableName) + { + for (SchemaRegistry schemaRegistry : schemaRegistries) { + PravegaStreamDescription streamDescription = schemaRegistry.getTable(schemaTableName); + if (streamDescription != null) { + return streamDescription; + } + } + return null; + } +} diff --git a/trino/src/main/java/io/trino/plugin/pravega/schemamanagement/LocalSchemaRegistry.java b/trino/src/main/java/io/trino/plugin/pravega/schemamanagement/LocalSchemaRegistry.java new file mode 100644 index 0000000..f5bf09c --- /dev/null +++ b/trino/src/main/java/io/trino/plugin/pravega/schemamanagement/LocalSchemaRegistry.java @@ -0,0 +1,153 @@ +/* + * Copyright (c) Pravega Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.pravega.schemamanagement; + +import com.google.common.collect.ImmutableList; +import io.airlift.json.JsonCodec; +import io.trino.plugin.pravega.ObjectType; +import io.trino.plugin.pravega.PravegaStreamDescription; +import io.trino.plugin.pravega.PravegaStreamFieldGroup; +import io.trino.plugin.pravega.PravegaTableHandle; +import io.trino.spi.connector.SchemaTableName; + +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + +import static io.trino.plugin.pravega.util.PravegaNameUtils.kvFieldMapping; +import static io.trino.plugin.pravega.util.PravegaStreamDescUtils.resolveAllSchemas; +import static java.nio.file.Files.readAllBytes; + +public class LocalSchemaRegistry + implements SchemaSupplier, SchemaRegistry +{ + private final File localTableDir; + + private final JsonCodec streamDescriptionCodec; + + public LocalSchemaRegistry(File localTableDir, JsonCodec streamDescriptionCodec) + { + this.localTableDir = localTableDir; + this.streamDescriptionCodec = streamDescriptionCodec; + } + + @Override + public List listSchemas() + { + // file name format: {schema}.{table}.json + return localSchemaStream() + // ensures matches full file name format + .filter(file -> file.getName().split("\\.").length == 3) + .map(file -> file.getName().split("\\.")[0]) + .distinct() + .collect(Collectors.toList()); + } + + @Override + public List listTables(String schema) + { + final List tables = new ArrayList<>(); + + // file name format: {schema}.{table}.json + localSchemaStream() + .filter(file -> file.getName().startsWith(schema)) + // ensures matches full file name format + .filter(file -> file.getName().split("\\.").length == 3) + // {table} from file name + .map(file -> file.getName().split("\\.")[1]) + .map(file -> getLocalTable(new SchemaTableName(schema, file))) + .forEach(table -> { + tables.add(new PravegaTableHandle(table.getSchemaName().get(), + table.getTableName(), + table.getObjectName(), + table.getObjectType(), + table.getObjectArgs())); + }); + + return tables; + } + + @Override + public List getSchema(SchemaTableName schemaTableName) { + PravegaStreamDescription streamDescription = getLocalTable(schemaTableName); + return streamDescription == null ? null : streamDescription.getEvent().orElse(null); + } + + @Override + public PravegaStreamDescription getTable(SchemaTableName schemaTableName) + { + // reads table definition from local file + // if table definition has pointers to schema, read it and populate fields + // (for e.g. local schema file or url to schema) + PravegaStreamDescription table = getLocalTable(schemaTableName); + + // either not found or no fields, nothing to do. will be resolved later + if (table == null || !table.getEvent().isPresent()) { + return table; + } + + // fields already defined + if (table.getEvent().get().stream().noneMatch( + schema -> schema.getDataSchema().isPresent())) { + return table; + } + + // at least 1 schema for a fieldGroup must be resolved. read schema from local file or url + List finalSchemas = + resolveAllSchemas(localTableDir, table.getEvent().get(), (i) -> columnPrefix(table, i)); + + return new PravegaStreamDescription(table, finalSchemas); + } + + static String columnPrefix(PravegaStreamDescription table, int schemaIndex) { + // if kv table, returns something like key/{fieldName}, value/{fieldName} + return table.getObjectType() == ObjectType.KV_TABLE ? kvFieldMapping(schemaIndex) : ""; + } + + private java.util.stream.Stream localSchemaStream() + { + return listFiles(localTableDir).stream() + .filter(file -> file.isFile() && file.getName().endsWith(".json")); + } + + private static List listFiles(File dir) + { + if ((dir != null) && dir.isDirectory()) { + File[] files = dir.listFiles(); + if (files != null) { + return ImmutableList.copyOf(files); + } + } + return ImmutableList.of(); + } + + private PravegaStreamDescription getLocalTable(SchemaTableName schemaTableName) + { + try { + File file = new File(localTableDir, String.format("%s.%s.json", + schemaTableName.getSchemaName(), schemaTableName.getTableName())); + return !file.exists() ? + null + : streamDescriptionCodec.fromJson(readAllBytes(file.toPath())); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } +} diff --git a/trino/src/main/java/io/trino/plugin/pravega/schemamanagement/PravegaSchemaRegistry.java b/trino/src/main/java/io/trino/plugin/pravega/schemamanagement/PravegaSchemaRegistry.java new file mode 100644 index 0000000..a6dbe23 --- /dev/null +++ b/trino/src/main/java/io/trino/plugin/pravega/schemamanagement/PravegaSchemaRegistry.java @@ -0,0 +1,196 @@ +/* + * Copyright (c) Pravega Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.pravega.schemamanagement; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Streams; +import io.pravega.client.ClientConfig; +import io.pravega.client.admin.StreamManager; +import io.trino.plugin.pravega.ObjectType; +import io.trino.plugin.pravega.PravegaStreamDescription; +import io.trino.plugin.pravega.PravegaStreamFieldGroup; +import io.trino.plugin.pravega.PravegaTableHandle; +import io.pravega.schemaregistry.client.SchemaRegistryClient; +import io.pravega.schemaregistry.client.SchemaRegistryClientConfig; +import io.pravega.schemaregistry.client.SchemaRegistryClientFactory; +import io.pravega.schemaregistry.client.exceptions.RegistryExceptions; +import io.pravega.schemaregistry.contract.data.GroupProperties; +import io.pravega.schemaregistry.contract.data.SchemaWithVersion; +import io.pravega.schemaregistry.contract.data.SerializationFormat; +import io.trino.spi.connector.SchemaTableName; + +import java.net.URI; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.Locale; +import java.util.Optional; + +import static io.trino.plugin.pravega.ProtobufCommon.encodeSchema; +import static io.trino.plugin.pravega.util.PravegaNameUtils.*; +import static io.trino.plugin.pravega.util.PravegaSchemaUtils.*; +import static io.trino.plugin.pravega.util.PravegaStreamDescUtils.mapFieldsFromSchema; + +public class PravegaSchemaRegistry + implements SchemaSupplier, SchemaRegistry +{ + private final SchemaRegistryClient registryClient; + + private final StreamManager streamManager; + + public PravegaSchemaRegistry(URI controllerURI, URI schemaRegistryURI) + { + SchemaRegistryClientConfig registryConfig = + SchemaRegistryClientConfig.builder().schemaRegistryUri(schemaRegistryURI).build(); + this.registryClient = SchemaRegistryClientFactory.withDefaultNamespace(registryConfig); + this.streamManager = StreamManager.create(ClientConfig.builder().controllerURI(controllerURI).build()); + } + + @VisibleForTesting + public PravegaSchemaRegistry(SchemaRegistryClient registryClient, StreamManager streamManager) + { + this.registryClient = registryClient; + this.streamManager = streamManager; + } + + @Override + public List listSchemas() + { + List schemas = new ArrayList<>(); + Streams.stream(streamManager.listScopes()).filter(s -> !internalObject(s)).forEach(schemas::add); + return schemas; + } + + @Override + public List listTables(String schema) + { + // (underlying streams used by kv table are seen as internal and thus are skipped) + List tables = new ArrayList<>(); + Streams.stream(streamManager.listStreams(schema)) + .filter(stream -> !internalStream(stream)) + .forEach(stream -> { + tables.add(new PravegaTableHandle(schema, + temp_streamNameToTableName(stream.getStreamName()), + stream.getStreamName(), + ObjectType.STREAM, + Optional.empty())); + }); + return tables; + } + + @Override + public List getSchema(SchemaTableName schemaTableName) { + String groupName = groupId(schemaTableName.getSchemaName(), + temp_tableNameToStreamName(schemaTableName.getTableName())); + + GroupProperties properties; + List schemas; + + try { + properties = registryClient.getGroupProperties(groupName); + schemas = registryClient.getSchemas(groupName); + } + catch (RegistryExceptions.ResourceNotFoundException e) { + return null; + } + + if (schemas.size() == 0 || schemas.size() > 2) { + throw new IllegalStateException(schemaTableName + " has " + schemas.size() + " registered schemas. expecting either 1 or 2"); + } + + // kv table will have > 1 schema. key+value likely different types + boolean kv = schemas.size() > 1; + List fieldGroups = new ArrayList<>(2); + for (int i = 0; i < schemas.size(); i++) { + // colPrefix used for display so can differentiate between fields from key or value + String colPrefix = kv ? kvFieldMapping(i) : ""; + + SerializationFormat format = schemas.get(i).getSchemaInfo().getSerializationFormat(); + fieldGroups.add(new PravegaStreamFieldGroup( + dataFormat(properties.getProperties(), format, kv, i), + Optional.of(colPrefix), + dataSchema(format, schemas.get(i)), + Optional.of(mapFieldsFromSchema(colPrefix, format, schemas.get(i))))); + } + + return fieldGroups; + } + + @Override + public PravegaStreamDescription getTable(SchemaTableName schemaTableName) + { + List schema = getSchema(schemaTableName); + if (schema == null) { + return null; + } + + return new PravegaStreamDescription( + schemaTableName.getTableName(), + Optional.of(schemaTableName.getSchemaName()), + temp_tableNameToStreamName(schemaTableName.getTableName()), + Optional.of(ObjectType.STREAM), + Optional.empty() /* args */, + Optional.of(schema)); + } + + private static String dataFormat(ImmutableMap groupProperties, + SerializationFormat format, + boolean kvTable, + int kvIdx) + { + /* + TODO: auto-detect https://github.com/pravega/presto-connector/issues/20 + + (1) no schema registry. + (2) Register and evolve schemas in registry but do not use registry client while writing data + (3) Register schemas in the registry and use registry client to encode schema Id with payload + "inline" is for #3. for e.g. "avro" -> "avro-inline". PravegaRecordSetProvider is interested in this + + hopefully this can all go away (see linked issue 58 above) + + but for now the following is our convention + if "inline" exists in our properties, all data uses SR + else if it is a kv table key+value may be different. both, neither, or either may use SR + look for "inlinekey" / "inlinevalue" + */ + + String key = GROUP_PROPERTIES_INLINE_KEY; + + if (kvTable && !groupProperties.containsKey(key)) { + key = kvIdx == 0 ? GROUP_PROPERTIES_INLINE_KV_KEY : GROUP_PROPERTIES_INLINE_KV_VALUE; + } + + String finalFormat = format == SerializationFormat.Custom + ? format.getFullTypeName().toLowerCase(Locale.ENGLISH) + : format.name().toLowerCase(Locale.ENGLISH); + return finalFormat + (groupProperties.containsKey(key) ? INLINE_SUFFIX : ""); + } + + public static Optional dataSchema(SerializationFormat format, SchemaWithVersion schemaWithVersion) + { + // it is intentional that nothing is returned for Custom + // pass schema to row decoders. refer to PravegaRecordSetProvider + switch (format) { + case Protobuf: + return Optional.of(encodeSchema(schemaWithVersion)); + case Avro: + return Optional.of(new String(schemaWithVersion.getSchemaInfo().getSchemaData().array(), StandardCharsets.UTF_8)); + default: + return Optional.empty(); + } + } +} diff --git a/trino/src/main/java/io/trino/plugin/pravega/schemamanagement/SchemaRegistry.java b/trino/src/main/java/io/trino/plugin/pravega/schemamanagement/SchemaRegistry.java new file mode 100644 index 0000000..14bce9a --- /dev/null +++ b/trino/src/main/java/io/trino/plugin/pravega/schemamanagement/SchemaRegistry.java @@ -0,0 +1,32 @@ +/* + * Copyright (c) Pravega Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.pravega.schemamanagement; + +import io.trino.plugin.pravega.PravegaStreamDescription; +import io.trino.plugin.pravega.PravegaStreamFieldGroup; +import io.trino.spi.connector.SchemaTableName; + +import java.util.List; + +/** + * return schema for the given schema.table + */ +public interface SchemaRegistry +{ + PravegaStreamDescription getTable(SchemaTableName schemaTableName); + + List getSchema(SchemaTableName schemaTableName); +} diff --git a/trino/src/main/java/io/trino/plugin/pravega/schemamanagement/SchemaSupplier.java b/trino/src/main/java/io/trino/plugin/pravega/schemamanagement/SchemaSupplier.java new file mode 100644 index 0000000..2652193 --- /dev/null +++ b/trino/src/main/java/io/trino/plugin/pravega/schemamanagement/SchemaSupplier.java @@ -0,0 +1,31 @@ +/* + * Copyright (c) Pravega Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.pravega.schemamanagement; + +import io.trino.plugin.pravega.PravegaTableHandle; + +import java.util.List; + +/** + * a source of schemas, and tables + * this may be reading files on local disk or listing scopes+streams from pravega + */ +public interface SchemaSupplier +{ + List listSchemas(); + + List listTables(String schema); +} diff --git a/trino/src/main/java/io/trino/plugin/pravega/util/PravegaNameUtils.java b/trino/src/main/java/io/trino/plugin/pravega/util/PravegaNameUtils.java index 07c0210..9e83fa9 100644 --- a/trino/src/main/java/io/trino/plugin/pravega/util/PravegaNameUtils.java +++ b/trino/src/main/java/io/trino/plugin/pravega/util/PravegaNameUtils.java @@ -13,13 +13,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package io.trino.plugin.pravega.util; -import io.pravega.shared.NameUtils; +import io.pravega.client.stream.Stream; import io.trino.plugin.pravega.ObjectType; import io.trino.plugin.pravega.PravegaStreamDescription; import io.trino.plugin.pravega.PravegaTableHandle; +import io.pravega.shared.NameUtils; import java.util.HashMap; import java.util.Map; @@ -35,7 +35,6 @@ private PravegaNameUtils() // used for prefixing field names when presenting them in presto // will default to these prefixes for kv table fields unless specified in user config static Map kvFieldNamePrefixMap = new HashMap<>(); - static { kvFieldNamePrefixMap.put(0, "key"); @@ -52,20 +51,24 @@ public static String groupId(String scope, String stream) return scope + "." + stream; } - // test stream name - if not valid pravega stream name assume it is regex for multi source public static boolean multiSourceStream(PravegaStreamDescription object) { + // if stream name is a regex, or if there are object args + // (objectArgs for stream are comma sep list of component streams) return object.getObjectType() == ObjectType.STREAM && - multiSourceStream(object.getObjectName()); + (multiSourceStream(object.getObjectName()) || object.getObjectArgs().isPresent()); } public static boolean multiSourceStream(PravegaTableHandle object) { + // if stream name is a regex, or if there are object args + // (objectArgs for stream are comma sep list of component streams) return object.getObjectType() == ObjectType.STREAM && - multiSourceStream(object.getObjectName()); + (multiSourceStream(object.getObjectName()) || object.getObjectArgs().isPresent()); } - private static boolean multiSourceStream(String stream) + // test stream name - if not valid pravega stream name assume it is regex for multi source + public static boolean multiSourceStream(String stream) { try { // test pattern for stream names pravega will allow @@ -109,4 +112,15 @@ public static String streamCutName(String stream) { return stream + STREAM_CUT_PREFIX; } + + public static boolean internalStream(Stream stream) + { + return internalObject(stream.getStreamName()); + } + + public static boolean internalObject(String object) + { + return object.startsWith("_") /* pravega internal */ || + object.endsWith("-SC") /* application internal - stream cuts */; + } } diff --git a/trino/src/main/java/io/trino/plugin/pravega/util/PravegaSchemaUtils.java b/trino/src/main/java/io/trino/plugin/pravega/util/PravegaSchemaUtils.java index 85d214f..a494bd7 100644 --- a/trino/src/main/java/io/trino/plugin/pravega/util/PravegaSchemaUtils.java +++ b/trino/src/main/java/io/trino/plugin/pravega/util/PravegaSchemaUtils.java @@ -20,10 +20,7 @@ import io.airlift.log.Logger; import io.trino.spi.TrinoException; -import java.io.FileInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; +import java.io.*; import java.net.MalformedURLException; import java.net.URI; import java.net.URL; @@ -54,7 +51,7 @@ private PravegaSchemaUtils() public static final String NESTED_RECORD_SEPARATOR = "/"; - public static String readSchema(String dataSchemaLocation) + public static String readSchema(File schemaDir, String dataSchemaLocation) { InputStream inputStream = null; try { @@ -65,11 +62,11 @@ public static String readSchema(String dataSchemaLocation) catch (MalformedURLException e) { // try again before failing log.warn("invalid URL: " + dataSchemaLocation); - inputStream = new FileInputStream(dataSchemaLocation); + inputStream = new FileInputStream(new File(schemaDir, dataSchemaLocation)); } } else { - inputStream = new FileInputStream(dataSchemaLocation); + inputStream = new FileInputStream(new File(schemaDir, dataSchemaLocation)); } return CharStreams.toString(new InputStreamReader(inputStream, UTF_8)); } diff --git a/trino/src/main/java/io/trino/plugin/pravega/util/PravegaStreamDescUtils.java b/trino/src/main/java/io/trino/plugin/pravega/util/PravegaStreamDescUtils.java index d08add0..95a4098 100644 --- a/trino/src/main/java/io/trino/plugin/pravega/util/PravegaStreamDescUtils.java +++ b/trino/src/main/java/io/trino/plugin/pravega/util/PravegaStreamDescUtils.java @@ -13,7 +13,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package io.trino.plugin.pravega.util; import com.google.protobuf.Descriptors; @@ -21,6 +20,7 @@ import io.pravega.schemaregistry.contract.data.SerializationFormat; import io.pravega.schemaregistry.serializer.json.schemas.JSONSchema; import io.trino.plugin.pravega.PravegaStreamFieldDescription; +import io.trino.plugin.pravega.PravegaStreamFieldGroup; import io.trino.plugin.pravega.ProtobufCommon; import io.trino.spi.type.Type; import org.everit.json.schema.BooleanSchema; @@ -29,13 +29,14 @@ import org.everit.json.schema.Schema; import org.everit.json.schema.StringSchema; +import java.io.File; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; -import static io.trino.plugin.pravega.util.PravegaSchemaUtils.AVRO; -import static io.trino.plugin.pravega.util.PravegaSchemaUtils.NESTED_RECORD_SEPARATOR; +import static io.trino.plugin.pravega.util.PravegaSchemaUtils.*; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.BooleanType.BOOLEAN; import static io.trino.spi.type.DoubleType.DOUBLE; @@ -344,4 +345,38 @@ private static String nestedPrefixFor(String prefix, String name) ? name : prefix + NESTED_RECORD_SEPARATOR + name; } + /** + * PravegaStreamFieldGroup may contain pointer to schema (local file, or url) + * for these, read the schema and build the field definitions + * + * @param schemaDir directory where we can find the schema + * @param fieldGroups fieldGroups to look through + * @param columnPrefix function to return columnPrefix to be used for the fields in the group + * @return list of PravegaStreamFieldGroup with all schemas resolved + */ + public static List resolveAllSchemas(File schemaDir, + List fieldGroups, + Function columnPrefix) + { + // fields already defined + if (fieldGroups.stream().noneMatch( + schema -> schema.getDataSchema().isPresent())) { + return fieldGroups; + } + + // at least 1 schema for a fieldGroup must be resolved. read schema from local file or url + List finalSchemas = new ArrayList<>(fieldGroups.size()); + for (int i = 0; i < fieldGroups.size(); i++) { + PravegaStreamFieldGroup fieldGroup = fieldGroups.get(i); + if (fieldGroup.getDataSchema().isPresent()) { + String dataSchema = readSchema(schemaDir, fieldGroup.getDataSchema().get()); + List fields = + mapFieldsFromSchema(columnPrefix.apply(i), fieldGroup.getDataFormat(), dataSchema); + finalSchemas.add(new PravegaStreamFieldGroup(fieldGroup, dataSchema, fields)); + } else { + finalSchemas.add(fieldGroup); + } + } + return finalSchemas; + } } diff --git a/trino/src/test/java/io/trino/plugin/pravega/PravegaTableDescriptionSupplierTest.java b/trino/src/test/java/io/trino/plugin/pravega/PravegaTableDescriptionSupplierTest.java new file mode 100644 index 0000000..c237dfa --- /dev/null +++ b/trino/src/test/java/io/trino/plugin/pravega/PravegaTableDescriptionSupplierTest.java @@ -0,0 +1,91 @@ +/* + * Copyright (c) Pravega Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.pravega; + +import io.trino.plugin.pravega.util.SchemaRegistryUtil; +import io.trino.spi.connector.SchemaTableName; +import org.testng.annotations.Test; + +import java.util.List; + +import static io.trino.plugin.pravega.util.TestSchemas.EMPLOYEE_AVSC; +import static org.testng.Assert.*; + +@Test +public class PravegaTableDescriptionSupplierTest +{ + private static final String SCHEMA = "ut"; + + @Test + public void testTableDoesNotExist() + { + PravegaTableDescriptionSupplier tableSupplier = + new PravegaTableDescriptionSupplier(new SchemaRegistryUtil().getSchemaRegistry()); + assertNull(tableSupplier.getTable(new SchemaTableName(SCHEMA, "stream1"))); + } + + @Test + public void testMultiSourceStreamRegex() + { + SchemaRegistryUtil schemaRegistryUtil = new SchemaRegistryUtil(); + schemaRegistryUtil.addLocalSchema(SCHEMA); + + PravegaTableDescriptionSupplier tableSupplier = + new PravegaTableDescriptionSupplier(schemaRegistryUtil.getSchemaRegistry()); + + + schemaRegistryUtil.addSchema(SCHEMA); + schemaRegistryUtil.addTable(new SchemaTableName(SCHEMA, "stream1"), EMPLOYEE_AVSC); + schemaRegistryUtil.addTable(new SchemaTableName(SCHEMA, "stream2"), EMPLOYEE_AVSC); + schemaRegistryUtil.addTable(new SchemaTableName(SCHEMA, "stream3"), EMPLOYEE_AVSC); + + PravegaStreamDescription table = + tableSupplier.getTable(new SchemaTableName(SCHEMA, "multiregex")); + + assertNotNull(table); + assertTrue(table.getObjectArgs().isPresent()); + + List components = table.getObjectArgs().get(); + assertEquals(components.size(), 3); + assertEquals(components.stream().sorted().toArray(), new String[]{"stream1", "stream2", "stream3"}); + } + + @Test + public void testMultiSourceStreamExplicit() + { + // same setup as regex. but multi source def. only has 2 component streams. + SchemaRegistryUtil schemaRegistryUtil = new SchemaRegistryUtil(); + schemaRegistryUtil.addLocalSchema(SCHEMA); + + PravegaTableDescriptionSupplier tableSupplier = + new PravegaTableDescriptionSupplier(schemaRegistryUtil.getSchemaRegistry()); + + schemaRegistryUtil.addSchema(SCHEMA); + schemaRegistryUtil.addTable(new SchemaTableName(SCHEMA, "stream1"), EMPLOYEE_AVSC); + schemaRegistryUtil.addTable(new SchemaTableName(SCHEMA, "stream2"), EMPLOYEE_AVSC); + schemaRegistryUtil.addTable(new SchemaTableName(SCHEMA, "stream3"), EMPLOYEE_AVSC); + + PravegaStreamDescription table = + tableSupplier.getTable(new SchemaTableName(SCHEMA, "multiexplicit")); + + assertNotNull(table); + assertTrue(table.getObjectArgs().isPresent()); + + List components = table.getObjectArgs().get(); + assertEquals(components.size(), 2); + assertEquals(components.stream().sorted().toArray(), new String[]{"stream1", "stream3"}); + } +} diff --git a/trino/src/test/java/io/trino/plugin/pravega/integration/PravegaQueryRunner.java b/trino/src/test/java/io/trino/plugin/pravega/integration/PravegaQueryRunner.java index 5f3ab81..dd362c0 100644 --- a/trino/src/test/java/io/trino/plugin/pravega/integration/PravegaQueryRunner.java +++ b/trino/src/test/java/io/trino/plugin/pravega/integration/PravegaQueryRunner.java @@ -15,35 +15,34 @@ */ package io.trino.plugin.pravega.integration; -import com.google.common.cache.Cache; -import com.google.common.cache.CacheBuilder; import com.google.common.collect.ImmutableMap; -import io.airlift.json.JsonCodec; import io.airlift.log.Logger; import io.airlift.log.Logging; import io.pravega.client.admin.StreamManager; + import io.trino.Session; -import io.trino.metadata.Metadata; import io.trino.metadata.QualifiedObjectName; import io.trino.plugin.pravega.PravegaPlugin; -import io.trino.plugin.pravega.PravegaStreamDescription; import io.trino.plugin.pravega.PravegaTableDescriptionSupplier; -import io.trino.plugin.pravega.PravegaTableName; +import io.trino.plugin.pravega.schemamanagement.CompositeSchemaRegistry; +import io.trino.plugin.pravega.schemamanagement.LocalSchemaRegistry; +import io.trino.plugin.pravega.schemamanagement.SchemaRegistry; +import io.trino.plugin.pravega.schemamanagement.SchemaSupplier; +import io.trino.plugin.pravega.util.PravegaTestUtils; import io.trino.plugin.tpch.TpchPlugin; -import io.trino.spi.connector.SchemaTableName; import io.trino.testing.DistributedQueryRunner; import io.trino.testing.QueryRunner; import io.trino.testing.TestingTrinoClient; import io.trino.tpch.TpchTable; import java.net.URI; +import java.util.ArrayList; +import java.util.List; import java.util.Map; -import java.util.Optional; import static io.airlift.testing.Closeables.closeAllSuppress; import static io.airlift.units.Duration.nanosSince; -import static io.trino.plugin.pravega.integration.PravegaTestUtils.getKvStreamDesc; -import static io.trino.plugin.pravega.integration.PravegaTestUtils.getStreamDesc; + import static io.trino.plugin.tpch.TpchMetadata.TINY_SCHEMA_NAME; import static io.trino.testing.TestingSession.testSessionBuilder; import static java.util.Locale.ENGLISH; @@ -71,8 +70,7 @@ public static DistributedQueryRunner createQueryRunner(URI controller, Iterable< queryRunner.installPlugin(new TpchPlugin()); queryRunner.createCatalog("tpch", "tpch"); - PravegaTableDescriptionSupplier tableDescriptionSupplier = - createSchemas(queryRunner.getCoordinator().getMetadata(), tpchTables, keyValueTables); + PravegaTableDescriptionSupplier tableDescriptionSupplier = createTableDescriptionSupplier(tpchTables, keyValueTables); installPlugin(controller, queryRunner, tableDescriptionSupplier); @@ -120,31 +118,24 @@ private static void loadPravegaKVTable(URI controller, StreamManager streamManag log.info("Imported %s in %s", 0, table, nanosSince(start).convertToMostSuccinctTimeUnit()); } - private static PravegaTableDescriptionSupplier createSchemas(Metadata metadata, Iterable> tables, Iterable keyValueTables) + private static PravegaTableDescriptionSupplier createTableDescriptionSupplier(Iterable> tpchTables, Iterable keyValueTables) { - JsonCodec streamDescCodec = new CodecSupplier<>(PravegaStreamDescription.class, metadata).get(); - - Cache schemaCache = CacheBuilder.newBuilder().build(); - Cache> tableCache = CacheBuilder.newBuilder().build(); + List schemaSuppliers = new ArrayList<>(); + List schemaRegistries = new ArrayList<>(); - for (TpchTable table : tables) { - SchemaTableName schemaTableName = new SchemaTableName(TPCH_SCHEMA, table.getTableName()); - PravegaTableName pravegaTableName = new PravegaTableName(schemaTableName); - - schemaCache.put(schemaTableName.getSchemaName(), new Object()); - tableCache.put(pravegaTableName, Optional.of(getStreamDesc(streamDescCodec, "tpch", table.getTableName()))); + if (tpchTables.iterator().hasNext()) { + LocalSchemaRegistry tpch = PravegaTestUtils.localSchemaRegistry("tpch"); + schemaSuppliers.add(tpch); + schemaRegistries.add(tpch); } - for (String table : keyValueTables) { - SchemaTableName schemaTableName = new SchemaTableName(KV_SCHEMA, table); - PravegaTableName pravegaTableName = new PravegaTableName(schemaTableName); - - schemaCache.put(schemaTableName.getSchemaName(), new Object()); - tableCache.put(pravegaTableName, Optional.of(getKvStreamDesc(table))); + if (keyValueTables.iterator().hasNext()) { + LocalSchemaRegistry kv = PravegaTestUtils.localSchemaRegistry("kv"); + schemaSuppliers.add(kv); + schemaRegistries.add(kv); } - // all schemas + tables will be served from these provided caches - return new PravegaTableDescriptionSupplier(null, schemaCache, tableCache); + return new PravegaTableDescriptionSupplier(new CompositeSchemaRegistry(schemaSuppliers, schemaRegistries)); } public static Session createSession() diff --git a/trino/src/test/java/io/trino/plugin/pravega/integration/TestPravegaKeyValue.java b/trino/src/test/java/io/trino/plugin/pravega/integration/TestPravegaKeyValue.java index 24d438f..c9bb6d1 100644 --- a/trino/src/test/java/io/trino/plugin/pravega/integration/TestPravegaKeyValue.java +++ b/trino/src/test/java/io/trino/plugin/pravega/integration/TestPravegaKeyValue.java @@ -25,7 +25,7 @@ import java.util.List; -import static io.trino.plugin.pravega.integration.PravegaTestUtils.getKvStreamDesc; +import static io.trino.plugin.pravega.util.PravegaTestUtils.getKvStreamDesc; import static io.trino.testing.TestingSession.testSessionBuilder; import static org.testng.Assert.assertEquals; diff --git a/trino/src/test/java/io/trino/plugin/pravega/schemamanagement/LocalSchemaRegistryTest.java b/trino/src/test/java/io/trino/plugin/pravega/schemamanagement/LocalSchemaRegistryTest.java new file mode 100644 index 0000000..2e18fb0 --- /dev/null +++ b/trino/src/test/java/io/trino/plugin/pravega/schemamanagement/LocalSchemaRegistryTest.java @@ -0,0 +1,115 @@ +/* + * Copyright (c) Pravega Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.pravega.schemamanagement; + +import io.trino.plugin.pravega.PravegaStreamDescription; +import io.trino.plugin.pravega.PravegaStreamFieldDescription; +import io.trino.plugin.pravega.PravegaStreamFieldGroup; +import io.trino.plugin.pravega.PravegaTableHandle; +import io.trino.plugin.pravega.util.PravegaTestUtils; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.type.BigintType; +import io.trino.spi.type.VarcharType; +import org.testng.annotations.Test; + +import java.util.List; + +import static org.testng.Assert.*; + +@Test +public class LocalSchemaRegistryTest +{ + // uses resources/tpch for table description dir + + @Test + public void testListSchemas() + { + LocalSchemaRegistry schemaRegistry = PravegaTestUtils.localSchemaRegistry("tpch"); + + List schemas = schemaRegistry.listSchemas(); + assertEquals(schemas.size(), 1); + assertEquals(schemas.get(0), "tpch"); + } + + @Test + public void testListTables() + { + LocalSchemaRegistry schemaRegistry = PravegaTestUtils.localSchemaRegistry("tpch"); + + List tables = schemaRegistry.listTables("tpch"); + assertEquals(tables.size(), 8); + + PravegaTableHandle customerTableHandle = + tables.stream().filter(h -> h.getTableName().equals("customer")).findFirst().get(); + assertEquals(customerTableHandle.getSchemaName(), "tpch"); + assertEquals(customerTableHandle.getTableName(), "customer"); + assertEquals(customerTableHandle.getObjectName(), "customer"); + } + + @Test + public void testGetSchema() + { + LocalSchemaRegistry schemaRegistry = PravegaTestUtils.localSchemaRegistry("tpch"); + + List schema = + schemaRegistry.getSchema(new SchemaTableName("tpch", "customer")); + + assertNotNull(schema); + assertEquals(1, schema.size()); + + validateCustomerSchema(schema.get(0)); + } + + @Test + public void testGetTable() + { + LocalSchemaRegistry schemaRegistry = PravegaTestUtils.localSchemaRegistry("tpch"); + + PravegaStreamDescription table = + schemaRegistry.getTable(new SchemaTableName("tpch", "customer")); + assertNotNull(table); + + assertTrue(table.getEvent().isPresent()); + assertEquals(1, table.getEvent().get().size()); + + validateCustomerSchema(table.getEvent().get().get(0)); + } + + @Test + public void testTableDoesNotExist() + { + LocalSchemaRegistry schemaRegistry = PravegaTestUtils.localSchemaRegistry("tpch"); + assertNull(schemaRegistry.getTable(new SchemaTableName("tpch", "abcxyz123"))); + } + + private void validateCustomerSchema(PravegaStreamFieldGroup fieldGroup) + { + // spot check a fiew fields + + assertEquals(fieldGroup.getDataFormat(), "json"); + assertEquals(fieldGroup.getFields().size(), 8); + + PravegaStreamFieldDescription field = fieldGroup.getFields().get(0); + assertEquals(field.getName(), "custkey"); + assertEquals(field.getMapping(), "custkey"); + assertTrue(field.getType() instanceof BigintType); + + field = fieldGroup.getFields().get(6); + assertEquals(field.getName(), "mktsegment"); + assertEquals(field.getMapping(), "mktsegment"); + assertTrue(field.getType() instanceof VarcharType); + } +} diff --git a/trino/src/test/java/io/trino/plugin/pravega/schemamanagement/PravegaSchemaRegistryTest.java b/trino/src/test/java/io/trino/plugin/pravega/schemamanagement/PravegaSchemaRegistryTest.java new file mode 100644 index 0000000..61a094e --- /dev/null +++ b/trino/src/test/java/io/trino/plugin/pravega/schemamanagement/PravegaSchemaRegistryTest.java @@ -0,0 +1,91 @@ +/* + * Copyright (c) Pravega Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.pravega.schemamanagement; + +import com.facebook.presto.common.type.VarcharType; +import com.facebook.presto.spi.SchemaTableName; +import io.pravega.connectors.presto.PravegaStreamDescription; +import io.pravega.connectors.presto.PravegaStreamFieldDescription; +import io.pravega.connectors.presto.PravegaStreamFieldGroup; +import io.pravega.connectors.presto.PravegaTableHandle; +import io.pravega.connectors.presto.util.SchemaRegistryUtil; +import org.testng.annotations.Test; + +import java.util.List; + +import static io.pravega.connectors.presto.util.TestSchemas.EMPLOYEE_AVSC; +import static org.testng.Assert.*; + +@Test +public class PravegaSchemaRegistryTest +{ + @Test + public void testListSchemas() + { + SchemaRegistryUtil schemaRegistryUtil = new SchemaRegistryUtil(); + + schemaRegistryUtil.addSchema("schema1"); + schemaRegistryUtil.addSchema("schema2"); + + List schemas = schemaRegistryUtil.getSchemaRegistry().listSchemas(); + assertEquals(schemas.size(), 2); + assertEquals("schema1", schemas.get(0)); + assertEquals("schema2", schemas.get(1)); + } + + @Test + public void testListTables() + { + SchemaRegistryUtil schemaRegistryUtil = new SchemaRegistryUtil(); + + schemaRegistryUtil.addTable("schema1", "stream1"); + schemaRegistryUtil.addTable("schema2", "stream2"); + // stream starting with '_' is internal/hidden + schemaRegistryUtil.addTable("schema2", "_markStream2"); + + List tables = schemaRegistryUtil.getSchemaRegistry().listTables("schema2"); + assertEquals(tables.size(), 1); + assertEquals("stream2", tables.get(0).getObjectName()); + } + + @Test + public void testGetTable() + { + SchemaRegistryUtil schemaRegistryUtil = new SchemaRegistryUtil(); + + SchemaTableName schemaTableName = new SchemaTableName("hr", "employee"); + schemaRegistryUtil.addAvroSchema(schemaTableName, EMPLOYEE_AVSC); + + SchemaRegistry schemaRegistry = schemaRegistryUtil.getSchemaRegistry(); + + PravegaStreamDescription table = schemaRegistry.getTable(schemaTableName); + + assertNotNull(table); + assertTrue(table.getEvent().isPresent()); + assertEquals(table.getEvent().get().size(), 1); + + PravegaStreamFieldGroup fieldGroup = table.getEvent().get().get(0); + assertEquals(fieldGroup.getFields().size(), 2); + + PravegaStreamFieldDescription field = fieldGroup.getFields().get(0); + assertEquals(field.getName(), "first"); + assertTrue(field.getType() instanceof VarcharType); + + field = fieldGroup.getFields().get(1); + assertEquals(field.getName(), "last"); + assertTrue(field.getType() instanceof VarcharType); + } +} diff --git a/trino/src/test/java/io/trino/plugin/pravega/integration/CodecSupplier.java b/trino/src/test/java/io/trino/plugin/pravega/util/CodecSupplier.java similarity index 98% rename from trino/src/test/java/io/trino/plugin/pravega/integration/CodecSupplier.java rename to trino/src/test/java/io/trino/plugin/pravega/util/CodecSupplier.java index 636c8d1..7c129bb 100644 --- a/trino/src/test/java/io/trino/plugin/pravega/integration/CodecSupplier.java +++ b/trino/src/test/java/io/trino/plugin/pravega/util/CodecSupplier.java @@ -18,7 +18,7 @@ * https://github.com/prestodb/presto/blob/0.247/presto-kafka/src/test/java/com/facebook/presto/kafka/util/CodecSupplier.java */ -package io.trino.plugin.pravega.integration; +package io.trino.plugin.pravega.util; import com.fasterxml.jackson.databind.DeserializationContext; import com.fasterxml.jackson.databind.deser.std.FromStringDeserializer; diff --git a/trino/src/test/java/io/trino/plugin/pravega/util/MockSchemaRegistryClient.java b/trino/src/test/java/io/trino/plugin/pravega/util/MockSchemaRegistryClient.java new file mode 100644 index 0000000..a3c87d7 --- /dev/null +++ b/trino/src/test/java/io/trino/plugin/pravega/util/MockSchemaRegistryClient.java @@ -0,0 +1,173 @@ +/* + * Copyright (c) Pravega Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.pravega.util; + +import io.pravega.schemaregistry.client.SchemaRegistryClient; +import io.pravega.schemaregistry.client.exceptions.RegistryExceptions; +import io.pravega.schemaregistry.contract.data.*; + +import javax.annotation.Nullable; +import java.util.*; + +/** + * this class has a very limited use in unit testing, so many methods not implemented + */ +public class MockSchemaRegistryClient + implements SchemaRegistryClient +{ + private final Map groups = new HashMap<>(); + + private final Map> schemas = new HashMap<>(); + + @Override + public boolean addGroup(String s, GroupProperties groupProperties) throws RegistryExceptions.BadArgumentException, RegistryExceptions.UnauthorizedException + { + return groups.putIfAbsent(s, groupProperties) == null; + } + + @Override + public void removeGroup(String s) throws RegistryExceptions.UnauthorizedException + { + groups.remove(s); + } + + @Override + public Iterator> listGroups() throws RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public GroupProperties getGroupProperties(String s) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + GroupProperties groupProperties = groups.get(s); + if (groupProperties == null) { + throw new RegistryExceptions.ResourceNotFoundException(s); + } + return groupProperties; + } + + @Override + public boolean updateCompatibility(String s, Compatibility compatibility, @Nullable Compatibility compatibility1) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public List getSchemas(String s) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + return schemas.get(s); + } + + @Override + public VersionInfo addSchema(String s, SchemaInfo schemaInfo) throws RegistryExceptions.SchemaValidationFailedException, RegistryExceptions.SerializationMismatchException, RegistryExceptions.MalformedSchemaException, RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + List list = schemas.computeIfAbsent(s, k -> new ArrayList<>()); + VersionInfo versionInfo = new VersionInfo("type", "avro", list.size() + 1, list.size() + 1); + list.add(new SchemaWithVersion(schemaInfo, versionInfo)); + return versionInfo; + } + + @Override + public void deleteSchemaVersion(String s, VersionInfo versionInfo) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public SchemaInfo getSchemaForVersion(String s, VersionInfo versionInfo) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public EncodingInfo getEncodingInfo(String s, EncodingId encodingId) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public EncodingId getEncodingId(String s, VersionInfo versionInfo, String s1) throws RegistryExceptions.CodecTypeNotRegisteredException, RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public SchemaWithVersion getLatestSchemaVersion(String s, @Nullable String s1) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public VersionInfo getVersionForSchema(String s, SchemaInfo schemaInfo) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public List getSchemaVersions(String s, @Nullable String s1) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean validateSchema(String s, SchemaInfo schemaInfo) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean canReadUsing(String s, SchemaInfo schemaInfo) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public List getCodecTypes(String s) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public void addCodecType(String s, CodecType codecType) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public List getGroupHistory(String s) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public Map getSchemaReferences(SchemaInfo schemaInfo) throws RegistryExceptions.ResourceNotFoundException, RegistryExceptions.UnauthorizedException + { + throw new UnsupportedOperationException(); + } + + @Override + public String getNamespace() + { + throw new UnsupportedOperationException(); + } + + @Override + public void close() throws Exception + { + + } +} diff --git a/trino/src/test/java/io/trino/plugin/pravega/util/MockStreamManager.java b/trino/src/test/java/io/trino/plugin/pravega/util/MockStreamManager.java new file mode 100644 index 0000000..96ccc9e --- /dev/null +++ b/trino/src/test/java/io/trino/plugin/pravega/util/MockStreamManager.java @@ -0,0 +1,123 @@ +/* + * Copyright (c) Pravega Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.pravega.util; + +import io.pravega.client.admin.StreamInfo; +import io.pravega.client.admin.StreamManager; +import io.pravega.client.stream.DeleteScopeFailedException; +import io.pravega.client.stream.Stream; +import io.pravega.client.stream.StreamConfiguration; +import io.pravega.client.stream.StreamCut; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +/** + * this class has a very limited use in unit testing, so many methods not implemented + */ +public class MockStreamManager + implements StreamManager { + + private final List scopes = new ArrayList<>(); + + private final List streams = new ArrayList<>(); + + @Override + public boolean createStream(String s, String s1, StreamConfiguration streamConfiguration) + { + Stream stream = Stream.of(s, s1); + return !streams.contains(stream) && streams.add(stream); + } + + @Override + public boolean updateStream(String s, String s1, StreamConfiguration streamConfiguration) + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean truncateStream(String s, String s1, StreamCut streamCut) + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean sealStream(String s, String s1) + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean deleteStream(String s, String s1) + { + throw new UnsupportedOperationException(); + } + + @Override + public Iterator listScopes() + { + return scopes.iterator(); + } + + @Override + public boolean createScope(String s) + { + return !scopes.contains(s) && scopes.add(s); + } + + @Override + public boolean checkScopeExists(String s) + { + return scopes.contains(s); + } + + @Override + public Iterator listStreams(String s) + { + return streams.stream().filter(stream -> stream.getScope().equals(s)).iterator(); + } + + @Override + public boolean checkStreamExists(String s, String s1) + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean deleteScope(String s) + { + return scopes.remove(s); + } + + @Override + public boolean deleteScope(String s, boolean b) throws DeleteScopeFailedException + { + throw new UnsupportedOperationException(); + } + + @Override + public StreamInfo getStreamInfo(String s, String s1) + { + throw new UnsupportedOperationException(); + } + + @Override + public void close() + { + + } +} diff --git a/trino/src/test/java/io/trino/plugin/pravega/integration/PravegaTestUtils.java b/trino/src/test/java/io/trino/plugin/pravega/util/PravegaTestUtils.java similarity index 63% rename from trino/src/test/java/io/trino/plugin/pravega/integration/PravegaTestUtils.java rename to trino/src/test/java/io/trino/plugin/pravega/util/PravegaTestUtils.java index 88e7639..4c1cdd7 100644 --- a/trino/src/test/java/io/trino/plugin/pravega/integration/PravegaTestUtils.java +++ b/trino/src/test/java/io/trino/plugin/pravega/util/PravegaTestUtils.java @@ -13,54 +13,40 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.trino.plugin.pravega.integration; +package io.trino.plugin.pravega.util; -import com.google.common.io.ByteStreams; -import com.google.common.io.CharStreams; import io.airlift.json.JsonCodec; import io.pravega.client.admin.StreamManager; import io.trino.metadata.QualifiedObjectName; import io.trino.plugin.pravega.PravegaStreamDescription; -import io.trino.plugin.pravega.PravegaStreamFieldDescription; -import io.trino.plugin.pravega.PravegaStreamFieldGroup; +import io.trino.plugin.pravega.integration.PravegaKeyValueLoader; +import io.trino.plugin.pravega.integration.PravegaLoader; +import io.trino.plugin.pravega.schemamanagement.LocalSchemaRegistry; +import io.trino.spi.connector.SchemaTableName; import io.trino.testing.TestingTrinoClient; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecordBuilder; import java.io.BufferedReader; +import java.io.File; import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; import java.io.UncheckedIOException; import java.net.URI; -import java.util.ArrayList; -import java.util.List; -import static io.trino.plugin.pravega.util.PravegaStreamDescUtils.mapFieldsFromSchema; import static java.lang.String.format; public final class PravegaTestUtils { private PravegaTestUtils() {} - public static PravegaStreamDescription getStreamDesc(JsonCodec streamDescriptionCodec, String directory, String table) + public static LocalSchemaRegistry localSchemaRegistry(String dir) { - try (InputStream inputStream = PravegaTestUtils.class.getResourceAsStream(String.format("/%s/%s.json", directory, table))) { - return streamDescriptionCodec.fromJson(ByteStreams.toByteArray(inputStream)); - } - catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - public static String readSchema(String directory, String schema) - { - try (InputStreamReader reader = new InputStreamReader(PravegaTestUtils.class.getResourceAsStream(String.format("/%s/%s", directory, schema)))) { - return CharStreams.toString(reader); - } - catch (IOException e) { - throw new UncheckedIOException(e); - } + JsonCodec streamDescCodec = new CodecSupplier<>( + PravegaStreamDescription.class, + FunctionAndTypeManager.createTestFunctionAndTypeManager()).get(); + return new LocalSchemaRegistry(new File("src/test/resources/" + dir).getAbsoluteFile(), streamDescCodec); } public static void loadTpchStream(URI controller, StreamManager streamManager, TestingTrinoClient prestoClient, String schema, String stream, QualifiedObjectName tpchTableName) @@ -86,7 +72,7 @@ public static void loadKeyValueTable(URI controller, StreamManager streamManager avroSchema(tableDesc, 0), avroSchema(tableDesc, 1))) { try (InputStream inputStream = PravegaTestUtils.class.getResourceAsStream(String.format("/kv/%s.records", table)); - BufferedReader reader = new BufferedReader(new InputStreamReader(inputStream))) { + BufferedReader reader = new BufferedReader(new InputStreamReader(inputStream))) { // each line in file is a record, key + value // '|' character separates key fields from values fields // fields separated by ',' @@ -139,34 +125,16 @@ static void setAvroValue(GenericRecordBuilder builder, Schema.Field field, Strin public static PravegaStreamDescription getKvStreamDesc(String table) { - JsonCodec jsonCodec = JsonCodec.jsonCodec(PravegaStreamDescription.class); - - PravegaStreamDescription streamDescription = getStreamDesc(jsonCodec, "kv", table); - streamDescription.getEvent().orElseThrow(IllegalArgumentException::new); - - PravegaStreamFieldGroup keyEvent = streamDescription.getEvent().get().get(0); - keyEvent.getDataSchema().orElseThrow(IllegalArgumentException::new); - String keySchema = PravegaTestUtils.readSchema("kv", keyEvent.getDataSchema().get()); - - PravegaStreamFieldGroup valueEvent = streamDescription.getEvent().get().get(1); - valueEvent.getDataSchema().orElseThrow(IllegalArgumentException::new); - String valueSchema = PravegaTestUtils.readSchema("kv", valueEvent.getDataSchema().get()); - - List keyFields = - mapFieldsFromSchema("key", keyEvent.getDataFormat(), keySchema); - - List valueFields = - mapFieldsFromSchema("value", valueEvent.getDataFormat(), valueSchema); - - List newFieldGroups = new ArrayList<>(2); - newFieldGroups.add(new PravegaStreamFieldGroup(keyEvent, keySchema, keyFields)); - newFieldGroups.add(new PravegaStreamFieldGroup(valueEvent, valueSchema, valueFields)); - - return new PravegaStreamDescription(streamDescription, newFieldGroups); + return localSchemaRegistry("kv").getTable(new SchemaTableName("kv", table)); } public static Schema avroSchema(PravegaStreamDescription streamDescription, int event) { return new Schema.Parser().parse(streamDescription.getEvent().get().get(event).getDataSchema().get()); } + + public static Schema avroSchema(String avroSchemaString) + { + return new Schema.Parser().parse(avroSchemaString); + } } diff --git a/trino/src/test/java/io/trino/plugin/pravega/util/SchemaRegistryUtil.java b/trino/src/test/java/io/trino/plugin/pravega/util/SchemaRegistryUtil.java new file mode 100644 index 0000000..ff6d626 --- /dev/null +++ b/trino/src/test/java/io/trino/plugin/pravega/util/SchemaRegistryUtil.java @@ -0,0 +1,120 @@ +/* + * Copyright (c) Pravega Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.pravega.util; + +import com.google.common.collect.ImmutableMap; +import io.pravega.client.admin.StreamManager; +import io.trino.plugin.pravega.schemamanagement.*; +import io.pravega.schemaregistry.client.SchemaRegistryClient; +import io.pravega.schemaregistry.contract.data.Compatibility; +import io.pravega.schemaregistry.contract.data.GroupProperties; +import io.pravega.schemaregistry.contract.data.SerializationFormat; +import io.pravega.schemaregistry.serializer.avro.schemas.AvroSchema; +import io.trino.spi.connector.SchemaTableName; + +import java.util.ArrayList; +import java.util.List; + +import static io.trino.plugin.pravega.util.PravegaTestUtils.avroSchema; + +/** + * build CompositeSchemaRegistry for use in unit tests + */ +public class SchemaRegistryUtil +{ + private final StreamManager streamManager; + private final SchemaRegistryClient schemaRegistryClient; + private final PravegaSchemaRegistry pravegaSchemaRegistry; + + private final List localSchemaRegistryList; + + public SchemaRegistryUtil() + { + this.streamManager = new MockStreamManager(); + this.schemaRegistryClient = new MockSchemaRegistryClient(); + + this.pravegaSchemaRegistry = new PravegaSchemaRegistry(schemaRegistryClient, streamManager); + + this.localSchemaRegistryList = new ArrayList<>(); + } + + public CompositeSchemaRegistry getSchemaRegistry() + { + List schemaSuppliers = new ArrayList<>(); + List schemaRegistries = new ArrayList<>(); + + localSchemaRegistryList.forEach(lsr -> { + schemaSuppliers.add(lsr); + schemaRegistries.add(lsr); + }); + + schemaSuppliers.add(pravegaSchemaRegistry); + schemaRegistries.add(pravegaSchemaRegistry); + + return new CompositeSchemaRegistry(schemaSuppliers, schemaRegistries); + } + + public void addLocalSchema(String dir) + { + localSchemaRegistryList.add(PravegaTestUtils.localSchemaRegistry(dir)); + } + + public boolean addSchema(String schema) + { + return streamManager.createScope(schema); + } + + public boolean addTable(SchemaTableName schemaTableName) + { + return streamManager.createStream(schemaTableName.getSchemaName(), + schemaTableName.getTableName(), + null); + } + + public boolean addTable(String schema, String stream) + { + return addTable(new SchemaTableName(schema, stream)); + } + + public boolean addTable(SchemaTableName schemaTableName, String schema) + { + if (!addTable(schemaTableName)) { + return false; + } + addAvroSchema(schemaTableName, schema); + return true; + } + + public void addAvroSchema(SchemaTableName schemaTableName, String schema) + { + schemaRegistryClient.addGroup(groupId(schemaTableName), groupProperties(false)); + schemaRegistryClient.addSchema(groupId(schemaTableName), AvroSchema.of(avroSchema(schema)).getSchemaInfo()); + } + + private static GroupProperties groupProperties(boolean inline) + { + return new GroupProperties( + SerializationFormat.Avro, + Compatibility.allowAny(), + false, + ImmutableMap.builder().put(inline ? "inline" : "", "").build()); + } + + private static String groupId(SchemaTableName schemaTableName) + { + return PravegaNameUtils.groupId(schemaTableName.getSchemaName(), schemaTableName.getTableName()); + } +} diff --git a/trino/src/test/java/io/trino/plugin/pravega/util/TestSchemas.java b/trino/src/test/java/io/trino/plugin/pravega/util/TestSchemas.java new file mode 100644 index 0000000..1707f77 --- /dev/null +++ b/trino/src/test/java/io/trino/plugin/pravega/util/TestSchemas.java @@ -0,0 +1,24 @@ +/* + * Copyright (c) Pravega Authors. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.pravega.util; + +public class TestSchemas +{ + private TestSchemas() {} + + public static final String EMPLOYEE_AVSC = + "{\"namespace\": \"io.pravega.avro\",\"type\": \"record\",\"name\": \"Employee\",\"fields\": [{\"name\": \"first\", \"type\": \"string\"},{\"name\": \"last\", \"type\": \"string\"}]}"; +} From 88bb4936ada20a89ff102c94240d27792fe4d28c Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Mon, 27 Sep 2021 11:59:48 -0700 Subject: [PATCH 30/32] change types in trino Signed-off-by: Andrew Robertson --- .../java/io/trino/plugin/pravega/util/PravegaTestUtils.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/trino/src/test/java/io/trino/plugin/pravega/util/PravegaTestUtils.java b/trino/src/test/java/io/trino/plugin/pravega/util/PravegaTestUtils.java index 4c1cdd7..d8c3277 100644 --- a/trino/src/test/java/io/trino/plugin/pravega/util/PravegaTestUtils.java +++ b/trino/src/test/java/io/trino/plugin/pravega/util/PravegaTestUtils.java @@ -17,6 +17,7 @@ import io.airlift.json.JsonCodec; import io.pravega.client.admin.StreamManager; +import io.trino.metadata.MetadataManager; import io.trino.metadata.QualifiedObjectName; import io.trino.plugin.pravega.PravegaStreamDescription; import io.trino.plugin.pravega.integration.PravegaKeyValueLoader; @@ -45,7 +46,7 @@ public static LocalSchemaRegistry localSchemaRegistry(String dir) { JsonCodec streamDescCodec = new CodecSupplier<>( PravegaStreamDescription.class, - FunctionAndTypeManager.createTestFunctionAndTypeManager()).get(); + MetadataManager.createTestMetadataManager()).get(); return new LocalSchemaRegistry(new File("src/test/resources/" + dir).getAbsoluteFile(), streamDescCodec); } From c928ca7d6cdd4588da4df6a65673252059d08eed Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Mon, 27 Sep 2021 12:02:27 -0700 Subject: [PATCH 31/32] missed files Signed-off-by: Andrew Robertson --- .../io/trino/plugin/pravega/PravegaMetadata.java | 3 +-- .../PravegaSchemaRegistryTest.java | 16 ++++++++-------- 2 files changed, 9 insertions(+), 10 deletions(-) diff --git a/trino/src/main/java/io/trino/plugin/pravega/PravegaMetadata.java b/trino/src/main/java/io/trino/plugin/pravega/PravegaMetadata.java index 38fb5b3..b467c35 100644 --- a/trino/src/main/java/io/trino/plugin/pravega/PravegaMetadata.java +++ b/trino/src/main/java/io/trino/plugin/pravega/PravegaMetadata.java @@ -104,8 +104,7 @@ public PravegaTableHandle getTableHandle(ConnectorSession session, SchemaTableNa return null; } - return new PravegaTableHandle(connectorId, - schemaTableName.getSchemaName(), + return new PravegaTableHandle(schemaTableName.getSchemaName(), schemaTableName.getTableName(), table.getObjectName(), table.getObjectType(), diff --git a/trino/src/test/java/io/trino/plugin/pravega/schemamanagement/PravegaSchemaRegistryTest.java b/trino/src/test/java/io/trino/plugin/pravega/schemamanagement/PravegaSchemaRegistryTest.java index 61a094e..be8fd74 100644 --- a/trino/src/test/java/io/trino/plugin/pravega/schemamanagement/PravegaSchemaRegistryTest.java +++ b/trino/src/test/java/io/trino/plugin/pravega/schemamanagement/PravegaSchemaRegistryTest.java @@ -15,18 +15,18 @@ */ package io.trino.plugin.pravega.schemamanagement; -import com.facebook.presto.common.type.VarcharType; -import com.facebook.presto.spi.SchemaTableName; -import io.pravega.connectors.presto.PravegaStreamDescription; -import io.pravega.connectors.presto.PravegaStreamFieldDescription; -import io.pravega.connectors.presto.PravegaStreamFieldGroup; -import io.pravega.connectors.presto.PravegaTableHandle; -import io.pravega.connectors.presto.util.SchemaRegistryUtil; +import io.trino.plugin.pravega.PravegaStreamDescription; +import io.trino.plugin.pravega.PravegaStreamFieldDescription; +import io.trino.plugin.pravega.PravegaStreamFieldGroup; +import io.trino.plugin.pravega.PravegaTableHandle; +import io.trino.plugin.pravega.util.SchemaRegistryUtil; +import io.trino.spi.connector.SchemaTableName; +import io.trino.spi.type.VarcharType; import org.testng.annotations.Test; import java.util.List; -import static io.pravega.connectors.presto.util.TestSchemas.EMPLOYEE_AVSC; +import static io.trino.plugin.pravega.util.TestSchemas.EMPLOYEE_AVSC; import static org.testng.Assert.*; @Test From 31ba8be97a9cdf0bba19b683dfa96b76633d13d8 Mon Sep 17 00:00:00 2001 From: Andrew Robertson Date: Mon, 27 Sep 2021 12:06:18 -0700 Subject: [PATCH 32/32] trino: add missing resource files; use test naming scheme Signed-off-by: Andrew Robertson --- ...erTest.java => TestPravegaTableDescriptionSupplier.java} | 2 +- ...SchemaRegistryTest.java => TestLocalSchemaRegistry.java} | 2 +- ...hemaRegistryTest.java => TestPravegaSchemaRegistry.java} | 2 +- trino/src/test/resources/ut/ut.multiexplicit.json | 6 ++++++ trino/src/test/resources/ut/ut.multiregex.json | 5 +++++ 5 files changed, 14 insertions(+), 3 deletions(-) rename trino/src/test/java/io/trino/plugin/pravega/{PravegaTableDescriptionSupplierTest.java => TestPravegaTableDescriptionSupplier.java} (98%) rename trino/src/test/java/io/trino/plugin/pravega/schemamanagement/{LocalSchemaRegistryTest.java => TestLocalSchemaRegistry.java} (99%) rename trino/src/test/java/io/trino/plugin/pravega/schemamanagement/{PravegaSchemaRegistryTest.java => TestPravegaSchemaRegistry.java} (98%) create mode 100644 trino/src/test/resources/ut/ut.multiexplicit.json create mode 100644 trino/src/test/resources/ut/ut.multiregex.json diff --git a/trino/src/test/java/io/trino/plugin/pravega/PravegaTableDescriptionSupplierTest.java b/trino/src/test/java/io/trino/plugin/pravega/TestPravegaTableDescriptionSupplier.java similarity index 98% rename from trino/src/test/java/io/trino/plugin/pravega/PravegaTableDescriptionSupplierTest.java rename to trino/src/test/java/io/trino/plugin/pravega/TestPravegaTableDescriptionSupplier.java index c237dfa..06d9e77 100644 --- a/trino/src/test/java/io/trino/plugin/pravega/PravegaTableDescriptionSupplierTest.java +++ b/trino/src/test/java/io/trino/plugin/pravega/TestPravegaTableDescriptionSupplier.java @@ -25,7 +25,7 @@ import static org.testng.Assert.*; @Test -public class PravegaTableDescriptionSupplierTest +public class TestPravegaTableDescriptionSupplier { private static final String SCHEMA = "ut"; diff --git a/trino/src/test/java/io/trino/plugin/pravega/schemamanagement/LocalSchemaRegistryTest.java b/trino/src/test/java/io/trino/plugin/pravega/schemamanagement/TestLocalSchemaRegistry.java similarity index 99% rename from trino/src/test/java/io/trino/plugin/pravega/schemamanagement/LocalSchemaRegistryTest.java rename to trino/src/test/java/io/trino/plugin/pravega/schemamanagement/TestLocalSchemaRegistry.java index 2e18fb0..24f7674 100644 --- a/trino/src/test/java/io/trino/plugin/pravega/schemamanagement/LocalSchemaRegistryTest.java +++ b/trino/src/test/java/io/trino/plugin/pravega/schemamanagement/TestLocalSchemaRegistry.java @@ -30,7 +30,7 @@ import static org.testng.Assert.*; @Test -public class LocalSchemaRegistryTest +public class TestLocalSchemaRegistry { // uses resources/tpch for table description dir diff --git a/trino/src/test/java/io/trino/plugin/pravega/schemamanagement/PravegaSchemaRegistryTest.java b/trino/src/test/java/io/trino/plugin/pravega/schemamanagement/TestPravegaSchemaRegistry.java similarity index 98% rename from trino/src/test/java/io/trino/plugin/pravega/schemamanagement/PravegaSchemaRegistryTest.java rename to trino/src/test/java/io/trino/plugin/pravega/schemamanagement/TestPravegaSchemaRegistry.java index be8fd74..c01ff15 100644 --- a/trino/src/test/java/io/trino/plugin/pravega/schemamanagement/PravegaSchemaRegistryTest.java +++ b/trino/src/test/java/io/trino/plugin/pravega/schemamanagement/TestPravegaSchemaRegistry.java @@ -30,7 +30,7 @@ import static org.testng.Assert.*; @Test -public class PravegaSchemaRegistryTest +public class TestPravegaSchemaRegistry { @Test public void testListSchemas() diff --git a/trino/src/test/resources/ut/ut.multiexplicit.json b/trino/src/test/resources/ut/ut.multiexplicit.json new file mode 100644 index 0000000..bf6b50c --- /dev/null +++ b/trino/src/test/resources/ut/ut.multiexplicit.json @@ -0,0 +1,6 @@ +{ + "schemaName": "ut", + "tableName": "multiexplicit", + "objectName": "multiexplicit", + "objectArgs": ["stream1", "stream3"] +} \ No newline at end of file diff --git a/trino/src/test/resources/ut/ut.multiregex.json b/trino/src/test/resources/ut/ut.multiregex.json new file mode 100644 index 0000000..911ee32 --- /dev/null +++ b/trino/src/test/resources/ut/ut.multiregex.json @@ -0,0 +1,5 @@ +{ + "schemaName": "ut", + "tableName": "multiregex", + "objectName": "stream[0-9]" +} \ No newline at end of file