damondouglas commented on code in PR #25927: URL: https://github.com/apache/beam/pull/25927#discussion_r1167317861
########## sdks/java/io/file-schema-transform/src/main/java/org/apache/beam/sdk/io/fileschematransform/FileReadSchemaTransformConfiguration.java: ########## @@ -0,0 +1,128 @@ +/* + * 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.sdk.io.fileschematransform; + +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; + +import com.google.auto.value.AutoValue; +import java.util.Set; +import javax.annotation.Nullable; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Sets; + +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +@DefaultSchema(AutoValueSchema.class) +@AutoValue +public abstract class FileReadSchemaTransformConfiguration { + static final Set<String> VALID_FORMATS = Sets.newHashSet("avro", "parquet", "json", "line"); + + public void validate() { + checkArgument( + !Strings.isNullOrEmpty(this.getFormat()) && VALID_FORMATS.contains(this.getFormat()), + "A valid file format must be specified. Please specify one of: " + + VALID_FORMATS.toString()); + + if (!this.getFormat().equals("line")) { + checkArgument( + !Strings.isNullOrEmpty(this.getSchema()), + String.format( + "A schema must be specified when reading files with %s format. You may provide a schema string or a path to a file containing the schema.", + this.getFormat())); + } Review Comment: I did an experiment that might help here. The experiment answers "Can we use a Schema as a field type in our SchemaTransform configurations?" If yes, the implications of the answer in FileReadSchemaTransformConfiguration's context would be: ``` abstract class FileReadSchemaTransformConfiguration { abstract Schema getSchema(); } ``` So far it looks like yes. Here's what I did: #### 1. Define a simple AutoValue class <details> <summary>AutoValue Class with Schema field</summary> ``` @DefaultSchema(AutoValueSchema.class) @AutoValue public abstract class ExampleModel { public static Builder builder() { return new AutoValue_ExampleModel.Builder(); } public abstract String getAString(); public abstract Schema getASchema(); @AutoValue.Builder public abstract static class Builder { public abstract Builder setAString(String value); public abstract Builder setASchema(Schema value); public abstract ExampleModel build(); } } ``` </details> #### 2. Generate to/from Row functions from schema provider <details> <summary>SchemaProvider boilerplate</summary> ``` private static final AutoValueSchema SCHEMA_PROVIDER = new AutoValueSchema(); private static final TypeDescriptor<ExampleModel> TD = TypeDescriptor.of(ExampleModel.class); private static final SerializableFunction<ExampleModel, Row> TO_ROW_FN = SCHEMA_PROVIDER .toRowFunction(TD); private static final SerializableFunction<Row, ExampleModel> FROM_ROW_FN = SCHEMA_PROVIDER .fromRowFunction(TD); ``` </details> #### 3. Instantiate the AutoValue class, setting the Schema field <details> <summary>Create schema then instantiate ExampleModel</summary> ``` Schema schema = Schema.of( Field.of("anInteger", FieldType.INT32), Field.of("aBoolean", FieldType.BOOLEAN), Field.of("something", FieldType.STRING) ); ExampleModel model = ExampleModel.builder() .setAString("hello") .setASchema(schema) .build(); ``` </details> #### 4. Convert ExampleModel to a Row This simulates what the SchemaProvider would receive as the configuration representation. ``` Row row = TO_ROW_FN.apply(model); ``` #### 5. Convert back to an ExampleModel from the Row ``` ExampleModel backAgain = FROM_ROW_FN.apply(row); ``` #### 6. Finally check whether the Schema fields were preserved during the conversions <details> <summary>Call getSchema() and run a bunch of checks</summary> ``` Schema backAgainASchema = backAgain.getASchema(); checkNotNull(backAgainASchema); checkState(backAgainASchema.hasField("anInteger")); checkState(backAgainASchema.hasField("aBoolean")); checkState(backAgainASchema.hasField("something")); checkState(backAgainASchema.getField("anInteger").getType().equals(FieldType.INT32)); checkState(backAgainASchema.getField("aBoolean").getType().equals(FieldType.BOOLEAN)); checkState(backAgainASchema.getField("something").getType().equals(FieldType.STRING)); ``` </details> -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
