BjornPrime commented on code in PR #24974:
URL: https://github.com/apache/beam/pull/24974#discussion_r1068618462


##########
sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroReadSchemaTransformProvider.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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;
+
+import com.google.auto.value.AutoValue;
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.List;
+import org.apache.beam.sdk.schemas.AutoValueSchema;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
+import org.apache.beam.sdk.schemas.transforms.Convert;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
+import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider;
+import org.apache.beam.sdk.schemas.utils.AvroUtils;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
+
+public class AvroReadSchemaTransformProvider
+    extends TypedSchemaTransformProvider<
+        AvroReadSchemaTransformProvider.AvroReadSchemaTransformConfiguration> {
+  @Override
+  public String identifier() {
+    return "beam:schematransform:org.apache.beam:avro_read:v1";
+  }
+
+  @Override
+  public List<String> inputCollectionNames() {
+    return Collections.emptyList();
+  }
+
+  @Override
+  public List<String> outputCollectionNames() {
+    return Collections.singletonList("output");
+  }
+
+  @Override
+  protected Class<AvroReadSchemaTransformConfiguration> configurationClass() {
+    return AvroReadSchemaTransformConfiguration.class;
+  }
+
+  @Override
+  protected SchemaTransform from(AvroReadSchemaTransformConfiguration 
configuration) {
+    return new AvroReadSchemaTransform(configuration);
+  }
+
+  static class AvroReadSchemaTransform implements SchemaTransform {
+
+    AvroReadSchemaTransformConfiguration config;
+
+    AvroReadSchemaTransform(AvroReadSchemaTransformConfiguration config) {
+      this.config = config;
+    }
+
+    @Override
+    public PTransform<PCollectionRowTuple, PCollectionRowTuple> 
buildTransform() {
+      return new PTransform<PCollectionRowTuple, PCollectionRowTuple>() {
+        @Override
+        public PCollectionRowTuple expand(PCollectionRowTuple input) {

Review Comment:
   My understanding was that we usually split the PTransform this builds into 
its own class, called a PCollectionRowTupleTransform, which would allow the 
logic to be broken down a little more. As it stands, this implementation, with 
the nested return statements, is a bit arcane to me. I can figure out what it 
is doing by referencing other SchemaTransformProviders but it isn't as obvious 
as it could be, I think.



##########
sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSchemaTransformsTest.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.testing.NeedsRunner;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.Row;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+public class AvroSchemaTransformsTest {
+  @Rule public TestPipeline writePipeline = TestPipeline.create();
+  @Rule public TestPipeline readPipeline = TestPipeline.create();
+  @Rule public TemporaryFolder tempFolder = new TemporaryFolder();
+
+  private static final Schema SCHEMA =
+      Schema.builder().addInt64Field("age").addStringField("age_str").build();
+
+  private Row createRow(long l) {
+    return Row.withSchema(SCHEMA).addValues(l, 
Long.valueOf(l).toString()).build();
+  }
+
+  @Test
+  @Ignore
+  @Category({NeedsRunner.class})
+  public void testWriteAndReadTable() {

Review Comment:
   I think we would ideally test read and write separately here, as well as 
testing some failure paths for each.



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroWriteSchemaTransformProvider.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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;
+
+import com.google.auto.value.AutoValue;
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.beam.sdk.schemas.AutoValueSchema;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
+import org.apache.beam.sdk.schemas.transforms.Convert;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
+import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider;
+import org.apache.beam.sdk.schemas.utils.AvroUtils;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
+import org.joda.time.Duration;
+
+public class AvroWriteSchemaTransformProvider
+    extends TypedSchemaTransformProvider<
+        
AvroWriteSchemaTransformProvider.AvroWriteSchemaTransformConfiguration> {
+  @Override
+  public String identifier() {
+    return "beam:schematransform:org.apache.beam:avro_write:v1";
+  }
+
+  @Override
+  public List<String> inputCollectionNames() {
+    return Collections.singletonList("input");
+  }
+
+  @Override
+  public List<String> outputCollectionNames() {
+    return Collections.emptyList();
+  }
+
+  @Override
+  protected Class<AvroWriteSchemaTransformConfiguration> configurationClass() {
+    return AvroWriteSchemaTransformConfiguration.class;
+  }
+
+  @Override
+  protected SchemaTransform from(AvroWriteSchemaTransformConfiguration 
configuration) {
+    return new AvroWriteSchemaTransform(configuration);
+  }
+
+  static class AvroWriteSchemaTransform implements SchemaTransform, 
Serializable {
+
+    AvroWriteSchemaTransformConfiguration config;
+
+    AvroWriteSchemaTransform(AvroWriteSchemaTransformConfiguration config) {
+      this.config = config;
+    }
+
+    @Override
+    public PTransform<PCollectionRowTuple, PCollectionRowTuple> 
buildTransform() {

Review Comment:
   Same concerns as for this part in the ReadSTProvider.



-- 
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]

Reply via email to