diff --git a/model/pipeline/src/main/proto/beam_runner_api.proto b/model/pipeline/src/main/proto/beam_runner_api.proto index aa3184d76f67..736bcdc17bad 100644 --- a/model/pipeline/src/main/proto/beam_runner_api.proto +++ b/model/pipeline/src/main/proto/beam_runner_api.proto @@ -645,61 +645,6 @@ message StandardCoders { } } -// Experimental: A representation of a Beam Schema. -message Schema { - enum TypeName { - 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; - } - - message LogicalType { - string id = 1; - string args = 2; - FieldType base_type = 3; - bytes serialized_class = 4; - } - - message MapType { - FieldType key_type = 1; - FieldType value_type = 2; - } - - message FieldType { - TypeName type_name = 1; - bool nullable = 2; - oneof type_info { - FieldType collection_element_type = 3; - MapType map_type = 4; - Schema row_schema = 5; - LogicalType logical_type = 6; - } - } - - message Field { - string name = 1; - string description = 2; - FieldType type = 3; - int32 id = 4; - int32 encoding_position = 5; - } - - repeated Field fields = 1; - string id = 2; -} - // A windowing strategy describes the window function, triggering, allowed // lateness, and accumulation mode for a PCollection. // diff --git a/model/pipeline/src/main/proto/schema.proto b/model/pipeline/src/main/proto/schema.proto new file mode 100644 index 000000000000..e420e3c91bac --- /dev/null +++ b/model/pipeline/src/main/proto/schema.proto @@ -0,0 +1,85 @@ +/* + * 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. + */ + +// ** Experimental ** +// Protocol Buffers describing Beam Schemas, a portable representation for +// complex types. + +syntax = "proto3"; + +package org.apache.beam.model.pipeline.v1; + +option go_package = "pipeline_v1"; +option java_package = "org.apache.beam.model.pipeline.v1"; +option java_outer_classname = "SchemaApi"; + +message Schema { + repeated Field fields = 1; + string id = 2; +} + +message Field { + string name = 1; + string description = 2; + FieldType type = 3; + int32 id = 4; + int32 encoding_position = 5; +} + +message FieldType { + bool nullable = 1; + oneof type_info { + AtomicType atomic_type = 2; + ArrayType array_type = 3; + MapType map_type = 4; + RowType row_type = 5; + LogicalType logical_type = 6; + } +} + +enum AtomicType { + 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; + bytes payload = 2; + FieldType representation = 3; +} 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 26b154d12ed2..6d6eb572480f 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 @@ -19,7 +19,7 @@ import java.util.Map; import java.util.UUID; -import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.model.pipeline.v1.SchemaApi; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.Field; import org.apache.beam.sdk.schemas.Schema.FieldType; @@ -27,37 +27,21 @@ import org.apache.beam.sdk.schemas.Schema.TypeName; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.BiMap; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableBiMap; import org.apache.beam.vendor.guava.v26_0_jre.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) - .build(); - - public static RunnerApi.Schema toProto(Schema schema) { + + private static final String URN_BEAM_LOGICAL_DATETIME = "beam:logical_type:datetime:v1"; + private static final String URN_BEAM_LOGICAL_DECIMAL = "beam:logical_type:decimal:v1"; + private static final String URN_BEAM_LOGICAL_JAVASDK = "beam:logical_type:javasdk:v1"; + + public static SchemaApi.Schema schemaToProto(Schema schema) { String uuid = schema.getUUID() != null ? schema.getUUID().toString() : ""; - RunnerApi.Schema.Builder builder = RunnerApi.Schema.newBuilder().setId(uuid); + SchemaApi.Schema.Builder builder = SchemaApi.Schema.newBuilder().setId(uuid); for (Field field : schema.getFields()) { - RunnerApi.Schema.Field protoField = - toProto( + SchemaApi.Field protoField = + fieldToProto( field, schema.indexOf(field.getName()), schema.getEncodingPositions().get(field.getName())); @@ -66,60 +50,103 @@ public static RunnerApi.Schema toProto(Schema schema) { return builder.build(); } - private static RunnerApi.Schema.Field toProto(Field field, int fieldId, int position) { - return RunnerApi.Schema.Field.newBuilder() + private static SchemaApi.Field fieldToProto(Field field, int fieldId, int position) { + return SchemaApi.Field.newBuilder() .setName(field.getName()) .setDescription(field.getDescription()) - .setType(toProto(field.getType())) + .setType(fieldTypeToProto(field.getType())) .setId(fieldId) .setEncodingPosition(position) .build(); } - private static RunnerApi.Schema.FieldType toProto(FieldType fieldType) { - RunnerApi.Schema.FieldType.Builder builder = - RunnerApi.Schema.FieldType.newBuilder() - .setTypeName(TYPE_NAME_MAPPING.get(fieldType.getTypeName())); + private static SchemaApi.FieldType fieldTypeToProto(FieldType fieldType) { + SchemaApi.FieldType.Builder builder = SchemaApi.FieldType.newBuilder(); switch (fieldType.getTypeName()) { case ROW: - builder.setRowSchema(toProto(fieldType.getRowSchema())); + builder.setRowType( + SchemaApi.RowType.newBuilder().setSchema(schemaToProto(fieldType.getRowSchema()))); break; case ARRAY: - builder.setCollectionElementType(toProto(fieldType.getCollectionElementType())); + builder.setArrayType( + SchemaApi.ArrayType.newBuilder() + .setElementType(fieldTypeToProto(fieldType.getCollectionElementType()))); break; case MAP: builder.setMapType( - RunnerApi.Schema.MapType.newBuilder() - .setKeyType(toProto(fieldType.getMapKeyType())) - .setValueType(toProto(fieldType.getMapValueType())) + SchemaApi.MapType.newBuilder() + .setKeyType(fieldTypeToProto(fieldType.getMapKeyType())) + .setValueType(fieldTypeToProto(fieldType.getMapValueType())) .build()); break; case LOGICAL_TYPE: LogicalType logicalType = fieldType.getLogicalType(); builder.setLogicalType( - RunnerApi.Schema.LogicalType.newBuilder() - .setId(logicalType.getIdentifier()) - .setArgs(logicalType.getArgument()) - .setBaseType(toProto(logicalType.getBaseType())) - .setSerializedClass( + SchemaApi.LogicalType.newBuilder() + // TODO(BEAM-7855): "javasdk" types should only be a last resort. Types defined in + // Beam should have their own URN, and there should be a mechanism for users to + // register their own types by URN. + .setUrn(URN_BEAM_LOGICAL_JAVASDK) + .setPayload( ByteString.copyFrom(SerializableUtils.serializeToByteArray(logicalType))) + .setRepresentation(fieldTypeToProto(logicalType.getBaseType())) .build()); break; - - default: + // Special-case for DATETIME and DECIMAL which are logical types in portable representation, + // but not yet in Java. (BEAM-7554) + case DATETIME: + builder.setLogicalType( + SchemaApi.LogicalType.newBuilder() + .setUrn(URN_BEAM_LOGICAL_DATETIME) + .setRepresentation(fieldTypeToProto(FieldType.INT64)) + .build()); + break; + case DECIMAL: + builder.setLogicalType( + SchemaApi.LogicalType.newBuilder() + .setUrn(URN_BEAM_LOGICAL_DECIMAL) + .setRepresentation(fieldTypeToProto(FieldType.BYTES)) + .build()); + break; + case BYTE: + builder.setAtomicType(SchemaApi.AtomicType.BYTE); + break; + case INT16: + builder.setAtomicType(SchemaApi.AtomicType.INT16); + break; + case INT32: + builder.setAtomicType(SchemaApi.AtomicType.INT32); + break; + case INT64: + builder.setAtomicType(SchemaApi.AtomicType.INT64); + break; + case FLOAT: + builder.setAtomicType(SchemaApi.AtomicType.FLOAT); + break; + case DOUBLE: + builder.setAtomicType(SchemaApi.AtomicType.DOUBLE); + break; + case STRING: + builder.setAtomicType(SchemaApi.AtomicType.STRING); + break; + case BOOLEAN: + builder.setAtomicType(SchemaApi.AtomicType.BOOLEAN); + break; + case BYTES: + builder.setAtomicType(SchemaApi.AtomicType.BYTES); break; } builder.setNullable(fieldType.getNullable()); return builder.build(); } - public static Schema fromProto(RunnerApi.Schema protoSchema) { + public static Schema fromProto(SchemaApi.Schema protoSchema) { Schema.Builder builder = Schema.builder(); Map encodingLocationMap = Maps.newHashMap(); - for (RunnerApi.Schema.Field protoField : protoSchema.getFieldsList()) { + for (SchemaApi.Field protoField : protoSchema.getFieldsList()) { Field field = fieldFromProto(protoField); builder.addField(field); encodingLocationMap.put(protoField.getName(), protoField.getEncodingPosition()); @@ -133,41 +160,76 @@ public static Schema fromProto(RunnerApi.Schema protoSchema) { return schema; } - private static Field fieldFromProto(RunnerApi.Schema.Field protoField) { + private static Field fieldFromProto(SchemaApi.Field protoField) { return Field.of(protoField.getName(), fieldTypeFromProto(protoField.getType())) .withDescription(protoField.getDescription()); } - 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())); - break; - case ARRAY: - fieldType = FieldType.array(fieldTypeFromProto(protoFieldType.getCollectionElementType())); - break; - case MAP: - 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); - break; - default: - fieldType = FieldType.of(typeName); - } + private static FieldType fieldTypeFromProto(SchemaApi.FieldType protoFieldType) { + FieldType fieldType = fieldTypeFromProtoWithoutNullable(protoFieldType); + if (protoFieldType.getNullable()) { fieldType = fieldType.withNullable(true); } + return fieldType; } + + private static FieldType fieldTypeFromProtoWithoutNullable(SchemaApi.FieldType protoFieldType) { + switch (protoFieldType.getTypeInfoCase()) { + case ATOMIC_TYPE: + switch (protoFieldType.getAtomicType()) { + case BYTE: + return FieldType.of(TypeName.BYTE); + case INT16: + return FieldType.of(TypeName.INT16); + case INT32: + return FieldType.of(TypeName.INT32); + case INT64: + return FieldType.of(TypeName.INT64); + case FLOAT: + return FieldType.of(TypeName.FLOAT); + case DOUBLE: + return FieldType.of(TypeName.DOUBLE); + case STRING: + return FieldType.of(TypeName.STRING); + case BOOLEAN: + return FieldType.of(TypeName.BOOLEAN); + case BYTES: + return FieldType.of(TypeName.BYTES); + case UNSPECIFIED: + throw new IllegalArgumentException("Encountered UNSPECIFIED AtomicType"); + default: + throw new IllegalArgumentException( + "Encountered unknown AtomicType: " + protoFieldType.getAtomicType()); + } + case ROW_TYPE: + return FieldType.row(fromProto(protoFieldType.getRowType().getSchema())); + case ARRAY_TYPE: + return FieldType.array(fieldTypeFromProto(protoFieldType.getArrayType().getElementType())); + case MAP_TYPE: + return FieldType.map( + fieldTypeFromProto(protoFieldType.getMapType().getKeyType()), + fieldTypeFromProto(protoFieldType.getMapType().getValueType())); + case LOGICAL_TYPE: + // 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)) { + return FieldType.DATETIME; + } else if (urn.equals(URN_BEAM_LOGICAL_DECIMAL)) { + return FieldType.DECIMAL; + } else if (urn.equals(URN_BEAM_LOGICAL_JAVASDK)) { + return FieldType.logicalType( + (LogicalType) + SerializableUtils.deserializeFromByteArray( + protoFieldType.getLogicalType().getPayload().toByteArray(), "logicalType")); + } else { + throw new IllegalArgumentException("Encountered unsupported logical type URN: " + urn); + } + default: + throw new IllegalArgumentException( + "Unexpected type_info: " + protoFieldType.getTypeInfoCase()); + } + } } 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..20208146843e --- /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.SchemaApi; +import org.apache.beam.sdk.schemas.LogicalTypes; +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.v26_0_jre.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))) + .add( + Schema.of(Field.of("logical", FieldType.logicalType(LogicalTypes.FixedBytes.of(24))))) + .build(); + } + + @Parameter(0) + public Schema schema; + + @Test + public void toAndFromProto() throws Exception { + SchemaApi.Schema schemaProto = SchemaTranslation.schemaToProto(schema); + + Schema decodedSchema = SchemaTranslation.fromProto(schemaProto); + assertThat(decodedSchema, equalTo(schema)); + } + } +} diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/SchemaCoderCloudObjectTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/SchemaCoderCloudObjectTranslator.java index 2395f12a2555..2ff8c771f79c 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/SchemaCoderCloudObjectTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/SchemaCoderCloudObjectTranslator.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.dataflow.util; import java.io.IOException; -import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.model.pipeline.v1.SchemaApi; import org.apache.beam.runners.core.construction.SchemaTranslation; import org.apache.beam.runners.core.construction.SdkComponents; import org.apache.beam.sdk.schemas.Schema; @@ -52,7 +52,7 @@ public CloudObject toCloudObject(SchemaCoder target, SdkComponents sdkComponents base, SCHEMA, StringUtils.byteArrayToJsonString( - SchemaTranslation.toProto(target.getSchema()).toByteArray())); + SchemaTranslation.schemaToProto(target.getSchema()).toByteArray())); return base; } @@ -72,8 +72,8 @@ public SchemaCoder fromCloudObject(CloudObject cloudObject) { StringUtils.jsonStringToByteArray( Structs.getString(cloudObject, FROM_ROW_FUNCTION)), "fromRowFunction"); - RunnerApi.Schema protoSchema = - RunnerApi.Schema.parseFrom( + SchemaApi.Schema protoSchema = + SchemaApi.Schema.parseFrom( StringUtils.jsonStringToByteArray(Structs.getString(cloudObject, SCHEMA))); Schema schema = SchemaTranslation.fromProto(protoSchema); return SchemaCoder.of(schema, toRowFunction, fromRowFunction);