ahmedabu98 commented on code in PR #34411:
URL: https://github.com/apache/beam/pull/34411#discussion_r2027493182


##########
sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordReadSchemaTransformProvider.java:
##########
@@ -0,0 +1,221 @@
+/*
+ * 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 static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull;
+
+import com.google.auto.service.AutoService;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaRegistry;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider;
+import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider;
+import org.apache.beam.sdk.schemas.transforms.providers.ErrorHandling;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn.ProcessElement;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@AutoService(SchemaTransformProvider.class)
+public class TFRecordReadSchemaTransformProvider
+    extends 
TypedSchemaTransformProvider<TFRecordReadSchemaTransformConfiguration> {
+  private static final String IDENTIFIER = 
"beam:schematransform:org.apache.beam:tfrecord_read:v1";
+  private static final String OUTPUT = "output";
+  private static final String ERROR = "errors";
+  public static final TupleTag<Row> OUTPUT_TAG = new TupleTag<Row>() {};
+  public static final TupleTag<Row> ERROR_TAG = new TupleTag<Row>() {};
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TFRecordReadSchemaTransformProvider.class);
+
+  /** Returns the expected class of the configuration. */
+  @Override
+  protected Class<TFRecordReadSchemaTransformConfiguration> 
configurationClass() {
+    return TFRecordReadSchemaTransformConfiguration.class;
+  }
+
+  /** Returns the expected {@link SchemaTransform} of the configuration. */
+  @Override
+  protected SchemaTransform from(TFRecordReadSchemaTransformConfiguration 
configuration) {
+    return new TFRecordReadSchemaTransform(configuration);
+  }
+
+  /** Implementation of the {@link TypedSchemaTransformProvider} identifier 
method. */
+  @Override
+  public String identifier() {
+    return IDENTIFIER;
+  }
+
+  /**
+   * Implementation of the {@link TypedSchemaTransformProvider} 
inputCollectionNames method. Since
+   * no input is expected, this returns an empty list.
+   */
+  @Override
+  public List<String> inputCollectionNames() {
+    return Collections.emptyList();
+  }
+
+  /** Implementation of the {@link TypedSchemaTransformProvider} 
outputCollectionNames method. */
+  @Override
+  public List<String> outputCollectionNames() {
+    return Arrays.asList(OUTPUT, ERROR);
+  }
+
+  /**
+   * An implementation of {@link SchemaTransform} for TFRecord read jobs 
configured using {@link
+   * TFRecordReadSchemaTransformConfiguration}.
+   */
+  static class TFRecordReadSchemaTransform extends SchemaTransform {
+    private final TFRecordReadSchemaTransformConfiguration configuration;
+
+    TFRecordReadSchemaTransform(TFRecordReadSchemaTransformConfiguration 
configuration) {
+      this.configuration = configuration;
+    }
+
+    public Row getConfigurationRow() {
+      try {
+        // To stay consistent with our SchemaTransform configuration naming 
conventions,
+        // we sort lexicographically
+        return SchemaRegistry.createDefault()
+            .getToRowFunction(TFRecordReadSchemaTransformConfiguration.class)
+            .apply(configuration)
+            .sorted()
+            .toSnakeCase();
+      } catch (NoSuchSchemaException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    @Override
+    public PCollectionRowTuple expand(PCollectionRowTuple input) {
+      // Validate configuration parameters
+      configuration.validate();
+
+      TFRecordIO.Read readTransform =
+          
TFRecordIO.read().withCompression(Compression.valueOf(configuration.getCompression()));
+
+      String filePattern = configuration.getFilePattern();
+      if (filePattern != null) {
+        readTransform = readTransform.from(filePattern);
+      }
+      if (!configuration.getValidate()) {
+        readTransform = readTransform.withoutValidation();
+      }
+
+      // Read TFRecord files into a PCollection of byte arrays.
+      PCollection<byte[]> tfRecordValues = 
input.getPipeline().apply(readTransform);
+
+      // Define the schema for the row
+      Schema schema = Schema.of(Schema.Field.of("record", 
Schema.FieldType.BYTES));
+      Schema errorSchema = ErrorHandling.errorSchemaBytes();
+      boolean handleErrors = 
ErrorHandling.hasOutput(configuration.getErrorHandling());
+
+      SerializableFunction<byte[], Row> bytesToRowFn = getBytesToRowFn(schema);
+
+      // Apply bytes to row fn
+      PCollectionTuple outputTuple =
+          tfRecordValues.apply(
+              ParDo.of(
+                      new ErrorFn(
+                          "TFRecord-read-error-counter", bytesToRowFn, 
errorSchema, handleErrors))
+                  .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG)));
+
+      PCollectionRowTuple outputRows =
+          PCollectionRowTuple.of("output", 
outputTuple.get(OUTPUT_TAG).setRowSchema(schema));
+
+      // Error handling
+      PCollection<Row> errorOutput = 
outputTuple.get(ERROR_TAG).setRowSchema(errorSchema);
+      if (handleErrors) {
+        outputRows =
+            outputRows.and(
+                
checkArgumentNotNull(configuration.getErrorHandling()).getOutput(), 
errorOutput);
+      }
+      return outputRows;
+    }
+  }
+
+  public static SerializableFunction<byte[], Row> getBytesToRowFn(Schema 
schema) {
+    return new SimpleFunction<byte[], Row>() {
+      @Override
+      public Row apply(byte[] input) {
+        Row row = Row.withSchema(schema).addValues(input).build();
+        if (row == null) {
+          throw new NullPointerException();
+        }

Review Comment:
   will this ever actually be true?



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordReadSchemaTransformProvider.java:
##########
@@ -0,0 +1,221 @@
+/*
+ * 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 static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull;
+
+import com.google.auto.service.AutoService;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaRegistry;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider;
+import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider;
+import org.apache.beam.sdk.schemas.transforms.providers.ErrorHandling;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn.ProcessElement;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@AutoService(SchemaTransformProvider.class)
+public class TFRecordReadSchemaTransformProvider
+    extends 
TypedSchemaTransformProvider<TFRecordReadSchemaTransformConfiguration> {
+  private static final String IDENTIFIER = 
"beam:schematransform:org.apache.beam:tfrecord_read:v1";
+  private static final String OUTPUT = "output";
+  private static final String ERROR = "errors";
+  public static final TupleTag<Row> OUTPUT_TAG = new TupleTag<Row>() {};
+  public static final TupleTag<Row> ERROR_TAG = new TupleTag<Row>() {};
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TFRecordReadSchemaTransformProvider.class);
+
+  /** Returns the expected class of the configuration. */
+  @Override
+  protected Class<TFRecordReadSchemaTransformConfiguration> 
configurationClass() {
+    return TFRecordReadSchemaTransformConfiguration.class;
+  }
+
+  /** Returns the expected {@link SchemaTransform} of the configuration. */
+  @Override
+  protected SchemaTransform from(TFRecordReadSchemaTransformConfiguration 
configuration) {
+    return new TFRecordReadSchemaTransform(configuration);
+  }
+
+  /** Implementation of the {@link TypedSchemaTransformProvider} identifier 
method. */
+  @Override
+  public String identifier() {
+    return IDENTIFIER;
+  }
+
+  /**
+   * Implementation of the {@link TypedSchemaTransformProvider} 
inputCollectionNames method. Since
+   * no input is expected, this returns an empty list.
+   */
+  @Override
+  public List<String> inputCollectionNames() {
+    return Collections.emptyList();
+  }
+
+  /** Implementation of the {@link TypedSchemaTransformProvider} 
outputCollectionNames method. */
+  @Override
+  public List<String> outputCollectionNames() {
+    return Arrays.asList(OUTPUT, ERROR);
+  }
+
+  /**
+   * An implementation of {@link SchemaTransform} for TFRecord read jobs 
configured using {@link
+   * TFRecordReadSchemaTransformConfiguration}.
+   */
+  static class TFRecordReadSchemaTransform extends SchemaTransform {
+    private final TFRecordReadSchemaTransformConfiguration configuration;
+
+    TFRecordReadSchemaTransform(TFRecordReadSchemaTransformConfiguration 
configuration) {
+      this.configuration = configuration;
+    }
+
+    public Row getConfigurationRow() {
+      try {
+        // To stay consistent with our SchemaTransform configuration naming 
conventions,
+        // we sort lexicographically
+        return SchemaRegistry.createDefault()
+            .getToRowFunction(TFRecordReadSchemaTransformConfiguration.class)
+            .apply(configuration)
+            .sorted()
+            .toSnakeCase();
+      } catch (NoSuchSchemaException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    @Override
+    public PCollectionRowTuple expand(PCollectionRowTuple input) {
+      // Validate configuration parameters
+      configuration.validate();
+
+      TFRecordIO.Read readTransform =
+          
TFRecordIO.read().withCompression(Compression.valueOf(configuration.getCompression()));
+
+      String filePattern = configuration.getFilePattern();
+      if (filePattern != null) {
+        readTransform = readTransform.from(filePattern);
+      }
+      if (!configuration.getValidate()) {
+        readTransform = readTransform.withoutValidation();
+      }
+
+      // Read TFRecord files into a PCollection of byte arrays.
+      PCollection<byte[]> tfRecordValues = 
input.getPipeline().apply(readTransform);
+
+      // Define the schema for the row
+      Schema schema = Schema.of(Schema.Field.of("record", 
Schema.FieldType.BYTES));

Review Comment:
   nit: should this output schema be final and static?



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordReadSchemaTransformProvider.java:
##########
@@ -0,0 +1,221 @@
+/*
+ * 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 static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull;
+
+import com.google.auto.service.AutoService;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaRegistry;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider;
+import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider;
+import org.apache.beam.sdk.schemas.transforms.providers.ErrorHandling;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn.ProcessElement;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@AutoService(SchemaTransformProvider.class)
+public class TFRecordReadSchemaTransformProvider
+    extends 
TypedSchemaTransformProvider<TFRecordReadSchemaTransformConfiguration> {
+  private static final String IDENTIFIER = 
"beam:schematransform:org.apache.beam:tfrecord_read:v1";
+  private static final String OUTPUT = "output";
+  private static final String ERROR = "errors";
+  public static final TupleTag<Row> OUTPUT_TAG = new TupleTag<Row>() {};
+  public static final TupleTag<Row> ERROR_TAG = new TupleTag<Row>() {};
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TFRecordReadSchemaTransformProvider.class);
+
+  /** Returns the expected class of the configuration. */
+  @Override
+  protected Class<TFRecordReadSchemaTransformConfiguration> 
configurationClass() {
+    return TFRecordReadSchemaTransformConfiguration.class;
+  }
+
+  /** Returns the expected {@link SchemaTransform} of the configuration. */
+  @Override
+  protected SchemaTransform from(TFRecordReadSchemaTransformConfiguration 
configuration) {
+    return new TFRecordReadSchemaTransform(configuration);
+  }
+
+  /** Implementation of the {@link TypedSchemaTransformProvider} identifier 
method. */
+  @Override
+  public String identifier() {
+    return IDENTIFIER;
+  }
+
+  /**
+   * Implementation of the {@link TypedSchemaTransformProvider} 
inputCollectionNames method. Since
+   * no input is expected, this returns an empty list.
+   */
+  @Override
+  public List<String> inputCollectionNames() {
+    return Collections.emptyList();
+  }

Review Comment:
   nit: also no longer needed. it's just a copy of the existing default: 
https://github.com/ahmedabu98/beam/blob/d2da943412847003903abbdf9308adfb9e545983/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/SchemaTransformProvider.java#L61-L64



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordWriteSchemaTransformProvider.java:
##########
@@ -0,0 +1,268 @@
+/*
+ * 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.service.AutoService;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaRegistry;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider;
+import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider;
+import org.apache.beam.sdk.schemas.transforms.providers.ErrorHandling;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn.Element;
+import org.apache.beam.sdk.transforms.DoFn.ProcessElement;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@AutoService(SchemaTransformProvider.class)
+public class TFRecordWriteSchemaTransformProvider
+    extends 
TypedSchemaTransformProvider<TFRecordWriteSchemaTransformConfiguration> {
+  private static final String IDENTIFIER = 
"beam:schematransform:org.apache.beam:tfrecord_write:v1";
+  private static final String INPUT = "input";
+  private static final String OUTPUT = "output";
+  private static final String ERROR = "errors";
+  public static final TupleTag<byte[]> OUTPUT_TAG = new TupleTag<byte[]>() {};
+  public static final TupleTag<Row> ERROR_TAG = new TupleTag<Row>() {};
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TFRecordWriteSchemaTransformProvider.class);
+
+  /** Returns the expected class of the configuration. */
+  @Override
+  protected Class<TFRecordWriteSchemaTransformConfiguration> 
configurationClass() {
+    return TFRecordWriteSchemaTransformConfiguration.class;
+  }
+
+  /** Returns the expected {@link SchemaTransform} of the configuration. */
+  @Override
+  protected SchemaTransform from(TFRecordWriteSchemaTransformConfiguration 
configuration) {
+    return new TFRecordWriteSchemaTransform(configuration);
+  }
+
+  /** Implementation of the {@link TypedSchemaTransformProvider} identifier 
method. */
+  @Override
+  public String identifier() {
+    return IDENTIFIER;
+  }
+
+  /** Implementation of the {@link TypedSchemaTransformProvider} 
inputCollectionNames method. */
+  @Override
+  public List<String> inputCollectionNames() {
+    return Arrays.asList(INPUT, ERROR);
+  }
+
+  /** Implementation of the {@link TypedSchemaTransformProvider} 
outputCollectionNames method. */
+  @Override
+  public List<String> outputCollectionNames() {
+    return Arrays.asList(OUTPUT, ERROR);
+  }
+
+  /**
+   * An implementation of {@link SchemaTransform} for TFRecord Write jobs 
configured using {@link
+   * TFRecordWriteSchemaTransformConfiguration}.
+   */
+  static class TFRecordWriteSchemaTransform extends SchemaTransform {
+    private final TFRecordWriteSchemaTransformConfiguration configuration;
+
+    TFRecordWriteSchemaTransform(TFRecordWriteSchemaTransformConfiguration 
configuration) {
+      this.configuration = configuration;
+    }
+
+    public Row getConfigurationRow() {
+      try {
+        // To stay consistent with our SchemaTransform configuration naming 
conventions,
+        // we sort lexicographically
+        return SchemaRegistry.createDefault()
+            .getToRowFunction(TFRecordWriteSchemaTransformConfiguration.class)
+            .apply(configuration)
+            .sorted()
+            .toSnakeCase();
+      } catch (NoSuchSchemaException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    @Override
+    public PCollectionRowTuple expand(PCollectionRowTuple input) {
+      // Validate configuration parameters
+      configuration.validate();
+
+      // Create basic transform
+      TFRecordIO.Write writeTransform =
+          
TFRecordIO.write().withCompression(Compression.valueOf(configuration.getCompression()));
+
+      // Add more parameters if not null
+      String outputPrefix = configuration.getOutputPrefix();
+      if (outputPrefix != null) {
+        writeTransform = writeTransform.to(outputPrefix);
+      }
+      String shardTemplate = configuration.getShardTemplate();
+      if (shardTemplate != null) {
+        writeTransform = writeTransform.withShardNameTemplate(shardTemplate);
+      }
+      String filenameSuffix = configuration.getFilenameSuffix();
+      if (filenameSuffix != null) {
+        writeTransform = writeTransform.withSuffix(filenameSuffix);
+      }
+      if (configuration.getNumShards() > 0) {
+        writeTransform = 
writeTransform.withNumShards(configuration.getNumShards());
+      } else {
+        writeTransform = writeTransform.withoutSharding();
+      }
+      if (configuration.getNoSpilling()) {
+        writeTransform = writeTransform.withNoSpilling();
+      }
+
+      // Obtain input schema and verify only one field and its bytes
+      Schema inputSchema = input.get("input").getSchema();

Review Comment:
   ```suggestion
         Schema inputSchema = input.get(INPUT).getSchema();
   ```



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordWriteSchemaTransformProvider.java:
##########
@@ -0,0 +1,268 @@
+/*
+ * 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.service.AutoService;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaRegistry;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider;
+import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider;
+import org.apache.beam.sdk.schemas.transforms.providers.ErrorHandling;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn.Element;
+import org.apache.beam.sdk.transforms.DoFn.ProcessElement;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@AutoService(SchemaTransformProvider.class)
+public class TFRecordWriteSchemaTransformProvider
+    extends 
TypedSchemaTransformProvider<TFRecordWriteSchemaTransformConfiguration> {
+  private static final String IDENTIFIER = 
"beam:schematransform:org.apache.beam:tfrecord_write:v1";
+  private static final String INPUT = "input";
+  private static final String OUTPUT = "output";
+  private static final String ERROR = "errors";
+  public static final TupleTag<byte[]> OUTPUT_TAG = new TupleTag<byte[]>() {};
+  public static final TupleTag<Row> ERROR_TAG = new TupleTag<Row>() {};
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TFRecordWriteSchemaTransformProvider.class);
+
+  /** Returns the expected class of the configuration. */
+  @Override
+  protected Class<TFRecordWriteSchemaTransformConfiguration> 
configurationClass() {
+    return TFRecordWriteSchemaTransformConfiguration.class;
+  }
+
+  /** Returns the expected {@link SchemaTransform} of the configuration. */
+  @Override
+  protected SchemaTransform from(TFRecordWriteSchemaTransformConfiguration 
configuration) {
+    return new TFRecordWriteSchemaTransform(configuration);
+  }
+
+  /** Implementation of the {@link TypedSchemaTransformProvider} identifier 
method. */
+  @Override
+  public String identifier() {
+    return IDENTIFIER;
+  }
+
+  /** Implementation of the {@link TypedSchemaTransformProvider} 
inputCollectionNames method. */
+  @Override
+  public List<String> inputCollectionNames() {
+    return Arrays.asList(INPUT, ERROR);

Review Comment:
   ```suggestion
       return Arrays.asList(INPUT);
   ```
   nit: we only use `INPUT` here



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordWriteSchemaTransformProvider.java:
##########
@@ -0,0 +1,268 @@
+/*
+ * 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.service.AutoService;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaRegistry;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider;
+import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider;
+import org.apache.beam.sdk.schemas.transforms.providers.ErrorHandling;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn.Element;
+import org.apache.beam.sdk.transforms.DoFn.ProcessElement;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@AutoService(SchemaTransformProvider.class)
+public class TFRecordWriteSchemaTransformProvider
+    extends 
TypedSchemaTransformProvider<TFRecordWriteSchemaTransformConfiguration> {
+  private static final String IDENTIFIER = 
"beam:schematransform:org.apache.beam:tfrecord_write:v1";
+  private static final String INPUT = "input";
+  private static final String OUTPUT = "output";
+  private static final String ERROR = "errors";
+  public static final TupleTag<byte[]> OUTPUT_TAG = new TupleTag<byte[]>() {};
+  public static final TupleTag<Row> ERROR_TAG = new TupleTag<Row>() {};
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TFRecordWriteSchemaTransformProvider.class);
+
+  /** Returns the expected class of the configuration. */
+  @Override
+  protected Class<TFRecordWriteSchemaTransformConfiguration> 
configurationClass() {
+    return TFRecordWriteSchemaTransformConfiguration.class;
+  }
+
+  /** Returns the expected {@link SchemaTransform} of the configuration. */
+  @Override
+  protected SchemaTransform from(TFRecordWriteSchemaTransformConfiguration 
configuration) {
+    return new TFRecordWriteSchemaTransform(configuration);
+  }
+
+  /** Implementation of the {@link TypedSchemaTransformProvider} identifier 
method. */
+  @Override
+  public String identifier() {
+    return IDENTIFIER;
+  }
+
+  /** Implementation of the {@link TypedSchemaTransformProvider} 
inputCollectionNames method. */
+  @Override
+  public List<String> inputCollectionNames() {
+    return Arrays.asList(INPUT, ERROR);
+  }
+
+  /** Implementation of the {@link TypedSchemaTransformProvider} 
outputCollectionNames method. */
+  @Override
+  public List<String> outputCollectionNames() {
+    return Arrays.asList(OUTPUT, ERROR);
+  }
+
+  /**
+   * An implementation of {@link SchemaTransform} for TFRecord Write jobs 
configured using {@link
+   * TFRecordWriteSchemaTransformConfiguration}.
+   */
+  static class TFRecordWriteSchemaTransform extends SchemaTransform {
+    private final TFRecordWriteSchemaTransformConfiguration configuration;
+
+    TFRecordWriteSchemaTransform(TFRecordWriteSchemaTransformConfiguration 
configuration) {
+      this.configuration = configuration;
+    }
+
+    public Row getConfigurationRow() {
+      try {
+        // To stay consistent with our SchemaTransform configuration naming 
conventions,
+        // we sort lexicographically
+        return SchemaRegistry.createDefault()
+            .getToRowFunction(TFRecordWriteSchemaTransformConfiguration.class)
+            .apply(configuration)
+            .sorted()
+            .toSnakeCase();
+      } catch (NoSuchSchemaException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    @Override
+    public PCollectionRowTuple expand(PCollectionRowTuple input) {
+      // Validate configuration parameters
+      configuration.validate();
+
+      // Create basic transform
+      TFRecordIO.Write writeTransform =
+          
TFRecordIO.write().withCompression(Compression.valueOf(configuration.getCompression()));
+
+      // Add more parameters if not null
+      String outputPrefix = configuration.getOutputPrefix();
+      if (outputPrefix != null) {
+        writeTransform = writeTransform.to(outputPrefix);
+      }
+      String shardTemplate = configuration.getShardTemplate();
+      if (shardTemplate != null) {
+        writeTransform = writeTransform.withShardNameTemplate(shardTemplate);
+      }
+      String filenameSuffix = configuration.getFilenameSuffix();
+      if (filenameSuffix != null) {
+        writeTransform = writeTransform.withSuffix(filenameSuffix);
+      }
+      if (configuration.getNumShards() > 0) {
+        writeTransform = 
writeTransform.withNumShards(configuration.getNumShards());
+      } else {
+        writeTransform = writeTransform.withoutSharding();
+      }
+      if (configuration.getNoSpilling()) {
+        writeTransform = writeTransform.withNoSpilling();
+      }
+
+      // Obtain input schema and verify only one field and its bytes
+      Schema inputSchema = input.get("input").getSchema();
+      int numFields = inputSchema.getFields().size();
+      if (numFields != 1) {
+        throw new IllegalArgumentException("Expecting exactly one field, found 
" + numFields);
+      } else if 
(!inputSchema.getField(0).getType().equals(Schema.FieldType.BYTES)) {
+        throw new IllegalArgumentException(
+            "The input schema must have exactly one field of type byte.");
+      }
+
+      final String schemaField;
+      if (inputSchema.getField(0).getName() != null) {
+        schemaField = inputSchema.getField(0).getName();
+      } else {
+        schemaField = "record";
+      }
+
+      PCollection<Row> inputRows = input.get(INPUT);
+
+      // Convert Beam Rows to byte arrays
+      SerializableFunction<Row, byte[]> rowToBytesFn = 
getRowToBytesFn(schemaField);
+
+      Schema errorSchema = ErrorHandling.errorSchema(inputSchema);
+      boolean handleErrors = 
ErrorHandling.hasOutput(configuration.getErrorHandling());
+
+      // Apply row to bytes fn
+      PCollectionTuple byteArrays =
+          inputRows.apply(
+              ParDo.of(
+                      new ErrorFn(
+                          "TFRecord-write-error-counter", rowToBytesFn, 
errorSchema, handleErrors))
+                  .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG)));
+
+      // Apply the write transform to byte arrays to write tfrecords to file.
+      
byteArrays.get(OUTPUT_TAG).setCoder(ByteArrayCoder.of()).apply(writeTransform);
+
+      // Error handling
+      String output = "";
+      ErrorHandling errorHandler = configuration.getErrorHandling();
+      if (errorHandler != null) {
+        String outputHandler = errorHandler.getOutput();
+        if (outputHandler != null) {
+          output = outputHandler;
+        } else {
+          output = "";
+        }
+      }
+      PCollection<Row> errorOutput =
+          
byteArrays.get(ERROR_TAG).setRowSchema(ErrorHandling.errorSchema(errorSchema));
+      return PCollectionRowTuple.of(handleErrors ? output : "errors", 
errorOutput);
+    }
+  }
+
+  public static class RowToBytesDoFn extends DoFn<Row, byte[]> {
+    private final SerializableFunction<Row, byte[]> rowToBytesFn;
+
+    public RowToBytesDoFn(SerializableFunction<Row, byte[]> rowToBytesFn) {
+      this.rowToBytesFn = rowToBytesFn;
+    }
+
+    @ProcessElement
+    public void processElement(@Element Row row, OutputReceiver<byte[]> out) {
+      byte[] bytes = rowToBytesFn.apply(row);
+      out.output(bytes);
+    }
+  }

Review Comment:
   Is this used anywhere?



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordReadSchemaTransformConfiguration.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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 static 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument;
+import static 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull;
+import static 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState;
+
+import com.google.auto.value.AutoValue;
+import java.io.IOException;
+import java.io.Serializable;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.io.fs.MatchResult;
+import org.apache.beam.sdk.schemas.AutoValueSchema;
+import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
+import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription;
+import org.apache.beam.sdk.schemas.transforms.providers.ErrorHandling;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings;
+
+/**
+ * Configuration for reading from TFRecord.
+ *
+ * <p>This class is meant to be used with {@link 
TFRecordReadSchemaTransformProvider}.
+ *
+ * <p><b>Internal only:</b> This class is actively being worked on, and it 
will likely change. We
+ * provide no backwards compatibility guarantees, and it should not be 
implemented outside the Beam
+ * repository.
+ */
+@DefaultSchema(AutoValueSchema.class)
+@AutoValue
+public abstract class TFRecordReadSchemaTransformConfiguration implements 
Serializable {
+
+  public void validate() {
+    String invalidConfigMessage = "Invalid TFRecord Read configuration: ";
+    checkNotNull(getValidate(), "To read from TFRecord, validation must be 
specified.");
+    checkNotNull(getCompression(), "To read from TFRecord, compression must be 
specified.");
+
+    String filePattern = getFilePattern();
+    if (filePattern == null || filePattern.isEmpty()) {
+      throw new IllegalStateException(
+          "Need to set the filepattern of a TFRecordReadSchema transform");
+    }

Review Comment:
   Similar to above, this is not needed. just make sure to remove the 
`@Nullable` decorator from `String getFilePattern()` below



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordSchemaTransformTranslation.java:
##########
@@ -0,0 +1,80 @@
+/*
+ * 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 static 
org.apache.beam.sdk.io.TFRecordReadSchemaTransformProvider.TFRecordReadSchemaTransform;
+import static 
org.apache.beam.sdk.io.TFRecordWriteSchemaTransformProvider.TFRecordWriteSchemaTransform;
+
+import com.google.auto.service.AutoService;
+import java.util.Map;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransformTranslation;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.util.construction.PTransformTranslation;
+import 
org.apache.beam.sdk.util.construction.TransformPayloadTranslatorRegistrar;
+import org.apache.beam.sdk.values.Row;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
+
+public class TFRecordSchemaTransformTranslation {
+  public static class TFRecordStorageReadSchemaTransformTranslator

Review Comment:
   ```suggestion
     public static class TFRecordReadSchemaTransformTranslator
   ```
   nit: typo?



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordReadSchemaTransformProvider.java:
##########
@@ -0,0 +1,221 @@
+/*
+ * 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 static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull;
+
+import com.google.auto.service.AutoService;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaRegistry;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider;
+import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider;
+import org.apache.beam.sdk.schemas.transforms.providers.ErrorHandling;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn.ProcessElement;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@AutoService(SchemaTransformProvider.class)
+public class TFRecordReadSchemaTransformProvider
+    extends 
TypedSchemaTransformProvider<TFRecordReadSchemaTransformConfiguration> {
+  private static final String IDENTIFIER = 
"beam:schematransform:org.apache.beam:tfrecord_read:v1";
+  private static final String OUTPUT = "output";
+  private static final String ERROR = "errors";
+  public static final TupleTag<Row> OUTPUT_TAG = new TupleTag<Row>() {};
+  public static final TupleTag<Row> ERROR_TAG = new TupleTag<Row>() {};
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TFRecordReadSchemaTransformProvider.class);
+
+  /** Returns the expected class of the configuration. */
+  @Override
+  protected Class<TFRecordReadSchemaTransformConfiguration> 
configurationClass() {
+    return TFRecordReadSchemaTransformConfiguration.class;
+  }

Review Comment:
   nit: this boilerplate is no longer needed after #30560



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordWriteSchemaTransformProvider.java:
##########
@@ -0,0 +1,268 @@
+/*
+ * 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.service.AutoService;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaRegistry;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider;
+import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider;
+import org.apache.beam.sdk.schemas.transforms.providers.ErrorHandling;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn.Element;
+import org.apache.beam.sdk.transforms.DoFn.ProcessElement;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@AutoService(SchemaTransformProvider.class)
+public class TFRecordWriteSchemaTransformProvider
+    extends 
TypedSchemaTransformProvider<TFRecordWriteSchemaTransformConfiguration> {
+  private static final String IDENTIFIER = 
"beam:schematransform:org.apache.beam:tfrecord_write:v1";
+  private static final String INPUT = "input";
+  private static final String OUTPUT = "output";
+  private static final String ERROR = "errors";
+  public static final TupleTag<byte[]> OUTPUT_TAG = new TupleTag<byte[]>() {};
+  public static final TupleTag<Row> ERROR_TAG = new TupleTag<Row>() {};
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TFRecordWriteSchemaTransformProvider.class);
+
+  /** Returns the expected class of the configuration. */
+  @Override
+  protected Class<TFRecordWriteSchemaTransformConfiguration> 
configurationClass() {
+    return TFRecordWriteSchemaTransformConfiguration.class;
+  }
+
+  /** Returns the expected {@link SchemaTransform} of the configuration. */
+  @Override
+  protected SchemaTransform from(TFRecordWriteSchemaTransformConfiguration 
configuration) {
+    return new TFRecordWriteSchemaTransform(configuration);
+  }
+
+  /** Implementation of the {@link TypedSchemaTransformProvider} identifier 
method. */
+  @Override
+  public String identifier() {
+    return IDENTIFIER;
+  }
+
+  /** Implementation of the {@link TypedSchemaTransformProvider} 
inputCollectionNames method. */
+  @Override
+  public List<String> inputCollectionNames() {
+    return Arrays.asList(INPUT, ERROR);
+  }
+
+  /** Implementation of the {@link TypedSchemaTransformProvider} 
outputCollectionNames method. */
+  @Override
+  public List<String> outputCollectionNames() {
+    return Arrays.asList(OUTPUT, ERROR);
+  }
+
+  /**
+   * An implementation of {@link SchemaTransform} for TFRecord Write jobs 
configured using {@link
+   * TFRecordWriteSchemaTransformConfiguration}.
+   */
+  static class TFRecordWriteSchemaTransform extends SchemaTransform {
+    private final TFRecordWriteSchemaTransformConfiguration configuration;
+
+    TFRecordWriteSchemaTransform(TFRecordWriteSchemaTransformConfiguration 
configuration) {
+      this.configuration = configuration;
+    }
+
+    public Row getConfigurationRow() {
+      try {
+        // To stay consistent with our SchemaTransform configuration naming 
conventions,
+        // we sort lexicographically
+        return SchemaRegistry.createDefault()
+            .getToRowFunction(TFRecordWriteSchemaTransformConfiguration.class)
+            .apply(configuration)
+            .sorted()
+            .toSnakeCase();
+      } catch (NoSuchSchemaException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    @Override
+    public PCollectionRowTuple expand(PCollectionRowTuple input) {
+      // Validate configuration parameters
+      configuration.validate();
+
+      // Create basic transform
+      TFRecordIO.Write writeTransform =
+          
TFRecordIO.write().withCompression(Compression.valueOf(configuration.getCompression()));
+
+      // Add more parameters if not null
+      String outputPrefix = configuration.getOutputPrefix();
+      if (outputPrefix != null) {

Review Comment:
   nit: no need for these null checks if the SchemaTransformConfiguration 
object does not mark them as `@Nullable`



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordWriteSchemaTransformConfiguration.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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 static 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument;
+import static 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull;
+import static 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState;
+
+import com.google.auto.value.AutoValue;
+import java.io.IOException;
+import org.apache.beam.sdk.io.fs.MatchResult;
+import org.apache.beam.sdk.schemas.AutoValueSchema;
+import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
+import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription;
+import org.apache.beam.sdk.schemas.transforms.providers.ErrorHandling;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Configuration for reading from TFRecord.
+ *
+ * <p>This class is meant to be used with {@link 
TFRecordWriteSchemaTransformProvider}.
+ *
+ * <p><b>Internal only:</b> This class is actively being worked on, and it 
will likely change. We
+ * provide no backwards compatibility guarantees, and it should not be 
implemented outside the Beam
+ * repository.
+ */
+@DefaultSchema(AutoValueSchema.class)
+@AutoValue
+public abstract class TFRecordWriteSchemaTransformConfiguration {
+
+  public void validate() {
+    checkNotNull(getNumShards(), "To write to TFRecord, numShards must be 
specified.");
+    checkNotNull(getCompression(), "To write to TFRecord, compression must be 
specified.");
+    checkNotNull(getNoSpilling(), "To write to TFRecord, noSpilling must be 
specified.");

Review Comment:
   See above comment for these null checks (same for the if-conditionals below).
   
   If an AutoValue getter field is not marked with `@Nullable`, the Beam schema 
converter will force it to be required and fail when initialized with null 
values.



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordWriteSchemaTransformProvider.java:
##########
@@ -0,0 +1,268 @@
+/*
+ * 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.service.AutoService;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaRegistry;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider;
+import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider;
+import org.apache.beam.sdk.schemas.transforms.providers.ErrorHandling;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn.Element;
+import org.apache.beam.sdk.transforms.DoFn.ProcessElement;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionRowTuple;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.sdk.values.TupleTagList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@AutoService(SchemaTransformProvider.class)
+public class TFRecordWriteSchemaTransformProvider
+    extends 
TypedSchemaTransformProvider<TFRecordWriteSchemaTransformConfiguration> {
+  private static final String IDENTIFIER = 
"beam:schematransform:org.apache.beam:tfrecord_write:v1";
+  private static final String INPUT = "input";
+  private static final String OUTPUT = "output";
+  private static final String ERROR = "errors";
+  public static final TupleTag<byte[]> OUTPUT_TAG = new TupleTag<byte[]>() {};
+  public static final TupleTag<Row> ERROR_TAG = new TupleTag<Row>() {};
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TFRecordWriteSchemaTransformProvider.class);
+
+  /** Returns the expected class of the configuration. */
+  @Override
+  protected Class<TFRecordWriteSchemaTransformConfiguration> 
configurationClass() {
+    return TFRecordWriteSchemaTransformConfiguration.class;
+  }

Review Comment:
   nit: similar to above, this is not needed anymore



##########
sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordReadSchemaTransformConfiguration.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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 static 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument;
+import static 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull;
+import static 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState;
+
+import com.google.auto.value.AutoValue;
+import java.io.IOException;
+import java.io.Serializable;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.io.fs.MatchResult;
+import org.apache.beam.sdk.schemas.AutoValueSchema;
+import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
+import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription;
+import org.apache.beam.sdk.schemas.transforms.providers.ErrorHandling;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings;
+
+/**
+ * Configuration for reading from TFRecord.
+ *
+ * <p>This class is meant to be used with {@link 
TFRecordReadSchemaTransformProvider}.
+ *
+ * <p><b>Internal only:</b> This class is actively being worked on, and it 
will likely change. We
+ * provide no backwards compatibility guarantees, and it should not be 
implemented outside the Beam
+ * repository.
+ */
+@DefaultSchema(AutoValueSchema.class)
+@AutoValue
+public abstract class TFRecordReadSchemaTransformConfiguration implements 
Serializable {
+
+  public void validate() {
+    String invalidConfigMessage = "Invalid TFRecord Read configuration: ";
+    checkNotNull(getValidate(), "To read from TFRecord, validation must be 
specified.");
+    checkNotNull(getCompression(), "To read from TFRecord, compression must be 
specified.");

Review Comment:
   nit: these checks are not needed. AutoValue will ensure that these fields 
are not nullable (I believe it throws an error when you build the object)



##########
sdks/java/core/src/test/java/org/apache/beam/sdk/io/TFRecordSchemaTransformProviderTest.java:
##########
@@ -0,0 +1,609 @@
+/*
+ * 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 static org.apache.beam.sdk.io.Compression.AUTO;
+import static org.apache.beam.sdk.io.Compression.DEFLATE;
+import static org.apache.beam.sdk.io.Compression.GZIP;
+import static org.apache.beam.sdk.io.Compression.UNCOMPRESSED;
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.CoreMatchers.startsWith;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.in;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.junit.internal.matchers.ThrowableMessageMatcher.hasMessage;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.ServiceLoader;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import 
org.apache.beam.sdk.io.TFRecordReadSchemaTransformProvider.TFRecordReadSchemaTransform;
+import 
org.apache.beam.sdk.io.TFRecordWriteSchemaTransformProvider.TFRecordWriteSchemaTransform;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransform;
+import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider;
+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.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.BaseEncoding;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.ByteStreams;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for TFRecordIO Read and Write transforms. */
+@RunWith(JUnit4.class)
+public class TFRecordSchemaTransformProviderTest {
+
+  /*
+  From 
https://github.com/apache/beam/blob/master/sdks/python/apache_beam/io/tfrecordio_test.py
+  Created by running following code in python:
+  >>> import tensorflow as tf
+  >>> import base64
+  >>> writer = tf.python_io.TFRecordWriter('/tmp/python_foo.tfrecord')
+  >>> writer.write('foo')
+  >>> writer.close()
+  >>> with open('/tmp/python_foo.tfrecord', 'rb') as f:
+  ...   data = base64.b64encode(f.read())
+  ...   print data
+  */
+  private static final String FOO_RECORD_BASE64 = 
"AwAAAAAAAACwmUkOZm9vYYq+/g==";
+
+  // Same as above but containing two records ['foo', 'bar']
+  private static final String FOO_BAR_RECORD_BASE64 =
+      "AwAAAAAAAACwmUkOZm9vYYq+/gMAAAAAAAAAsJlJDmJhckYA5cg=";
+  private static final String BAR_FOO_RECORD_BASE64 =
+      "AwAAAAAAAACwmUkOYmFyRgDlyAMAAAAAAAAAsJlJDmZvb2GKvv4=";
+
+  private static final String[] FOO_RECORDS = {"foo"};
+  private static final String[] FOO_BAR_RECORDS = {"foo", "bar"};
+
+  private static final Iterable<String> EMPTY = Collections.emptyList();
+  private static final Iterable<String> SMALL = makeLines(1, 4);
+  private static final Iterable<String> LARGE = makeLines(1000, 4);
+  private static final Iterable<String> LARGE_RECORDS = makeLines(100, 100000);
+
+  @Rule public TemporaryFolder tempFolder = new TemporaryFolder();
+
+  @Rule public TestPipeline readPipeline = TestPipeline.create();
+
+  @Rule public TestPipeline writePipeline = TestPipeline.create();
+
+  @Rule public ExpectedException expectedException = ExpectedException.none();
+
+  private static TFRecordReadSchemaTransformConfiguration readConfig;
+  private static TFRecordWriteSchemaTransformConfiguration writeConfig;
+
+  @Before
+  public void setUp() throws IOException {
+    readConfig =
+        TFRecordReadSchemaTransformConfiguration.builder()
+            .setValidate(false)
+            .setCompression("AUTO")
+            .setFilePattern("foo.*")
+            .build();
+
+    writeConfig =
+        TFRecordWriteSchemaTransformConfiguration.builder()
+            .setOutputPrefix(tempFolder.getRoot().toPath().toString())
+            .setFilenameSuffix("bar")
+            .setShardTemplate("xyz")
+            .setNumShards(10)
+            .setCompression("UNCOMPRESSED")
+            .setNoSpilling(true)
+            .build();

Review Comment:
   nit: i think you can initialize these inline? 



-- 
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: github-unsubscr...@beam.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to