You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2020/06/28 01:36:50 UTC

[GitHub] [beam] robinyqiu commented on a change in pull request #12090: [BEAM-10336] Add SchemaIO abstraction and implement for PubSub

robinyqiu commented on a change in pull request #12090:
URL: https://github.com/apache/beam/pull/12090#discussion_r446584887



##########
File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubIOJsonTable.java
##########
@@ -139,65 +134,19 @@ static PubsubIOJsonTable withConfiguration(PubsubIOTableConfiguration config) {
 
   @Override
   public Schema getSchema() {
-    return config.getSchema();
+    return schema;
   }
 
   @Override
   public PCollection<Row> buildIOReader(PBegin begin) {
-    PCollectionTuple rowsWithDlq =
-        begin
-            .apply("ReadFromPubsub", readMessagesWithAttributes())
-            .apply(
-                "PubsubMessageToRow",
-                PubsubMessageToRow.builder()
-                    .messageSchema(getSchema())
-                    .useDlq(config.useDlq())
-                    .useFlatSchema(config.getUseFlatSchema())
-                    .build());
-    rowsWithDlq.get(MAIN_TAG).setRowSchema(getSchema());
-
-    if (config.useDlq()) {
-      rowsWithDlq.get(DLQ_TAG).apply(writeMessagesToDlq());
-    }
-
-    return rowsWithDlq.get(MAIN_TAG);
-  }
-
-  private PubsubIO.Read<PubsubMessage> readMessagesWithAttributes() {
-    PubsubIO.Read<PubsubMessage> read =
-        PubsubIO.readMessagesWithAttributes().fromTopic(config.getTopic());
-
-    return config.useTimestampAttribute()
-        ? read.withTimestampAttribute(config.getTimestampAttribute())
-        : read;
-  }
-
-  private PubsubIO.Write<PubsubMessage> writeMessagesToDlq() {
-    PubsubIO.Write<PubsubMessage> write = PubsubIO.writeMessages().to(config.getDeadLetterQueue());
-
-    return config.useTimestampAttribute()
-        ? write.withTimestampAttribute(config.getTimestampAttribute())
-        : write;
+    PTransform<PBegin, PCollection<Row>> readerTransform = pubsubSchemaIO.buildReader();
+    return readerTransform.expand(begin);

Review comment:
       Nit: there is nothing wrong with this line, but usually we would write `begin.apply(readerTransform)` (same below)

##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubSchemaCapableIOProvider.java
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.gcp.pubsub;
+
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.ATTRIBUTES_FIELD;
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.DLQ_TAG;
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.MAIN_TAG;
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.PAYLOAD_FIELD;
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.TIMESTAMP_FIELD;
+import static org.apache.beam.sdk.schemas.Schema.TypeName.ROW;
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.service.AutoService;
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.io.InvalidConfigurationException;
+import org.apache.beam.sdk.schemas.io.SchemaCapableIOProvider;
+import org.apache.beam.sdk.schemas.io.SchemaIO;
+import org.apache.beam.sdk.schemas.transforms.DropFields;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ToJson;
+import org.apache.beam.sdk.transforms.WithTimestamps;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+
+/**
+ * {@link org.apache.beam.sdk.schemas.io.SchemaCapableIOProvider} to create {@link PubsubSchemaIO}
+ * that implements {@link org.apache.beam.sdk.schemas.io.SchemaIO}.
+ */
+@Internal
+@AutoService(SchemaCapableIOProvider.class)
+public class PubsubSchemaCapableIOProvider implements SchemaCapableIOProvider {
+  public static final FieldType VARCHAR = FieldType.STRING;
+  public static final FieldType TIMESTAMP = FieldType.DATETIME;
+
+  /** Returns an id that uniquely represents this IO. */
+  @Override
+  public String identifier() {
+    return "pubsub";
+  }
+
+  /**
+   * Returns the expected schema of the configuration object. Note this is distinct from the schema
+   * of the data source itself.
+   */
+  @Override
+  public Schema configurationSchema() {
+    return Schema.builder()
+        .addNullableField("timestampAttributeKey", FieldType.STRING)
+        .addNullableField("deadLetterQueue", FieldType.STRING)
+        .build();
+  }
+
+  /**
+   * Produce a SchemaIO given a String representing the data's location, the schema of the data that
+   * resides there, and some IO-specific configuration object.
+   */
+  @Override
+  public PubsubSchemaIO from(String location, Row configuration, Schema dataSchema) {

Review comment:
       How about adding `validateConfiguration()` here to make sure the configuration row matches `configurationSchema`?

##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubSchemaCapableIOProvider.java
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.gcp.pubsub;
+
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.ATTRIBUTES_FIELD;
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.DLQ_TAG;
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.MAIN_TAG;
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.PAYLOAD_FIELD;
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.TIMESTAMP_FIELD;
+import static org.apache.beam.sdk.schemas.Schema.TypeName.ROW;
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.service.AutoService;
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.io.InvalidConfigurationException;
+import org.apache.beam.sdk.schemas.io.SchemaCapableIOProvider;
+import org.apache.beam.sdk.schemas.io.SchemaIO;
+import org.apache.beam.sdk.schemas.transforms.DropFields;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ToJson;
+import org.apache.beam.sdk.transforms.WithTimestamps;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+
+/**
+ * {@link org.apache.beam.sdk.schemas.io.SchemaCapableIOProvider} to create {@link PubsubSchemaIO}
+ * that implements {@link org.apache.beam.sdk.schemas.io.SchemaIO}.
+ */
+@Internal
+@AutoService(SchemaCapableIOProvider.class)
+public class PubsubSchemaCapableIOProvider implements SchemaCapableIOProvider {
+  public static final FieldType VARCHAR = FieldType.STRING;
+  public static final FieldType TIMESTAMP = FieldType.DATETIME;
+
+  /** Returns an id that uniquely represents this IO. */
+  @Override
+  public String identifier() {
+    return "pubsub";
+  }
+
+  /**
+   * Returns the expected schema of the configuration object. Note this is distinct from the schema
+   * of the data source itself.
+   */
+  @Override
+  public Schema configurationSchema() {
+    return Schema.builder()
+        .addNullableField("timestampAttributeKey", FieldType.STRING)
+        .addNullableField("deadLetterQueue", FieldType.STRING)
+        .build();
+  }
+
+  /**
+   * Produce a SchemaIO given a String representing the data's location, the schema of the data that
+   * resides there, and some IO-specific configuration object.
+   */
+  @Override
+  public PubsubSchemaIO from(String location, Row configuration, Schema dataSchema) {
+    validateDlq(configuration.getValue("deadLetterQueue"));
+    validateEventTimestamp(dataSchema);
+    return PubsubSchemaIO.withConfiguration(location, configuration, dataSchema);
+  }
+
+  private void validateEventTimestamp(Schema schema) {
+    if (!PubsubSchemaIO.fieldPresent(schema, TIMESTAMP_FIELD, TIMESTAMP)) {
+      throw new InvalidConfigurationException(
+          "Unsupported schema specified for Pubsub source in CREATE TABLE."
+              + "CREATE TABLE for Pubsub topic must include at least 'event_timestamp' field of "
+              + "type 'TIMESTAMP'");
+    }
+  }
+
+  private void validateDlq(String deadLetterQueue) {
+    if (deadLetterQueue != null && deadLetterQueue.isEmpty()) {
+      throw new InvalidConfigurationException("Dead letter queue topic name is not specified");
+    }
+  }
+
+  /** An abstraction to create schema aware IOs. */
+  @Internal
+  private static class PubsubSchemaIO implements SchemaIO, Serializable {
+    protected final Row config;
+    protected final Schema dataSchema;
+    protected final String location;
+    protected final Boolean useFlatSchema;
+
+    private PubsubSchemaIO(String location, Row config, Schema dataSchema) {
+      this.config = config;
+      this.dataSchema = dataSchema;
+      this.location = location;
+      this.useFlatSchema = !definesAttributeAndPayload(dataSchema);
+    }
+
+    static PubsubSchemaIO withConfiguration(String location, Row config, Schema dataSchema) {
+      return new PubsubSchemaIO(location, config, dataSchema);
+    }
+
+    @Override
+    public Schema schema() {
+      return dataSchema;
+    }
+
+    @Override
+    public PTransform<PBegin, PCollection<Row>> buildReader() {
+      return new PTransform<PBegin, PCollection<Row>>() {
+        @Override
+        public PCollection<Row> expand(PBegin begin) {
+          PCollectionTuple rowsWithDlq =
+              begin
+                  .apply("ReadFromPubsub", readMessagesWithAttributes())
+                  .apply(
+                      "PubsubMessageToRow",
+                      PubsubMessageToRow.builder()
+                          .messageSchema(dataSchema)
+                          .useDlq(useDlqCheck(config))
+                          .useFlatSchema(useFlatSchema)
+                          .build());
+          rowsWithDlq.get(MAIN_TAG).setRowSchema(dataSchema);
+
+          if (useDlqCheck(config)) {
+            rowsWithDlq.get(DLQ_TAG).apply(writeMessagesToDlq());
+          }
+
+          return rowsWithDlq.get(MAIN_TAG);
+        }
+      };
+    }
+
+    @Override
+    public PTransform<PCollection<Row>, POutput> buildWriter() {
+      if (!useFlatSchema) {
+        throw new UnsupportedOperationException(
+            "Writing to a Pubsub topic is only supported for flattened schemas");
+      }
+
+      return new PTransform<PCollection<Row>, POutput>() {
+        @Override
+        public POutput expand(PCollection<Row> input) {
+          return input
+              .apply(RowToPubsubMessage.fromTableConfig(config, useFlatSchema))
+              .apply(createPubsubMessageWrite());
+        }
+      };
+    }
+
+    private PubsubIO.Read<PubsubMessage> readMessagesWithAttributes() {
+      PubsubIO.Read<PubsubMessage> read = PubsubIO.readMessagesWithAttributes().fromTopic(location);
+
+      return useTimestampAttribute(config)
+          ? read.withTimestampAttribute(config.getValue("timestampAttributeKey"))
+          : read;
+    }
+
+    private PubsubIO.Write<PubsubMessage> createPubsubMessageWrite() {
+      PubsubIO.Write<PubsubMessage> write = PubsubIO.writeMessages().to(location);
+      if (useTimestampAttribute(config)) {
+        write = write.withTimestampAttribute(config.getValue("timestampAttributeKey"));
+      }
+      return write;
+    }
+
+    private PubsubIO.Write<PubsubMessage> writeMessagesToDlq() {
+      PubsubIO.Write<PubsubMessage> write =
+          PubsubIO.writeMessages().to(config.getString("deadLetterQueue"));
+
+      return useTimestampAttribute(config)
+          ? write.withTimestampAttribute(config.getString("timestampAttributeKey"))
+          : write;
+    }
+
+    private boolean useDlqCheck(Row config) {
+      return config.getValue("deadLetterQueue") != null;
+    }
+
+    private boolean useTimestampAttribute(Row config) {
+      return config.getValue("timestampAttributeKey") != null;
+    }
+
+    private boolean definesAttributeAndPayload(Schema schema) {
+      return fieldPresent(
+              schema, ATTRIBUTES_FIELD, Schema.FieldType.map(VARCHAR.withNullable(false), VARCHAR))
+          && (schema.hasField(PAYLOAD_FIELD)
+              && ROW.equals(schema.getField(PAYLOAD_FIELD).getType().getTypeName()));
+    }
+
+    private static boolean fieldPresent(
+        Schema schema, String field, Schema.FieldType expectedType) {
+      return schema.hasField(field)
+          && expectedType.equivalent(
+              schema.getField(field).getType(), Schema.EquivalenceNullablePolicy.IGNORE);
+    }
+  }
+
+  /**
+   * A {@link PTransform} to convert {@link Row} to {@link PubsubMessage} with JSON payload.
+   *
+   * <p>Currently only supports writing a flat schema into a JSON payload. This means that all Row
+   * field values are written to the {@link PubsubMessage} JSON payload, except for {@code
+   * event_timestamp}, which is either ignored or written to the message attributes, depending on
+   * whether config.getValue("timestampAttributeKey") is set.
+   */
+  private static class RowToPubsubMessage
+      extends PTransform<PCollection<Row>, PCollection<PubsubMessage>> {
+    private final Row config;
+
+    private RowToPubsubMessage(Row config, Boolean useFlatSchema) {
+      checkArgument(useFlatSchema, "RowToPubsubMessage is only supported for flattened schemas.");
+
+      this.config = config;
+    }
+
+    public static RowToPubsubMessage fromTableConfig(Row config, Boolean useFlatSchema) {

Review comment:
       Same here.

##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubSchemaCapableIOProvider.java
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.gcp.pubsub;
+
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.ATTRIBUTES_FIELD;
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.DLQ_TAG;
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.MAIN_TAG;
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.PAYLOAD_FIELD;
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.TIMESTAMP_FIELD;
+import static org.apache.beam.sdk.schemas.Schema.TypeName.ROW;
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.service.AutoService;
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.io.InvalidConfigurationException;
+import org.apache.beam.sdk.schemas.io.SchemaCapableIOProvider;
+import org.apache.beam.sdk.schemas.io.SchemaIO;
+import org.apache.beam.sdk.schemas.transforms.DropFields;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ToJson;
+import org.apache.beam.sdk.transforms.WithTimestamps;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+
+/**
+ * {@link org.apache.beam.sdk.schemas.io.SchemaCapableIOProvider} to create {@link PubsubSchemaIO}
+ * that implements {@link org.apache.beam.sdk.schemas.io.SchemaIO}.
+ */
+@Internal
+@AutoService(SchemaCapableIOProvider.class)
+public class PubsubSchemaCapableIOProvider implements SchemaCapableIOProvider {
+  public static final FieldType VARCHAR = FieldType.STRING;
+  public static final FieldType TIMESTAMP = FieldType.DATETIME;
+
+  /** Returns an id that uniquely represents this IO. */
+  @Override
+  public String identifier() {
+    return "pubsub";
+  }
+
+  /**
+   * Returns the expected schema of the configuration object. Note this is distinct from the schema
+   * of the data source itself.
+   */
+  @Override
+  public Schema configurationSchema() {
+    return Schema.builder()
+        .addNullableField("timestampAttributeKey", FieldType.STRING)
+        .addNullableField("deadLetterQueue", FieldType.STRING)
+        .build();
+  }
+
+  /**
+   * Produce a SchemaIO given a String representing the data's location, the schema of the data that
+   * resides there, and some IO-specific configuration object.
+   */
+  @Override
+  public PubsubSchemaIO from(String location, Row configuration, Schema dataSchema) {
+    validateDlq(configuration.getValue("deadLetterQueue"));
+    validateEventTimestamp(dataSchema);
+    return PubsubSchemaIO.withConfiguration(location, configuration, dataSchema);
+  }
+
+  private void validateEventTimestamp(Schema schema) {
+    if (!PubsubSchemaIO.fieldPresent(schema, TIMESTAMP_FIELD, TIMESTAMP)) {
+      throw new InvalidConfigurationException(
+          "Unsupported schema specified for Pubsub source in CREATE TABLE."
+              + "CREATE TABLE for Pubsub topic must include at least 'event_timestamp' field of "
+              + "type 'TIMESTAMP'");
+    }
+  }
+
+  private void validateDlq(String deadLetterQueue) {
+    if (deadLetterQueue != null && deadLetterQueue.isEmpty()) {
+      throw new InvalidConfigurationException("Dead letter queue topic name is not specified");
+    }
+  }
+
+  /** An abstraction to create schema aware IOs. */
+  @Internal
+  private static class PubsubSchemaIO implements SchemaIO, Serializable {
+    protected final Row config;
+    protected final Schema dataSchema;
+    protected final String location;
+    protected final Boolean useFlatSchema;
+
+    private PubsubSchemaIO(String location, Row config, Schema dataSchema) {
+      this.config = config;
+      this.dataSchema = dataSchema;
+      this.location = location;
+      this.useFlatSchema = !definesAttributeAndPayload(dataSchema);
+    }
+
+    static PubsubSchemaIO withConfiguration(String location, Row config, Schema dataSchema) {

Review comment:
       This static method is no longer necessary as the class became private. You can call the constructor directly from the outer class.

##########
File path: sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/io/SchemaIO.java
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.schemas.io;
+
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+
+public interface SchemaIO {

Review comment:
       `SchemaCapableIOProvider` and `SchemaIO` are very important interfaces. Please add thorough class-level and method-level javadoc to explain what they are to help people better understand these concepts.

##########
File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubJsonTableProvider.java
##########
@@ -54,125 +47,29 @@ public String getTableType() {
   }

Review comment:
       I am not sure if we expect the table type here to be the same as `PubsubSchemaCapableIOProvider.identifier()`. WDYT Brian?

##########
File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubIOJsonTable.java
##########
@@ -122,14 +115,16 @@
 @Experimental
 class PubsubIOJsonTable extends BaseBeamTable implements Serializable {
 
-  protected final PubsubIOTableConfiguration config;
+  protected final SchemaIO pubsubSchemaIO;
+  protected final Schema schema;
 
-  private PubsubIOJsonTable(PubsubIOTableConfiguration config) {
-    this.config = config;
+  private PubsubIOJsonTable(SchemaIO pubsubSchemaIO, Schema schema) {
+    this.pubsubSchemaIO = pubsubSchemaIO;
+    this.schema = schema;
   }
 
-  static PubsubIOJsonTable withConfiguration(PubsubIOTableConfiguration config) {
-    return new PubsubIOJsonTable(config);
+  static PubsubIOJsonTable withConfiguration(SchemaIO pubsubSchemaIO, Schema schema) {

Review comment:
       Now that configuration is gone, this function could be better named as something like `fromSchemaIO()`.
   
   Also, the second parameter `schema` doesn't seem necessary. You can get that information from `pubsubSchemaIO.schema()`.

##########
File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubJsonTableProvider.java
##########
@@ -54,125 +47,29 @@ public String getTableType() {
   }
 
   @Override
-  public BeamSqlTable buildBeamSqlTable(Table tableDefintion) {
-    JSONObject tableProperties = tableDefintion.getProperties();
+  public BeamSqlTable buildBeamSqlTable(Table tableDefinition) {
+    JSONObject tableProperties = tableDefinition.getProperties();
     String timestampAttributeKey = tableProperties.getString("timestampAttributeKey");
     String deadLetterQueue = tableProperties.getString("deadLetterQueue");
-    validateDlq(deadLetterQueue);
-
-    Schema schema = tableDefintion.getSchema();
-    validateEventTimestamp(schema);
-
-    PubsubIOTableConfiguration config =
-        PubsubIOTableConfiguration.builder()
-            .setSchema(schema)
-            .setTimestampAttribute(timestampAttributeKey)
-            .setDeadLetterQueue(deadLetterQueue)
-            .setTopic(tableDefintion.getLocation())
-            .setUseFlatSchema(!definesAttributeAndPayload(schema))
-            .build();
-
-    return PubsubIOJsonTable.withConfiguration(config);
-  }
-
-  private void validateEventTimestamp(Schema schema) {
-    if (!fieldPresent(schema, TIMESTAMP_FIELD, TIMESTAMP)) {
-      throw new InvalidTableException(
-          "Unsupported schema specified for Pubsub source in CREATE TABLE."
-              + "CREATE TABLE for Pubsub topic must include at least 'event_timestamp' field of "
-              + "type 'TIMESTAMP'");
-    }
-  }
-
-  private boolean definesAttributeAndPayload(Schema schema) {
-    return fieldPresent(
-            schema, ATTRIBUTES_FIELD, Schema.FieldType.map(VARCHAR.withNullable(false), VARCHAR))
-        && (schema.hasField(PAYLOAD_FIELD)
-            && ROW.equals(schema.getField(PAYLOAD_FIELD).getType().getTypeName()));
-  }
-
-  private boolean fieldPresent(Schema schema, String field, Schema.FieldType expectedType) {
-    return schema.hasField(field)
-        && expectedType.equivalent(
-            schema.getField(field).getType(), Schema.EquivalenceNullablePolicy.IGNORE);
-  }
-
-  private void validateDlq(String deadLetterQueue) {
-    if (deadLetterQueue != null && deadLetterQueue.isEmpty()) {
-      throw new InvalidTableException("Dead letter queue topic name is not specified");
-    }
-  }
-
-  @AutoValue
-  public abstract static class PubsubIOTableConfiguration implements Serializable {
-    public boolean useDlq() {
-      return getDeadLetterQueue() != null;
-    }
-
-    public boolean useTimestampAttribute() {
-      return getTimestampAttribute() != null;
-    }
-
-    /** Determines whether or not the messages should be represented with a flattened schema. */
-    abstract boolean getUseFlatSchema();
-
-    /**
-     * Optional attribute key of the Pubsub message from which to extract the event timestamp.
-     *
-     * <p>This attribute has to conform to the same requirements as in {@link
-     * PubsubIO.Read.Builder#withTimestampAttribute}.
-     *
-     * <p>Short version: it has to be either millis since epoch or string in RFC 3339 format.
-     *
-     * <p>If the attribute is specified then event timestamps will be extracted from the specified
-     * attribute. If it is not specified then message publish timestamp will be used.
-     */
-    @Nullable
-    abstract String getTimestampAttribute();
-
-    /**
-     * Optional topic path which will be used as a dead letter queue.
-     *
-     * <p>Messages that cannot be processed will be sent to this topic. If it is not specified then
-     * exception will be thrown for errors during processing causing the pipeline to crash.
-     */
-    @Nullable
-    abstract String getDeadLetterQueue();
 
-    /**
-     * Pubsub topic name.
-     *
-     * <p>Topic is the only way to specify the Pubsub source. Explicitly specifying the subscription
-     * is not supported at the moment. Subscriptions are automatically created (but not deleted).
-     */
-    abstract String getTopic();
+    Schema schema = tableDefinition.getSchema();

Review comment:
       This is the same as `dataSchema`.
   
   Also, almost all the local variables defined here is used only once. Maybe moving them inline will make the function looks better.

##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubSchemaCapableIOProvider.java
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.gcp.pubsub;
+
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.ATTRIBUTES_FIELD;
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.DLQ_TAG;
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.MAIN_TAG;
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.PAYLOAD_FIELD;
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.TIMESTAMP_FIELD;
+import static org.apache.beam.sdk.schemas.Schema.TypeName.ROW;
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.service.AutoService;
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.io.InvalidConfigurationException;
+import org.apache.beam.sdk.schemas.io.SchemaCapableIOProvider;
+import org.apache.beam.sdk.schemas.io.SchemaIO;
+import org.apache.beam.sdk.schemas.transforms.DropFields;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ToJson;
+import org.apache.beam.sdk.transforms.WithTimestamps;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+
+/**
+ * {@link org.apache.beam.sdk.schemas.io.SchemaCapableIOProvider} to create {@link PubsubSchemaIO}
+ * that implements {@link org.apache.beam.sdk.schemas.io.SchemaIO}.
+ */
+@Internal
+@AutoService(SchemaCapableIOProvider.class)
+public class PubsubSchemaCapableIOProvider implements SchemaCapableIOProvider {
+  public static final FieldType VARCHAR = FieldType.STRING;
+  public static final FieldType TIMESTAMP = FieldType.DATETIME;
+
+  /** Returns an id that uniquely represents this IO. */
+  @Override
+  public String identifier() {
+    return "pubsub";
+  }
+
+  /**
+   * Returns the expected schema of the configuration object. Note this is distinct from the schema
+   * of the data source itself.
+   */
+  @Override
+  public Schema configurationSchema() {
+    return Schema.builder()
+        .addNullableField("timestampAttributeKey", FieldType.STRING)
+        .addNullableField("deadLetterQueue", FieldType.STRING)
+        .build();
+  }
+
+  /**
+   * Produce a SchemaIO given a String representing the data's location, the schema of the data that
+   * resides there, and some IO-specific configuration object.
+   */
+  @Override
+  public PubsubSchemaIO from(String location, Row configuration, Schema dataSchema) {
+    validateDlq(configuration.getValue("deadLetterQueue"));
+    validateEventTimestamp(dataSchema);
+    return PubsubSchemaIO.withConfiguration(location, configuration, dataSchema);
+  }
+
+  private void validateEventTimestamp(Schema schema) {
+    if (!PubsubSchemaIO.fieldPresent(schema, TIMESTAMP_FIELD, TIMESTAMP)) {
+      throw new InvalidConfigurationException(
+          "Unsupported schema specified for Pubsub source in CREATE TABLE."
+              + "CREATE TABLE for Pubsub topic must include at least 'event_timestamp' field of "
+              + "type 'TIMESTAMP'");
+    }
+  }
+
+  private void validateDlq(String deadLetterQueue) {
+    if (deadLetterQueue != null && deadLetterQueue.isEmpty()) {
+      throw new InvalidConfigurationException("Dead letter queue topic name is not specified");
+    }
+  }
+
+  /** An abstraction to create schema aware IOs. */
+  @Internal
+  private static class PubsubSchemaIO implements SchemaIO, Serializable {
+    protected final Row config;
+    protected final Schema dataSchema;
+    protected final String location;
+    protected final Boolean useFlatSchema;
+
+    private PubsubSchemaIO(String location, Row config, Schema dataSchema) {
+      this.config = config;
+      this.dataSchema = dataSchema;
+      this.location = location;
+      this.useFlatSchema = !definesAttributeAndPayload(dataSchema);
+    }
+
+    static PubsubSchemaIO withConfiguration(String location, Row config, Schema dataSchema) {
+      return new PubsubSchemaIO(location, config, dataSchema);
+    }
+
+    @Override
+    public Schema schema() {
+      return dataSchema;
+    }
+
+    @Override
+    public PTransform<PBegin, PCollection<Row>> buildReader() {
+      return new PTransform<PBegin, PCollection<Row>>() {
+        @Override
+        public PCollection<Row> expand(PBegin begin) {
+          PCollectionTuple rowsWithDlq =
+              begin
+                  .apply("ReadFromPubsub", readMessagesWithAttributes())
+                  .apply(
+                      "PubsubMessageToRow",
+                      PubsubMessageToRow.builder()
+                          .messageSchema(dataSchema)
+                          .useDlq(useDlqCheck(config))
+                          .useFlatSchema(useFlatSchema)
+                          .build());
+          rowsWithDlq.get(MAIN_TAG).setRowSchema(dataSchema);
+
+          if (useDlqCheck(config)) {
+            rowsWithDlq.get(DLQ_TAG).apply(writeMessagesToDlq());
+          }
+
+          return rowsWithDlq.get(MAIN_TAG);
+        }
+      };
+    }
+
+    @Override
+    public PTransform<PCollection<Row>, POutput> buildWriter() {
+      if (!useFlatSchema) {
+        throw new UnsupportedOperationException(
+            "Writing to a Pubsub topic is only supported for flattened schemas");
+      }
+
+      return new PTransform<PCollection<Row>, POutput>() {
+        @Override
+        public POutput expand(PCollection<Row> input) {
+          return input
+              .apply(RowToPubsubMessage.fromTableConfig(config, useFlatSchema))
+              .apply(createPubsubMessageWrite());
+        }
+      };
+    }
+
+    private PubsubIO.Read<PubsubMessage> readMessagesWithAttributes() {
+      PubsubIO.Read<PubsubMessage> read = PubsubIO.readMessagesWithAttributes().fromTopic(location);
+
+      return useTimestampAttribute(config)
+          ? read.withTimestampAttribute(config.getValue("timestampAttributeKey"))
+          : read;
+    }
+
+    private PubsubIO.Write<PubsubMessage> createPubsubMessageWrite() {
+      PubsubIO.Write<PubsubMessage> write = PubsubIO.writeMessages().to(location);
+      if (useTimestampAttribute(config)) {
+        write = write.withTimestampAttribute(config.getValue("timestampAttributeKey"));
+      }
+      return write;
+    }
+
+    private PubsubIO.Write<PubsubMessage> writeMessagesToDlq() {
+      PubsubIO.Write<PubsubMessage> write =
+          PubsubIO.writeMessages().to(config.getString("deadLetterQueue"));
+
+      return useTimestampAttribute(config)
+          ? write.withTimestampAttribute(config.getString("timestampAttributeKey"))
+          : write;
+    }
+
+    private boolean useDlqCheck(Row config) {
+      return config.getValue("deadLetterQueue") != null;
+    }
+
+    private boolean useTimestampAttribute(Row config) {
+      return config.getValue("timestampAttributeKey") != null;
+    }
+
+    private boolean definesAttributeAndPayload(Schema schema) {
+      return fieldPresent(
+              schema, ATTRIBUTES_FIELD, Schema.FieldType.map(VARCHAR.withNullable(false), VARCHAR))
+          && (schema.hasField(PAYLOAD_FIELD)
+              && ROW.equals(schema.getField(PAYLOAD_FIELD).getType().getTypeName()));
+    }
+
+    private static boolean fieldPresent(
+        Schema schema, String field, Schema.FieldType expectedType) {
+      return schema.hasField(field)
+          && expectedType.equivalent(
+              schema.getField(field).getType(), Schema.EquivalenceNullablePolicy.IGNORE);
+    }
+  }
+
+  /**
+   * A {@link PTransform} to convert {@link Row} to {@link PubsubMessage} with JSON payload.
+   *
+   * <p>Currently only supports writing a flat schema into a JSON payload. This means that all Row
+   * field values are written to the {@link PubsubMessage} JSON payload, except for {@code
+   * event_timestamp}, which is either ignored or written to the message attributes, depending on
+   * whether config.getValue("timestampAttributeKey") is set.
+   */
+  private static class RowToPubsubMessage

Review comment:
       Consider moving `PubsubMessageToRow` into this file as a inner class as well?

##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubSchemaCapableIOProvider.java
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.gcp.pubsub;
+
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.ATTRIBUTES_FIELD;
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.DLQ_TAG;
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.MAIN_TAG;
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.PAYLOAD_FIELD;
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.TIMESTAMP_FIELD;
+import static org.apache.beam.sdk.schemas.Schema.TypeName.ROW;
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.service.AutoService;
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.io.InvalidConfigurationException;
+import org.apache.beam.sdk.schemas.io.SchemaCapableIOProvider;
+import org.apache.beam.sdk.schemas.io.SchemaIO;
+import org.apache.beam.sdk.schemas.transforms.DropFields;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ToJson;
+import org.apache.beam.sdk.transforms.WithTimestamps;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+
+/**
+ * {@link org.apache.beam.sdk.schemas.io.SchemaCapableIOProvider} to create {@link PubsubSchemaIO}
+ * that implements {@link org.apache.beam.sdk.schemas.io.SchemaIO}.
+ */
+@Internal
+@AutoService(SchemaCapableIOProvider.class)
+public class PubsubSchemaCapableIOProvider implements SchemaCapableIOProvider {
+  public static final FieldType VARCHAR = FieldType.STRING;
+  public static final FieldType TIMESTAMP = FieldType.DATETIME;
+
+  /** Returns an id that uniquely represents this IO. */
+  @Override
+  public String identifier() {
+    return "pubsub";
+  }
+
+  /**
+   * Returns the expected schema of the configuration object. Note this is distinct from the schema
+   * of the data source itself.
+   */
+  @Override
+  public Schema configurationSchema() {
+    return Schema.builder()
+        .addNullableField("timestampAttributeKey", FieldType.STRING)
+        .addNullableField("deadLetterQueue", FieldType.STRING)
+        .build();
+  }
+
+  /**
+   * Produce a SchemaIO given a String representing the data's location, the schema of the data that
+   * resides there, and some IO-specific configuration object.
+   */
+  @Override
+  public PubsubSchemaIO from(String location, Row configuration, Schema dataSchema) {
+    validateDlq(configuration.getValue("deadLetterQueue"));
+    validateEventTimestamp(dataSchema);
+    return PubsubSchemaIO.withConfiguration(location, configuration, dataSchema);
+  }
+
+  private void validateEventTimestamp(Schema schema) {
+    if (!PubsubSchemaIO.fieldPresent(schema, TIMESTAMP_FIELD, TIMESTAMP)) {
+      throw new InvalidConfigurationException(
+          "Unsupported schema specified for Pubsub source in CREATE TABLE."
+              + "CREATE TABLE for Pubsub topic must include at least 'event_timestamp' field of "
+              + "type 'TIMESTAMP'");
+    }
+  }
+
+  private void validateDlq(String deadLetterQueue) {
+    if (deadLetterQueue != null && deadLetterQueue.isEmpty()) {
+      throw new InvalidConfigurationException("Dead letter queue topic name is not specified");
+    }
+  }
+
+  /** An abstraction to create schema aware IOs. */
+  @Internal
+  private static class PubsubSchemaIO implements SchemaIO, Serializable {
+    protected final Row config;
+    protected final Schema dataSchema;
+    protected final String location;
+    protected final Boolean useFlatSchema;
+
+    private PubsubSchemaIO(String location, Row config, Schema dataSchema) {
+      this.config = config;
+      this.dataSchema = dataSchema;
+      this.location = location;
+      this.useFlatSchema = !definesAttributeAndPayload(dataSchema);
+    }
+
+    static PubsubSchemaIO withConfiguration(String location, Row config, Schema dataSchema) {
+      return new PubsubSchemaIO(location, config, dataSchema);
+    }
+
+    @Override
+    public Schema schema() {
+      return dataSchema;
+    }
+
+    @Override
+    public PTransform<PBegin, PCollection<Row>> buildReader() {
+      return new PTransform<PBegin, PCollection<Row>>() {
+        @Override
+        public PCollection<Row> expand(PBegin begin) {
+          PCollectionTuple rowsWithDlq =
+              begin
+                  .apply("ReadFromPubsub", readMessagesWithAttributes())
+                  .apply(
+                      "PubsubMessageToRow",
+                      PubsubMessageToRow.builder()
+                          .messageSchema(dataSchema)
+                          .useDlq(useDlqCheck(config))
+                          .useFlatSchema(useFlatSchema)
+                          .build());
+          rowsWithDlq.get(MAIN_TAG).setRowSchema(dataSchema);
+
+          if (useDlqCheck(config)) {
+            rowsWithDlq.get(DLQ_TAG).apply(writeMessagesToDlq());
+          }
+
+          return rowsWithDlq.get(MAIN_TAG);
+        }
+      };
+    }
+
+    @Override
+    public PTransform<PCollection<Row>, POutput> buildWriter() {
+      if (!useFlatSchema) {
+        throw new UnsupportedOperationException(
+            "Writing to a Pubsub topic is only supported for flattened schemas");
+      }
+
+      return new PTransform<PCollection<Row>, POutput>() {
+        @Override
+        public POutput expand(PCollection<Row> input) {
+          return input
+              .apply(RowToPubsubMessage.fromTableConfig(config, useFlatSchema))
+              .apply(createPubsubMessageWrite());
+        }
+      };
+    }
+
+    private PubsubIO.Read<PubsubMessage> readMessagesWithAttributes() {
+      PubsubIO.Read<PubsubMessage> read = PubsubIO.readMessagesWithAttributes().fromTopic(location);
+
+      return useTimestampAttribute(config)
+          ? read.withTimestampAttribute(config.getValue("timestampAttributeKey"))
+          : read;
+    }
+
+    private PubsubIO.Write<PubsubMessage> createPubsubMessageWrite() {
+      PubsubIO.Write<PubsubMessage> write = PubsubIO.writeMessages().to(location);
+      if (useTimestampAttribute(config)) {
+        write = write.withTimestampAttribute(config.getValue("timestampAttributeKey"));
+      }
+      return write;
+    }
+
+    private PubsubIO.Write<PubsubMessage> writeMessagesToDlq() {
+      PubsubIO.Write<PubsubMessage> write =
+          PubsubIO.writeMessages().to(config.getString("deadLetterQueue"));
+
+      return useTimestampAttribute(config)
+          ? write.withTimestampAttribute(config.getString("timestampAttributeKey"))
+          : write;
+    }
+
+    private boolean useDlqCheck(Row config) {
+      return config.getValue("deadLetterQueue") != null;
+    }
+
+    private boolean useTimestampAttribute(Row config) {
+      return config.getValue("timestampAttributeKey") != null;
+    }
+
+    private boolean definesAttributeAndPayload(Schema schema) {
+      return fieldPresent(
+              schema, ATTRIBUTES_FIELD, Schema.FieldType.map(VARCHAR.withNullable(false), VARCHAR))
+          && (schema.hasField(PAYLOAD_FIELD)
+              && ROW.equals(schema.getField(PAYLOAD_FIELD).getType().getTypeName()));
+    }
+
+    private static boolean fieldPresent(
+        Schema schema, String field, Schema.FieldType expectedType) {
+      return schema.hasField(field)
+          && expectedType.equivalent(
+              schema.getField(field).getType(), Schema.EquivalenceNullablePolicy.IGNORE);
+    }
+  }
+
+  /**
+   * A {@link PTransform} to convert {@link Row} to {@link PubsubMessage} with JSON payload.
+   *
+   * <p>Currently only supports writing a flat schema into a JSON payload. This means that all Row
+   * field values are written to the {@link PubsubMessage} JSON payload, except for {@code
+   * event_timestamp}, which is either ignored or written to the message attributes, depending on
+   * whether config.getValue("timestampAttributeKey") is set.
+   */
+  private static class RowToPubsubMessage
+      extends PTransform<PCollection<Row>, PCollection<PubsubMessage>> {
+    private final Row config;
+
+    private RowToPubsubMessage(Row config, Boolean useFlatSchema) {

Review comment:
       This constructor can just take 2 booleans instead (`useTimestampAttribute` and `useFlatSchema`), and the duplicate `useTimestampAttribute()` function at the end of the file can be removed.

##########
File path: sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubSchemaCapableIOProvider.java
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.gcp.pubsub;
+
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.ATTRIBUTES_FIELD;
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.DLQ_TAG;
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.MAIN_TAG;
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.PAYLOAD_FIELD;
+import static org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageToRow.TIMESTAMP_FIELD;
+import static org.apache.beam.sdk.schemas.Schema.TypeName.ROW;
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.auto.service.AutoService;
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.io.InvalidConfigurationException;
+import org.apache.beam.sdk.schemas.io.SchemaCapableIOProvider;
+import org.apache.beam.sdk.schemas.io.SchemaIO;
+import org.apache.beam.sdk.schemas.transforms.DropFields;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ToJson;
+import org.apache.beam.sdk.transforms.WithTimestamps;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+
+/**
+ * {@link org.apache.beam.sdk.schemas.io.SchemaCapableIOProvider} to create {@link PubsubSchemaIO}
+ * that implements {@link org.apache.beam.sdk.schemas.io.SchemaIO}.
+ */
+@Internal
+@AutoService(SchemaCapableIOProvider.class)
+public class PubsubSchemaCapableIOProvider implements SchemaCapableIOProvider {
+  public static final FieldType VARCHAR = FieldType.STRING;
+  public static final FieldType TIMESTAMP = FieldType.DATETIME;
+
+  /** Returns an id that uniquely represents this IO. */
+  @Override
+  public String identifier() {
+    return "pubsub";
+  }
+
+  /**
+   * Returns the expected schema of the configuration object. Note this is distinct from the schema
+   * of the data source itself.
+   */
+  @Override
+  public Schema configurationSchema() {
+    return Schema.builder()
+        .addNullableField("timestampAttributeKey", FieldType.STRING)
+        .addNullableField("deadLetterQueue", FieldType.STRING)
+        .build();
+  }
+
+  /**
+   * Produce a SchemaIO given a String representing the data's location, the schema of the data that
+   * resides there, and some IO-specific configuration object.
+   */
+  @Override
+  public PubsubSchemaIO from(String location, Row configuration, Schema dataSchema) {
+    validateDlq(configuration.getValue("deadLetterQueue"));
+    validateEventTimestamp(dataSchema);
+    return PubsubSchemaIO.withConfiguration(location, configuration, dataSchema);
+  }
+
+  private void validateEventTimestamp(Schema schema) {
+    if (!PubsubSchemaIO.fieldPresent(schema, TIMESTAMP_FIELD, TIMESTAMP)) {
+      throw new InvalidConfigurationException(
+          "Unsupported schema specified for Pubsub source in CREATE TABLE."
+              + "CREATE TABLE for Pubsub topic must include at least 'event_timestamp' field of "
+              + "type 'TIMESTAMP'");
+    }
+  }
+
+  private void validateDlq(String deadLetterQueue) {
+    if (deadLetterQueue != null && deadLetterQueue.isEmpty()) {
+      throw new InvalidConfigurationException("Dead letter queue topic name is not specified");
+    }
+  }
+
+  /** An abstraction to create schema aware IOs. */
+  @Internal
+  private static class PubsubSchemaIO implements SchemaIO, Serializable {
+    protected final Row config;
+    protected final Schema dataSchema;
+    protected final String location;
+    protected final Boolean useFlatSchema;
+
+    private PubsubSchemaIO(String location, Row config, Schema dataSchema) {
+      this.config = config;
+      this.dataSchema = dataSchema;
+      this.location = location;
+      this.useFlatSchema = !definesAttributeAndPayload(dataSchema);
+    }
+
+    static PubsubSchemaIO withConfiguration(String location, Row config, Schema dataSchema) {
+      return new PubsubSchemaIO(location, config, dataSchema);
+    }
+
+    @Override
+    public Schema schema() {
+      return dataSchema;
+    }
+
+    @Override
+    public PTransform<PBegin, PCollection<Row>> buildReader() {
+      return new PTransform<PBegin, PCollection<Row>>() {
+        @Override
+        public PCollection<Row> expand(PBegin begin) {
+          PCollectionTuple rowsWithDlq =
+              begin
+                  .apply("ReadFromPubsub", readMessagesWithAttributes())
+                  .apply(
+                      "PubsubMessageToRow",
+                      PubsubMessageToRow.builder()
+                          .messageSchema(dataSchema)
+                          .useDlq(useDlqCheck(config))
+                          .useFlatSchema(useFlatSchema)
+                          .build());
+          rowsWithDlq.get(MAIN_TAG).setRowSchema(dataSchema);
+
+          if (useDlqCheck(config)) {
+            rowsWithDlq.get(DLQ_TAG).apply(writeMessagesToDlq());
+          }
+
+          return rowsWithDlq.get(MAIN_TAG);
+        }
+      };
+    }
+
+    @Override
+    public PTransform<PCollection<Row>, POutput> buildWriter() {
+      if (!useFlatSchema) {
+        throw new UnsupportedOperationException(
+            "Writing to a Pubsub topic is only supported for flattened schemas");
+      }
+
+      return new PTransform<PCollection<Row>, POutput>() {
+        @Override
+        public POutput expand(PCollection<Row> input) {
+          return input
+              .apply(RowToPubsubMessage.fromTableConfig(config, useFlatSchema))
+              .apply(createPubsubMessageWrite());
+        }
+      };
+    }
+
+    private PubsubIO.Read<PubsubMessage> readMessagesWithAttributes() {
+      PubsubIO.Read<PubsubMessage> read = PubsubIO.readMessagesWithAttributes().fromTopic(location);
+
+      return useTimestampAttribute(config)
+          ? read.withTimestampAttribute(config.getValue("timestampAttributeKey"))
+          : read;
+    }
+
+    private PubsubIO.Write<PubsubMessage> createPubsubMessageWrite() {
+      PubsubIO.Write<PubsubMessage> write = PubsubIO.writeMessages().to(location);
+      if (useTimestampAttribute(config)) {
+        write = write.withTimestampAttribute(config.getValue("timestampAttributeKey"));
+      }
+      return write;
+    }
+
+    private PubsubIO.Write<PubsubMessage> writeMessagesToDlq() {
+      PubsubIO.Write<PubsubMessage> write =
+          PubsubIO.writeMessages().to(config.getString("deadLetterQueue"));
+
+      return useTimestampAttribute(config)
+          ? write.withTimestampAttribute(config.getString("timestampAttributeKey"))
+          : write;
+    }
+
+    private boolean useDlqCheck(Row config) {
+      return config.getValue("deadLetterQueue") != null;
+    }
+
+    private boolean useTimestampAttribute(Row config) {
+      return config.getValue("timestampAttributeKey") != null;
+    }
+
+    private boolean definesAttributeAndPayload(Schema schema) {
+      return fieldPresent(
+              schema, ATTRIBUTES_FIELD, Schema.FieldType.map(VARCHAR.withNullable(false), VARCHAR))
+          && (schema.hasField(PAYLOAD_FIELD)
+              && ROW.equals(schema.getField(PAYLOAD_FIELD).getType().getTypeName()));
+    }
+
+    private static boolean fieldPresent(
+        Schema schema, String field, Schema.FieldType expectedType) {
+      return schema.hasField(field)
+          && expectedType.equivalent(
+              schema.getField(field).getType(), Schema.EquivalenceNullablePolicy.IGNORE);
+    }
+  }
+
+  /**
+   * A {@link PTransform} to convert {@link Row} to {@link PubsubMessage} with JSON payload.
+   *
+   * <p>Currently only supports writing a flat schema into a JSON payload. This means that all Row
+   * field values are written to the {@link PubsubMessage} JSON payload, except for {@code
+   * event_timestamp}, which is either ignored or written to the message attributes, depending on
+   * whether config.getValue("timestampAttributeKey") is set.
+   */
+  private static class RowToPubsubMessage
+      extends PTransform<PCollection<Row>, PCollection<PubsubMessage>> {
+    private final Row config;
+
+    private RowToPubsubMessage(Row config, Boolean useFlatSchema) {
+      checkArgument(useFlatSchema, "RowToPubsubMessage is only supported for flattened schemas.");
+
+      this.config = config;
+    }
+
+    public static RowToPubsubMessage fromTableConfig(Row config, Boolean useFlatSchema) {
+      return new RowToPubsubMessage(config, useFlatSchema);
+    }
+
+    @Override
+    public PCollection<PubsubMessage> expand(PCollection<Row> input) {
+      PCollection<Row> withTimestamp =
+          (useTimestampAttribute(config))
+              ? input.apply(
+                  WithTimestamps.of((row) -> row.getDateTime("event_timestamp").toInstant()))

Review comment:
       Consider defining a constant for `event_timestamp`?




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

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