From 1012a01b70a3217fd0bd82c374a4652f9ff5817f Mon Sep 17 00:00:00 2001 From: Brian Hulette Date: Thu, 13 Jun 2019 15:51:45 -0700 Subject: [PATCH 1/2] Update portable schema representation and java SchemaTranslation. Add SchemaTranslationTest --- .../src/main/proto/beam_runner_api.proto | 35 +++--- .../core/construction/SchemaTranslation.java | 108 +++++++++++------- .../construction/SchemaTranslationTest.java | 89 +++++++++++++++ 3 files changed, 170 insertions(+), 62 deletions(-) create mode 100644 runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SchemaTranslationTest.java diff --git a/model/pipeline/src/main/proto/beam_runner_api.proto b/model/pipeline/src/main/proto/beam_runner_api.proto index 1c6768f2c963..5cdefc4ee780 100644 --- a/model/pipeline/src/main/proto/beam_runner_api.proto +++ b/model/pipeline/src/main/proto/beam_runner_api.proto @@ -647,29 +647,26 @@ message StandardCoders { // Experimental: A representation of a Beam Schema. message Schema { - enum TypeName { + enum AtomicType { BYTE = 0; INT16 = 1; INT32 = 2; INT64 = 3; - DECIMAL = 4; - FLOAT = 5; - DOUBLE = 6; - STRING = 7; - DATETIME = 8; - BOOLEAN = 9; - BYTES = 10; - ARRAY = 11; - MAP = 13; - ROW = 14; - LOGICAL_TYPE = 15; + FLOAT = 4; + DOUBLE = 5; + STRING = 6; + BOOLEAN = 7; + BYTES = 8; + } + + message ArrayType { + FieldType element_type = 1; } message LogicalType { - string id = 1; + string urn = 1; string args = 2; - FieldType base_type = 3; - bytes serialized_class = 4; + FieldType representation = 3; } message MapType { @@ -678,12 +675,12 @@ message Schema { } message FieldType { - TypeName type_name = 1; - bool nullable = 2; + bool nullable = 1; oneof type_info { - FieldType collection_element_type = 3; + AtomicType atomic_type = 2; + ArrayType array_type = 3; MapType map_type = 4; - Schema row_schema = 5; + Schema row_type = 5; LogicalType logical_type = 6; } } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SchemaTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SchemaTranslation.java index 90af770420dd..1b73b155b1a7 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SchemaTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SchemaTranslation.java @@ -25,33 +25,29 @@ import org.apache.beam.sdk.schemas.Schema.FieldType; import org.apache.beam.sdk.schemas.Schema.LogicalType; import org.apache.beam.sdk.schemas.Schema.TypeName; -import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.vendor.grpc.v1p13p1.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.BiMap; import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableBiMap; import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Maps; /** Utility methods for translating schemas. */ public class SchemaTranslation { - private static final BiMap TYPE_NAME_MAPPING = - ImmutableBiMap.builder() - .put(TypeName.BYTE, RunnerApi.Schema.TypeName.BYTE) - .put(TypeName.INT16, RunnerApi.Schema.TypeName.INT16) - .put(TypeName.INT32, RunnerApi.Schema.TypeName.INT32) - .put(TypeName.INT64, RunnerApi.Schema.TypeName.INT64) - .put(TypeName.DECIMAL, RunnerApi.Schema.TypeName.DECIMAL) - .put(TypeName.FLOAT, RunnerApi.Schema.TypeName.FLOAT) - .put(TypeName.DOUBLE, RunnerApi.Schema.TypeName.DOUBLE) - .put(TypeName.STRING, RunnerApi.Schema.TypeName.STRING) - .put(TypeName.DATETIME, RunnerApi.Schema.TypeName.DATETIME) - .put(TypeName.BOOLEAN, RunnerApi.Schema.TypeName.BOOLEAN) - .put(TypeName.BYTES, RunnerApi.Schema.TypeName.BYTES) - .put(TypeName.ARRAY, RunnerApi.Schema.TypeName.ARRAY) - .put(TypeName.MAP, RunnerApi.Schema.TypeName.MAP) - .put(TypeName.ROW, RunnerApi.Schema.TypeName.ROW) - .put(TypeName.LOGICAL_TYPE, RunnerApi.Schema.TypeName.LOGICAL_TYPE) + + private static final BiMap ATOMIC_TYPE_MAPPING = + ImmutableBiMap.builder() + .put(TypeName.BYTE, RunnerApi.Schema.AtomicType.BYTE) + .put(TypeName.INT16, RunnerApi.Schema.AtomicType.INT16) + .put(TypeName.INT32, RunnerApi.Schema.AtomicType.INT32) + .put(TypeName.INT64, RunnerApi.Schema.AtomicType.INT64) + .put(TypeName.FLOAT, RunnerApi.Schema.AtomicType.FLOAT) + .put(TypeName.DOUBLE, RunnerApi.Schema.AtomicType.DOUBLE) + .put(TypeName.STRING, RunnerApi.Schema.AtomicType.STRING) + .put(TypeName.BOOLEAN, RunnerApi.Schema.AtomicType.BOOLEAN) + .put(TypeName.BYTES, RunnerApi.Schema.AtomicType.BYTES) .build(); + private static final String URN_BEAM_LOGICAL_DATETIME = "urn:beam:logical:datetime"; + private static final String URN_BEAM_LOGICAL_DECIMAL = "urn:beam:logical:decimal"; + public static RunnerApi.Schema toProto(Schema schema) { String uuid = schema.getUUID() != null ? schema.getUUID().toString() : ""; RunnerApi.Schema.Builder builder = RunnerApi.Schema.newBuilder().setId(uuid); @@ -77,16 +73,17 @@ private static RunnerApi.Schema.Field toProto(Field field, int fieldId, int posi } private static RunnerApi.Schema.FieldType toProto(FieldType fieldType) { - RunnerApi.Schema.FieldType.Builder builder = - RunnerApi.Schema.FieldType.newBuilder() - .setTypeName(TYPE_NAME_MAPPING.get(fieldType.getTypeName())); + RunnerApi.Schema.FieldType.Builder builder = RunnerApi.Schema.FieldType.newBuilder(); switch (fieldType.getTypeName()) { case ROW: - builder.setRowSchema(toProto(fieldType.getRowSchema())); + fieldType.getRowSchema(); + builder.setRowType(toProto(fieldType.getRowSchema())); break; case ARRAY: - builder.setCollectionElementType(toProto(fieldType.getCollectionElementType())); + builder.setArrayType( + RunnerApi.Schema.ArrayType.newBuilder() + .setElementType(toProto(fieldType.getCollectionElementType()))); break; case MAP: @@ -101,15 +98,29 @@ private static RunnerApi.Schema.FieldType toProto(FieldType fieldType) { LogicalType logicalType = fieldType.getLogicalType(); builder.setLogicalType( RunnerApi.Schema.LogicalType.newBuilder() - .setId(logicalType.getIdentifier()) + .setUrn(logicalType.getIdentifier()) .setArgs(logicalType.getArgument()) - .setBaseType(toProto(logicalType.getBaseType())) - .setSerializedClass( - ByteString.copyFrom(SerializableUtils.serializeToByteArray(logicalType))) + .setRepresentation(toProto(logicalType.getBaseType())) + .build()); + break; + // Special-case for DATETIME and DECIMAL which are logical types in portable representation, + // but not yet in Java. (BEAM-7554) + case DATETIME: + builder.setLogicalType( + RunnerApi.Schema.LogicalType.newBuilder() + .setUrn(URN_BEAM_LOGICAL_DATETIME) + .setRepresentation(toProto(FieldType.INT64)) + .build()); + break; + case DECIMAL: + builder.setLogicalType( + RunnerApi.Schema.LogicalType.newBuilder() + .setUrn(URN_BEAM_LOGICAL_DECIMAL) + .setRepresentation(toProto(FieldType.BYTES)) .build()); break; - default: + builder.setAtomicType(ATOMIC_TYPE_MAPPING.get(fieldType.getTypeName())); break; } builder.setNullable(fieldType.getNullable()); @@ -139,32 +150,43 @@ private static Field fieldFromProto(RunnerApi.Schema.Field protoField) { } private static FieldType fieldTypeFromProto(RunnerApi.Schema.FieldType protoFieldType) { - TypeName typeName = TYPE_NAME_MAPPING.inverse().get(protoFieldType.getTypeName()); FieldType fieldType; - switch (typeName) { - case ROW: - fieldType = FieldType.row(fromProto(protoFieldType.getRowSchema())); + switch (protoFieldType.getTypeInfoCase()) { + case ATOMIC_TYPE: + TypeName typeName = ATOMIC_TYPE_MAPPING.inverse().get(protoFieldType.getAtomicType()); + fieldType = FieldType.of(typeName); break; - case ARRAY: - fieldType = FieldType.array(fieldTypeFromProto(protoFieldType.getCollectionElementType())); + case ROW_TYPE: + fieldType = FieldType.row(fromProto(protoFieldType.getRowType())); break; - case MAP: + case ARRAY_TYPE: + fieldType = + FieldType.array(fieldTypeFromProto(protoFieldType.getArrayType().getElementType())); + break; + case MAP_TYPE: fieldType = FieldType.map( fieldTypeFromProto(protoFieldType.getMapType().getKeyType()), fieldTypeFromProto(protoFieldType.getMapType().getValueType())); break; case LOGICAL_TYPE: - LogicalType logicalType = - (LogicalType) - SerializableUtils.deserializeFromByteArray( - protoFieldType.getLogicalType().getSerializedClass().toByteArray(), - "logicalType"); - fieldType = FieldType.logicalType(logicalType); + // Special-case for DATETIME and DECIMAL which are logical types in portable representation, + // but not yet in Java. (BEAM-7554) + String urn = protoFieldType.getLogicalType().getUrn(); + if (urn.equals(URN_BEAM_LOGICAL_DATETIME)) { + fieldType = FieldType.DATETIME; + } else if (urn.equals(URN_BEAM_LOGICAL_DECIMAL)) { + fieldType = FieldType.DECIMAL; + } else { + // TODO: Look up logical type class by URN. + throw new IllegalArgumentException("Decoding logical types is not yet supported."); + } break; default: - fieldType = FieldType.of(typeName); + throw new IllegalArgumentException( + "Unexpected type_info: " + protoFieldType.getTypeInfoCase()); } + if (protoFieldType.getNullable()) { fieldType = fieldType.withNullable(true); } diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SchemaTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SchemaTranslationTest.java new file mode 100644 index 000000000000..282030658c12 --- /dev/null +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SchemaTranslationTest.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.beam.runners.core.construction; + +import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertThat; + +import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.Schema.Field; +import org.apache.beam.sdk.schemas.Schema.FieldType; +import org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableList; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameter; +import org.junit.runners.Parameterized.Parameters; + +/** Tests for {@link SchemaTranslation}. */ +public class SchemaTranslationTest { + + /** Tests round-trip proto encodings for {@link Schema}. */ + @RunWith(Parameterized.class) + public static class ToFromProtoTest { + @Parameters(name = "{index}: {0}") + public static Iterable data() { + return ImmutableList.builder() + .add(Schema.of(Field.of("string", FieldType.STRING))) + .add( + Schema.of( + Field.of("boolean", FieldType.BOOLEAN), + Field.of("byte", FieldType.BYTE), + Field.of("int16", FieldType.INT16), + Field.of("int32", FieldType.INT32), + Field.of("int64", FieldType.INT64))) + .add( + Schema.of( + Field.of( + "row", + FieldType.row( + Schema.of( + Field.of("foo", FieldType.STRING), + Field.of("bar", FieldType.DOUBLE), + Field.of("baz", FieldType.BOOLEAN)))))) + .add( + Schema.of( + Field.of( + "array(array(int64)))", + FieldType.array(FieldType.array(FieldType.INT64.withNullable(true)))))) + .add( + Schema.of( + Field.of("nullable", FieldType.STRING.withNullable(true)), + Field.of("non_nullable", FieldType.STRING.withNullable(false)))) + .add( + Schema.of( + Field.of("decimal", FieldType.DECIMAL), Field.of("datetime", FieldType.DATETIME))) + // Test for when Logical types are supported + // .add(Schema.of(Field.of("logical", + // FieldType.logicalType(LogicalTypes.FixedBytes.of(24))))) + .build(); + } + + @Parameter(0) + public Schema schema; + + @Test + public void toAndFromProto() throws Exception { + RunnerApi.Schema schemaProto = SchemaTranslation.toProto(schema); + + Schema decodedSchema = SchemaTranslation.fromProto(schemaProto); + assertThat(decodedSchema, equalTo(schema)); + } + } +} From aaa70c8866e189ac6d85157152373e601acf3c2a Mon Sep 17 00:00:00 2001 From: Brian Hulette Date: Wed, 19 Jun 2019 13:28:22 -0700 Subject: [PATCH 2/2] !fixup - add UNSPECIFIED to AtomicType - add RowType - urn:beam:logical:(.*) -> beam:fieldtype:\1 --- .../src/main/proto/beam_runner_api.proto | 35 +++++++++++-------- .../core/construction/SchemaTranslation.java | 10 +++--- 2 files changed, 25 insertions(+), 20 deletions(-) diff --git a/model/pipeline/src/main/proto/beam_runner_api.proto b/model/pipeline/src/main/proto/beam_runner_api.proto index 5cdefc4ee780..13de5b5d18dc 100644 --- a/model/pipeline/src/main/proto/beam_runner_api.proto +++ b/model/pipeline/src/main/proto/beam_runner_api.proto @@ -648,39 +648,44 @@ message StandardCoders { // Experimental: A representation of a Beam Schema. message Schema { enum AtomicType { - BYTE = 0; - INT16 = 1; - INT32 = 2; - INT64 = 3; - FLOAT = 4; - DOUBLE = 5; - STRING = 6; - BOOLEAN = 7; - BYTES = 8; + UNSPECIFIED = 0; + BYTE = 1; + INT16 = 2; + INT32 = 3; + INT64 = 4; + FLOAT = 5; + DOUBLE = 6; + STRING = 7; + BOOLEAN = 8; + BYTES = 9; } message ArrayType { FieldType element_type = 1; } + message MapType { + FieldType key_type = 1; + FieldType value_type = 2; + } + + message RowType { + Schema schema = 1; + } + message LogicalType { string urn = 1; string args = 2; FieldType representation = 3; } - message MapType { - FieldType key_type = 1; - FieldType value_type = 2; - } - message FieldType { bool nullable = 1; oneof type_info { AtomicType atomic_type = 2; ArrayType array_type = 3; MapType map_type = 4; - Schema row_type = 5; + RowType row_type = 5; LogicalType logical_type = 6; } } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SchemaTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SchemaTranslation.java index 1b73b155b1a7..313b20e3f838 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SchemaTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SchemaTranslation.java @@ -45,8 +45,8 @@ public class SchemaTranslation { .put(TypeName.BYTES, RunnerApi.Schema.AtomicType.BYTES) .build(); - private static final String URN_BEAM_LOGICAL_DATETIME = "urn:beam:logical:datetime"; - private static final String URN_BEAM_LOGICAL_DECIMAL = "urn:beam:logical:decimal"; + private static final String URN_BEAM_LOGICAL_DATETIME = "beam:fieldtype:datetime"; + private static final String URN_BEAM_LOGICAL_DECIMAL = "beam:fieldtype:decimal"; public static RunnerApi.Schema toProto(Schema schema) { String uuid = schema.getUUID() != null ? schema.getUUID().toString() : ""; @@ -76,8 +76,8 @@ private static RunnerApi.Schema.FieldType toProto(FieldType fieldType) { RunnerApi.Schema.FieldType.Builder builder = RunnerApi.Schema.FieldType.newBuilder(); switch (fieldType.getTypeName()) { case ROW: - fieldType.getRowSchema(); - builder.setRowType(toProto(fieldType.getRowSchema())); + builder.setRowType( + RunnerApi.Schema.RowType.newBuilder().setSchema(toProto(fieldType.getRowSchema()))); break; case ARRAY: @@ -157,7 +157,7 @@ private static FieldType fieldTypeFromProto(RunnerApi.Schema.FieldType protoFiel fieldType = FieldType.of(typeName); break; case ROW_TYPE: - fieldType = FieldType.row(fromProto(protoFieldType.getRowType())); + fieldType = FieldType.row(fromProto(protoFieldType.getRowType().getSchema())); break; case ARRAY_TYPE: fieldType =