This is an automated email from the ASF dual-hosted git repository.

yhu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git


The following commit(s) were added to refs/heads/master by this push:
     new 3de1eb18345 [25705] Refactor BigQuery SchemaTransforms naming (#25706)
3de1eb18345 is described below

commit 3de1eb183450fb1c9bca0438c5d6c4fa55d26e61
Author: Damon <[email protected]>
AuthorDate: Mon Mar 13 16:05:55 2023 -0700

    [25705] Refactor BigQuery SchemaTransforms naming (#25706)
---
 ...eryExportReadSchemaTransformConfiguration.java} | 18 ++++----
 ...BigQueryExportReadSchemaTransformProvider.java} | 36 +++++++++-------
 ...ileLoadsWriteSchemaTransformConfiguration.java} | 18 ++++----
 ...ueryFileLoadsWriteSchemaTransformProvider.java} | 32 ++++++++------
 ...ueryExportReadSchemaTransformProviderTest.java} | 49 +++++++++++----------
 ...FileLoadsWriteSchemaTransformProviderTest.java} | 50 +++++++++++++---------
 6 files changed, 111 insertions(+), 92 deletions(-)

diff --git 
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaTransformReadConfiguration.java
 
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryExportReadSchemaTransformConfiguration.java
similarity index 80%
rename from 
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaTransformReadConfiguration.java
rename to 
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryExportReadSchemaTransformConfiguration.java
index 5c500cdfc85..9eb4ac87ff9 100644
--- 
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaTransformReadConfiguration.java
+++ 
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryExportReadSchemaTransformConfiguration.java
@@ -28,7 +28,7 @@ import org.apache.beam.sdk.values.TypeDescriptor;
 /**
  * Configuration for reading from BigQuery.
  *
- * <p>This class is meant to be used with {@link 
BigQuerySchemaTransformReadProvider}.
+ * <p>This class is meant to be used with {@link 
BigQueryExportReadSchemaTransformProvider}.
  *
  * <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
@@ -39,17 +39,17 @@ import org.apache.beam.sdk.values.TypeDescriptor;
 })
 @DefaultSchema(AutoValueSchema.class)
 @AutoValue
-public abstract class BigQuerySchemaTransformReadConfiguration {
+public abstract class BigQueryExportReadSchemaTransformConfiguration {
 
-  /** Instantiates a {@link BigQuerySchemaTransformReadConfiguration.Builder}. 
*/
+  /** Instantiates a {@link 
BigQueryExportReadSchemaTransformConfiguration.Builder}. */
   public static Builder builder() {
-    return new AutoValue_BigQuerySchemaTransformReadConfiguration.Builder();
+    return new 
AutoValue_BigQueryExportReadSchemaTransformConfiguration.Builder();
   }
 
   private static final AutoValueSchema AUTO_VALUE_SCHEMA = new 
AutoValueSchema();
-  private static final 
TypeDescriptor<BigQuerySchemaTransformReadConfiguration> TYPE_DESCRIPTOR =
-      TypeDescriptor.of(BigQuerySchemaTransformReadConfiguration.class);
-  private static final 
SerializableFunction<BigQuerySchemaTransformReadConfiguration, Row>
+  private static final 
TypeDescriptor<BigQueryExportReadSchemaTransformConfiguration>
+      TYPE_DESCRIPTOR = 
TypeDescriptor.of(BigQueryExportReadSchemaTransformConfiguration.class);
+  private static final 
SerializableFunction<BigQueryExportReadSchemaTransformConfiguration, Row>
       ROW_SERIALIZABLE_FUNCTION = 
AUTO_VALUE_SCHEMA.toRowFunction(TYPE_DESCRIPTOR);
 
   /** Serializes configuration to a {@link Row}. */
@@ -94,7 +94,7 @@ public abstract class 
BigQuerySchemaTransformReadConfiguration {
     /** Enables BigQuery's Standard SQL dialect when reading from a query. */
     public abstract Builder setUseStandardSql(Boolean value);
 
-    /** Builds the {@link BigQuerySchemaTransformReadConfiguration} 
configuration. */
-    public abstract BigQuerySchemaTransformReadConfiguration build();
+    /** Builds the {@link BigQueryExportReadSchemaTransformConfiguration} 
configuration. */
+    public abstract BigQueryExportReadSchemaTransformConfiguration build();
   }
 }
diff --git 
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaTransformReadProvider.java
 
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryExportReadSchemaTransformProvider.java
similarity index 81%
rename from 
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaTransformReadProvider.java
rename to 
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryExportReadSchemaTransformProvider.java
index 8c1debbf463..a7613a82c58 100644
--- 
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaTransformReadProvider.java
+++ 
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryExportReadSchemaTransformProvider.java
@@ -18,6 +18,7 @@
 package org.apache.beam.sdk.io.gcp.bigquery;
 
 import com.google.api.services.bigquery.model.TableRow;
+import com.google.auto.service.AutoService;
 import java.util.Collections;
 import java.util.List;
 import org.apache.beam.sdk.annotations.Experimental;
@@ -25,6 +26,7 @@ import org.apache.beam.sdk.annotations.Experimental.Kind;
 import org.apache.beam.sdk.annotations.Internal;
 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.schemas.transforms.TypedSchemaTransformProvider;
 import org.apache.beam.sdk.transforms.MapElements;
 import org.apache.beam.sdk.transforms.PTransform;
@@ -37,7 +39,7 @@ import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
 
 /**
  * An implementation of {@link TypedSchemaTransformProvider} for BigQuery read 
jobs configured using
- * {@link BigQuerySchemaTransformReadConfiguration}.
+ * {@link BigQueryExportReadSchemaTransformConfiguration}.
  *
  * <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
@@ -48,28 +50,30 @@ import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
 })
 @Internal
 @Experimental(Kind.SCHEMAS)
-public class BigQuerySchemaTransformReadProvider
-    extends 
TypedSchemaTransformProvider<BigQuerySchemaTransformReadConfiguration> {
+@AutoService(SchemaTransformProvider.class)
+public class BigQueryExportReadSchemaTransformProvider
+    extends 
TypedSchemaTransformProvider<BigQueryExportReadSchemaTransformConfiguration> {
 
-  private static final String API = "bigquery";
+  private static final String IDENTIFIER =
+      "beam:schematransform:org.apache.beam:bigquery_export_read:v1";
   private static final String OUTPUT_TAG = "OUTPUT";
 
   /** Returns the expected class of the configuration. */
   @Override
-  protected Class<BigQuerySchemaTransformReadConfiguration> 
configurationClass() {
-    return BigQuerySchemaTransformReadConfiguration.class;
+  protected Class<BigQueryExportReadSchemaTransformConfiguration> 
configurationClass() {
+    return BigQueryExportReadSchemaTransformConfiguration.class;
   }
 
   /** Returns the expected {@link SchemaTransform} of the configuration. */
   @Override
-  protected SchemaTransform from(BigQuerySchemaTransformReadConfiguration 
configuration) {
-    return new BigQueryReadSchemaTransform(configuration);
+  protected SchemaTransform 
from(BigQueryExportReadSchemaTransformConfiguration configuration) {
+    return new BigQueryExportSchemaTransform(configuration);
   }
 
   /** Implementation of the {@link TypedSchemaTransformProvider} identifier 
method. */
   @Override
   public String identifier() {
-    return String.format("%s:read", API);
+    return IDENTIFIER;
   }
 
   /**
@@ -92,12 +96,12 @@ public class BigQuerySchemaTransformReadProvider
 
   /**
    * An implementation of {@link SchemaTransform} for BigQuery read jobs 
configured using {@link
-   * BigQuerySchemaTransformReadConfiguration}.
+   * BigQueryExportReadSchemaTransformConfiguration}.
    */
-  private static class BigQueryReadSchemaTransform implements SchemaTransform {
-    private final BigQuerySchemaTransformReadConfiguration configuration;
+  private static class BigQueryExportSchemaTransform implements 
SchemaTransform {
+    private final BigQueryExportReadSchemaTransformConfiguration configuration;
 
-    BigQueryReadSchemaTransform(BigQuerySchemaTransformReadConfiguration 
configuration) {
+    
BigQueryExportSchemaTransform(BigQueryExportReadSchemaTransformConfiguration 
configuration) {
       this.configuration = configuration;
     }
 
@@ -110,17 +114,17 @@ public class BigQuerySchemaTransformReadProvider
 
   /**
    * An implementation of {@link PTransform} for BigQuery read jobs configured 
using {@link
-   * BigQuerySchemaTransformReadConfiguration}.
+   * BigQueryExportReadSchemaTransformConfiguration}.
    */
   static class PCollectionRowTupleTransform
       extends PTransform<PCollectionRowTuple, PCollectionRowTuple> {
 
-    private final BigQuerySchemaTransformReadConfiguration configuration;
+    private final BigQueryExportReadSchemaTransformConfiguration configuration;
 
     /** An instance of {@link BigQueryServices} used for testing. */
     private BigQueryServices testBigQueryServices = null;
 
-    PCollectionRowTupleTransform(BigQuerySchemaTransformReadConfiguration 
configuration) {
+    
PCollectionRowTupleTransform(BigQueryExportReadSchemaTransformConfiguration 
configuration) {
       this.configuration = configuration;
     }
 
diff --git 
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaTransformWriteConfiguration.java
 
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryFileLoadsWriteSchemaTransformConfiguration.java
similarity index 77%
rename from 
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaTransformWriteConfiguration.java
rename to 
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryFileLoadsWriteSchemaTransformConfiguration.java
index 5cbea3c49f0..0c91ce1cd07 100644
--- 
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaTransformWriteConfiguration.java
+++ 
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryFileLoadsWriteSchemaTransformConfiguration.java
@@ -27,7 +27,7 @@ import org.apache.beam.sdk.values.TypeDescriptor;
 /**
  * Configuration for writing to BigQuery.
  *
- * <p>This class is meant to be used with {@link 
BigQuerySchemaTransformWriteProvider}.
+ * <p>This class is meant to be used with {@link 
BigQueryFileLoadsWriteSchemaTransformProvider}.
  *
  * <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
@@ -35,17 +35,17 @@ import org.apache.beam.sdk.values.TypeDescriptor;
  */
 @DefaultSchema(AutoValueSchema.class)
 @AutoValue
-public abstract class BigQuerySchemaTransformWriteConfiguration {
+public abstract class BigQueryFileLoadsWriteSchemaTransformConfiguration {
 
-  /** Instantiates a {@link 
BigQuerySchemaTransformWriteConfiguration.Builder}. */
+  /** Instantiates a {@link 
BigQueryFileLoadsWriteSchemaTransformConfiguration.Builder}. */
   public static Builder builder() {
-    return new AutoValue_BigQuerySchemaTransformWriteConfiguration.Builder();
+    return new 
AutoValue_BigQueryFileLoadsWriteSchemaTransformConfiguration.Builder();
   }
 
   private static final AutoValueSchema AUTO_VALUE_SCHEMA = new 
AutoValueSchema();
-  private static final 
TypeDescriptor<BigQuerySchemaTransformWriteConfiguration> TYPE_DESCRIPTOR =
-      TypeDescriptor.of(BigQuerySchemaTransformWriteConfiguration.class);
-  private static final 
SerializableFunction<BigQuerySchemaTransformWriteConfiguration, Row>
+  private static final 
TypeDescriptor<BigQueryFileLoadsWriteSchemaTransformConfiguration>
+      TYPE_DESCRIPTOR = 
TypeDescriptor.of(BigQueryFileLoadsWriteSchemaTransformConfiguration.class);
+  private static final 
SerializableFunction<BigQueryFileLoadsWriteSchemaTransformConfiguration, Row>
       ROW_SERIALIZABLE_FUNCTION = 
AUTO_VALUE_SCHEMA.toRowFunction(TYPE_DESCRIPTOR);
 
   /**
@@ -80,7 +80,7 @@ public abstract class 
BigQuerySchemaTransformWriteConfiguration {
     /** Specifies what to do with existing data in the table, in case the 
table already exists. */
     public abstract Builder setWriteDisposition(String value);
 
-    /** Builds the {@link BigQuerySchemaTransformWriteConfiguration} 
configuration. */
-    public abstract BigQuerySchemaTransformWriteConfiguration build();
+    /** Builds the {@link BigQueryFileLoadsWriteSchemaTransformConfiguration} 
configuration. */
+    public abstract BigQueryFileLoadsWriteSchemaTransformConfiguration build();
   }
 }
diff --git 
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaTransformWriteProvider.java
 
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryFileLoadsWriteSchemaTransformProvider.java
similarity index 88%
rename from 
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaTransformWriteProvider.java
rename to 
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryFileLoadsWriteSchemaTransformProvider.java
index 87839cceb6a..278d121e87a 100644
--- 
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaTransformWriteProvider.java
+++ 
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryFileLoadsWriteSchemaTransformProvider.java
@@ -21,6 +21,7 @@ import com.google.api.services.bigquery.model.Table;
 import com.google.api.services.bigquery.model.TableReference;
 import com.google.api.services.bigquery.model.TableRow;
 import com.google.api.services.bigquery.model.TableSchema;
+import com.google.auto.service.AutoService;
 import java.io.IOException;
 import java.util.Collections;
 import java.util.List;
@@ -34,6 +35,7 @@ import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.schemas.io.InvalidConfigurationException;
 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.transforms.MapElements;
 import org.apache.beam.sdk.transforms.PTransform;
@@ -45,7 +47,7 @@ import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.Visi
 
 /**
  * An implementation of {@link TypedSchemaTransformProvider} for BigQuery 
write jobs configured
- * using {@link BigQuerySchemaTransformWriteConfiguration}.
+ * using {@link BigQueryFileLoadsWriteSchemaTransformConfiguration}.
  *
  * <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
@@ -56,28 +58,30 @@ import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.Visi
 })
 @Internal
 @Experimental(Kind.SCHEMAS)
-public class BigQuerySchemaTransformWriteProvider
-    extends 
TypedSchemaTransformProvider<BigQuerySchemaTransformWriteConfiguration> {
+@AutoService(SchemaTransformProvider.class)
+public class BigQueryFileLoadsWriteSchemaTransformProvider
+    extends 
TypedSchemaTransformProvider<BigQueryFileLoadsWriteSchemaTransformConfiguration>
 {
 
-  private static final String API = "bigquery";
+  private static final String IDENTIFIER =
+      "beam:schematransform:org.apache.beam:bigquery_fileloads_write:v1";
   static final String INPUT_TAG = "INPUT";
 
   /** Returns the expected class of the configuration. */
   @Override
-  protected Class<BigQuerySchemaTransformWriteConfiguration> 
configurationClass() {
-    return BigQuerySchemaTransformWriteConfiguration.class;
+  protected Class<BigQueryFileLoadsWriteSchemaTransformConfiguration> 
configurationClass() {
+    return BigQueryFileLoadsWriteSchemaTransformConfiguration.class;
   }
 
   /** Returns the expected {@link SchemaTransform} of the configuration. */
   @Override
-  protected SchemaTransform from(BigQuerySchemaTransformWriteConfiguration 
configuration) {
+  protected SchemaTransform 
from(BigQueryFileLoadsWriteSchemaTransformConfiguration configuration) {
     return new BigQueryWriteSchemaTransform(configuration);
   }
 
   /** Implementation of the {@link TypedSchemaTransformProvider} identifier 
method. */
   @Override
   public String identifier() {
-    return String.format("%s:write", API);
+    return IDENTIFIER;
   }
 
   /**
@@ -100,12 +104,12 @@ public class BigQuerySchemaTransformWriteProvider
 
   /**
    * A {@link SchemaTransform} that performs {@link BigQueryIO.Write}s based 
on a {@link
-   * BigQuerySchemaTransformWriteConfiguration}.
+   * BigQueryFileLoadsWriteSchemaTransformConfiguration}.
    */
   private static class BigQueryWriteSchemaTransform implements SchemaTransform 
{
-    private final BigQuerySchemaTransformWriteConfiguration configuration;
+    private final BigQueryFileLoadsWriteSchemaTransformConfiguration 
configuration;
 
-    BigQueryWriteSchemaTransform(BigQuerySchemaTransformWriteConfiguration 
configuration) {
+    
BigQueryWriteSchemaTransform(BigQueryFileLoadsWriteSchemaTransformConfiguration 
configuration) {
       this.configuration = configuration;
     }
 
@@ -121,17 +125,17 @@ public class BigQuerySchemaTransformWriteProvider
 
   /**
    * An implementation of {@link PTransform} for BigQuery write jobs 
configured using {@link
-   * BigQuerySchemaTransformWriteConfiguration}.
+   * BigQueryFileLoadsWriteSchemaTransformConfiguration}.
    */
   static class PCollectionRowTupleTransform
       extends PTransform<PCollectionRowTuple, PCollectionRowTuple> {
 
-    private final BigQuerySchemaTransformWriteConfiguration configuration;
+    private final BigQueryFileLoadsWriteSchemaTransformConfiguration 
configuration;
 
     /** An instance of {@link BigQueryServices} used for testing. */
     private BigQueryServices testBigQueryServices = null;
 
-    PCollectionRowTupleTransform(BigQuerySchemaTransformWriteConfiguration 
configuration) {
+    
PCollectionRowTupleTransform(BigQueryFileLoadsWriteSchemaTransformConfiguration 
configuration) {
       this.configuration = configuration;
     }
 
diff --git 
a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaTransformReadProviderTest.java
 
b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryExportReadSchemaTransformProviderTest.java
similarity index 82%
rename from 
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaTransformReadProviderTest.java
rename to 
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryExportReadSchemaTransformProviderTest.java
index 5bca0cb2ad3..af2f1351e18 100644
--- 
a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaTransformReadProviderTest.java
+++ 
b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryExportReadSchemaTransformProviderTest.java
@@ -31,8 +31,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ExecutionException;
+import 
org.apache.beam.sdk.io.gcp.bigquery.BigQueryExportReadSchemaTransformProvider.PCollectionRowTupleTransform;
 import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TypedRead;
-import 
org.apache.beam.sdk.io.gcp.bigquery.BigQuerySchemaTransformReadProvider.PCollectionRowTupleTransform;
 import org.apache.beam.sdk.io.gcp.testing.FakeBigQueryServices;
 import org.apache.beam.sdk.io.gcp.testing.FakeDatasetService;
 import org.apache.beam.sdk.io.gcp.testing.FakeJobService;
@@ -60,9 +60,9 @@ import org.junit.rules.TemporaryFolder;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/** Test for {@link BigQuerySchemaTransformReadProvider}. */
+/** Test for {@link BigQueryExportReadSchemaTransformProvider}. */
 @RunWith(JUnit4.class)
-public class BigQuerySchemaTransformReadProviderTest {
+public class BigQueryExportReadSchemaTransformProviderTest {
   private static final String PROJECT = "fakeproject";
   private static final String DATASET = "fakedataset";
   private static final String TABLE_ID = "faketable";
@@ -130,31 +130,32 @@ public class BigQuerySchemaTransformReadProviderTest {
   public void testQuery() {
     // Previous attempts using FakeBigQueryServices with a Read configuration 
using a query failed.
     // For now, we test using DisplayData and the toTypedRead method.
-    List<Pair<BigQuerySchemaTransformReadConfiguration.Builder, 
TypedRead<TableRow>>> cases =
+    List<Pair<BigQueryExportReadSchemaTransformConfiguration.Builder, 
TypedRead<TableRow>>> cases =
         Arrays.asList(
             Pair.of(
-                
BigQuerySchemaTransformReadConfiguration.builder().setQuery(QUERY),
+                
BigQueryExportReadSchemaTransformConfiguration.builder().setQuery(QUERY),
                 BigQueryIO.readTableRowsWithSchema().fromQuery(QUERY)),
             Pair.of(
-                BigQuerySchemaTransformReadConfiguration.builder()
+                BigQueryExportReadSchemaTransformConfiguration.builder()
                     .setQuery(QUERY)
                     .setQueryLocation(LOCATION),
                 
BigQueryIO.readTableRowsWithSchema().fromQuery(QUERY).withQueryLocation(LOCATION)),
             Pair.of(
-                BigQuerySchemaTransformReadConfiguration.builder()
+                BigQueryExportReadSchemaTransformConfiguration.builder()
                     .setQuery(QUERY)
                     .setUseStandardSql(true),
                 
BigQueryIO.readTableRowsWithSchema().fromQuery(QUERY).usingStandardSql()),
             Pair.of(
-                BigQuerySchemaTransformReadConfiguration.builder()
+                BigQueryExportReadSchemaTransformConfiguration.builder()
                     .setQuery(QUERY)
                     .setUseStandardSql(false),
                 BigQueryIO.readTableRowsWithSchema().fromQuery(QUERY)));
 
-    for (Pair<BigQuerySchemaTransformReadConfiguration.Builder, 
TypedRead<TableRow>> caze : cases) {
+    for (Pair<BigQueryExportReadSchemaTransformConfiguration.Builder, 
TypedRead<TableRow>> caze :
+        cases) {
       Map<Identifier, Item> want = DisplayData.from(caze.getRight()).asMap();
-      SchemaTransformProvider provider = new 
BigQuerySchemaTransformReadProvider();
-      BigQuerySchemaTransformReadConfiguration configuration = 
caze.getLeft().build();
+      SchemaTransformProvider provider = new 
BigQueryExportReadSchemaTransformProvider();
+      BigQueryExportReadSchemaTransformConfiguration configuration = 
caze.getLeft().build();
       Row configurationRow = configuration.toBeamRow();
       SchemaTransform schemaTransform = provider.from(configurationRow);
       PCollectionRowTupleTransform pCollectionRowTupleTransform =
@@ -167,9 +168,9 @@ public class BigQuerySchemaTransformReadProviderTest {
 
   @Test
   public void testExtract() {
-    SchemaTransformProvider provider = new 
BigQuerySchemaTransformReadProvider();
-    BigQuerySchemaTransformReadConfiguration configuration =
-        
BigQuerySchemaTransformReadConfiguration.builder().setTableSpec(TABLE_SPEC).build();
+    SchemaTransformProvider provider = new 
BigQueryExportReadSchemaTransformProvider();
+    BigQueryExportReadSchemaTransformConfiguration configuration =
+        
BigQueryExportReadSchemaTransformConfiguration.builder().setTableSpec(TABLE_SPEC).build();
     Row configurationRow = configuration.toBeamRow();
     SchemaTransform schemaTransform = provider.from(configurationRow);
     PCollectionRowTupleTransform pCollectionRowTupleTransform =
@@ -188,25 +189,27 @@ public class BigQuerySchemaTransformReadProviderTest {
 
   @Test
   public void testInvalidConfiguration() {
-    SchemaTransformProvider provider = new 
BigQuerySchemaTransformReadProvider();
+    SchemaTransformProvider provider = new 
BigQueryExportReadSchemaTransformProvider();
     for (Pair<
-            BigQuerySchemaTransformReadConfiguration.Builder,
+            BigQueryExportReadSchemaTransformConfiguration.Builder,
             ? extends Class<? extends RuntimeException>>
         caze :
             Arrays.asList(
                 Pair.of(
-                    BigQuerySchemaTransformReadConfiguration.builder(),
+                    BigQueryExportReadSchemaTransformConfiguration.builder(),
                     IllegalArgumentException.class),
                 Pair.of(
-                    BigQuerySchemaTransformReadConfiguration.builder()
+                    BigQueryExportReadSchemaTransformConfiguration.builder()
                         .setQuery(QUERY)
                         .setTableSpec(TABLE_SPEC),
                     IllegalStateException.class),
                 Pair.of(
-                    
BigQuerySchemaTransformReadConfiguration.builder().setQueryLocation(LOCATION),
+                    BigQueryExportReadSchemaTransformConfiguration.builder()
+                        .setQueryLocation(LOCATION),
                     IllegalArgumentException.class),
                 Pair.of(
-                    
BigQuerySchemaTransformReadConfiguration.builder().setUseStandardSql(true),
+                    BigQueryExportReadSchemaTransformConfiguration.builder()
+                        .setUseStandardSql(true),
                     IllegalArgumentException.class))) {
       Row configurationRow = caze.getLeft().build().toBeamRow();
       SchemaTransform schemaTransform = provider.from(configurationRow);
@@ -220,9 +223,9 @@ public class BigQuerySchemaTransformReadProviderTest {
 
   @Test
   public void testInvalidInput() {
-    SchemaTransformProvider provider = new 
BigQuerySchemaTransformReadProvider();
-    BigQuerySchemaTransformReadConfiguration configuration =
-        
BigQuerySchemaTransformReadConfiguration.builder().setTableSpec(TABLE_SPEC).build();
+    SchemaTransformProvider provider = new 
BigQueryExportReadSchemaTransformProvider();
+    BigQueryExportReadSchemaTransformConfiguration configuration =
+        
BigQueryExportReadSchemaTransformConfiguration.builder().setTableSpec(TABLE_SPEC).build();
     Row configurationRow = configuration.toBeamRow();
     SchemaTransform schemaTransform = provider.from(configurationRow);
     PCollectionRowTupleTransform pCollectionRowTupleTransform =
diff --git 
a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaTransformWriteProviderTest.java
 
b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryFileLoadsWriteSchemaTransformProviderTest.java
similarity index 85%
rename from 
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaTransformWriteProviderTest.java
rename to 
sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryFileLoadsWriteSchemaTransformProviderTest.java
index 7e77b72a16c..eb881801cb7 100644
--- 
a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaTransformWriteProviderTest.java
+++ 
b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryFileLoadsWriteSchemaTransformProviderTest.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.sdk.io.gcp.bigquery;
 
-import static 
org.apache.beam.sdk.io.gcp.bigquery.BigQuerySchemaTransformWriteProvider.INPUT_TAG;
+import static 
org.apache.beam.sdk.io.gcp.bigquery.BigQueryFileLoadsWriteSchemaTransformProvider.INPUT_TAG;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertThrows;
@@ -31,9 +31,9 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import 
org.apache.beam.sdk.io.gcp.bigquery.BigQueryFileLoadsWriteSchemaTransformProvider.PCollectionRowTupleTransform;
 import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition;
 import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition;
-import 
org.apache.beam.sdk.io.gcp.bigquery.BigQuerySchemaTransformWriteProvider.PCollectionRowTupleTransform;
 import org.apache.beam.sdk.io.gcp.testing.FakeBigQueryServices;
 import org.apache.beam.sdk.io.gcp.testing.FakeDatasetService;
 import org.apache.beam.sdk.io.gcp.testing.FakeJobService;
@@ -59,9 +59,9 @@ import org.junit.rules.TemporaryFolder;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/** Test for {@link BigQuerySchemaTransformWriteProvider}. */
+/** Test for {@link BigQueryFileLoadsWriteSchemaTransformProvider}. */
 @RunWith(JUnit4.class)
-public class BigQuerySchemaTransformWriteProviderTest {
+public class BigQueryFileLoadsWriteSchemaTransformProviderTest {
 
   private static final String PROJECT = "fakeproject";
   private static final String DATASET = "fakedataset";
@@ -109,9 +109,9 @@ public class BigQuerySchemaTransformWriteProviderTest {
 
   @Test
   public void testLoad() throws IOException, InterruptedException {
-    SchemaTransformProvider provider = new 
BigQuerySchemaTransformWriteProvider();
-    BigQuerySchemaTransformWriteConfiguration configuration =
-        BigQuerySchemaTransformWriteConfiguration.builder()
+    SchemaTransformProvider provider = new 
BigQueryFileLoadsWriteSchemaTransformProvider();
+    BigQueryFileLoadsWriteSchemaTransformConfiguration configuration =
+        BigQueryFileLoadsWriteSchemaTransformConfiguration.builder()
             .setTableSpec(BigQueryHelpers.toTableSpec(TABLE_REFERENCE))
             .setWriteDisposition(WriteDisposition.WRITE_TRUNCATE.name())
             .setCreateDisposition(CreateDisposition.CREATE_IF_NEEDED.name())
@@ -134,29 +134,33 @@ public class BigQuerySchemaTransformWriteProviderTest {
 
   @Test
   public void testValidatePipelineOptions() {
-    List<Pair<BigQuerySchemaTransformWriteConfiguration.Builder, Class<? 
extends Exception>>>
+    List<
+            Pair<
+                BigQueryFileLoadsWriteSchemaTransformConfiguration.Builder,
+                Class<? extends Exception>>>
         cases =
             Arrays.asList(
                 Pair.of(
-                    BigQuerySchemaTransformWriteConfiguration.builder()
+                    
BigQueryFileLoadsWriteSchemaTransformConfiguration.builder()
                         .setTableSpec("project.doesnot.exist")
                         
.setCreateDisposition(CreateDisposition.CREATE_NEVER.name())
                         
.setWriteDisposition(WriteDisposition.WRITE_APPEND.name()),
                     InvalidConfigurationException.class),
                 Pair.of(
-                    BigQuerySchemaTransformWriteConfiguration.builder()
+                    
BigQueryFileLoadsWriteSchemaTransformConfiguration.builder()
                         .setTableSpec(String.format("%s.%s.%s", PROJECT, 
DATASET, "doesnotexist"))
                         
.setCreateDisposition(CreateDisposition.CREATE_NEVER.name())
                         
.setWriteDisposition(WriteDisposition.WRITE_EMPTY.name()),
                     InvalidConfigurationException.class),
                 Pair.of(
-                    BigQuerySchemaTransformWriteConfiguration.builder()
+                    
BigQueryFileLoadsWriteSchemaTransformConfiguration.builder()
                         .setTableSpec("project.doesnot.exist")
                         
.setCreateDisposition(CreateDisposition.CREATE_IF_NEEDED.name())
                         
.setWriteDisposition(WriteDisposition.WRITE_APPEND.name()),
                     null));
-    for (Pair<BigQuerySchemaTransformWriteConfiguration.Builder, Class<? 
extends Exception>> caze :
-        cases) {
+    for (Pair<
+            BigQueryFileLoadsWriteSchemaTransformConfiguration.Builder, 
Class<? extends Exception>>
+        caze : cases) {
       PCollectionRowTupleTransform transform = 
transformFrom(caze.getLeft().build());
       if (caze.getRight() != null) {
         assertThrows(caze.getRight(), () -> 
transform.validate(p.getOptions()));
@@ -168,11 +172,14 @@ public class BigQuerySchemaTransformWriteProviderTest {
 
   @Test
   public void testToWrite() {
-    List<Pair<BigQuerySchemaTransformWriteConfiguration.Builder, 
BigQueryIO.Write<TableRow>>>
+    List<
+            Pair<
+                BigQueryFileLoadsWriteSchemaTransformConfiguration.Builder,
+                BigQueryIO.Write<TableRow>>>
         cases =
             Arrays.asList(
                 Pair.of(
-                    BigQuerySchemaTransformWriteConfiguration.builder()
+                    
BigQueryFileLoadsWriteSchemaTransformConfiguration.builder()
                         
.setTableSpec(BigQueryHelpers.toTableSpec(TABLE_REFERENCE))
                         
.setCreateDisposition(CreateDisposition.CREATE_NEVER.name())
                         
.setWriteDisposition(WriteDisposition.WRITE_EMPTY.name()),
@@ -182,7 +189,7 @@ public class BigQuerySchemaTransformWriteProviderTest {
                         .withWriteDisposition(WriteDisposition.WRITE_EMPTY)
                         .withSchema(TABLE_SCHEMA)),
                 Pair.of(
-                    BigQuerySchemaTransformWriteConfiguration.builder()
+                    
BigQueryFileLoadsWriteSchemaTransformConfiguration.builder()
                         
.setTableSpec(BigQueryHelpers.toTableSpec(TABLE_REFERENCE))
                         
.setCreateDisposition(CreateDisposition.CREATE_IF_NEEDED.name())
                         
.setWriteDisposition(WriteDisposition.WRITE_TRUNCATE.name()),
@@ -191,8 +198,9 @@ public class BigQuerySchemaTransformWriteProviderTest {
                         
.withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
                         .withWriteDisposition(WriteDisposition.WRITE_TRUNCATE)
                         .withSchema(TABLE_SCHEMA)));
-    for (Pair<BigQuerySchemaTransformWriteConfiguration.Builder, 
BigQueryIO.Write<TableRow>> caze :
-        cases) {
+    for (Pair<
+            BigQueryFileLoadsWriteSchemaTransformConfiguration.Builder, 
BigQueryIO.Write<TableRow>>
+        caze : cases) {
       PCollectionRowTupleTransform transform = 
transformFrom(caze.getLeft().build());
       Map<Identifier, Item> gotDisplayData = 
DisplayData.from(transform.toWrite(SCHEMA)).asMap();
       Map<Identifier, Item> wantDisplayData = 
DisplayData.from(caze.getRight()).asMap();
@@ -231,7 +239,7 @@ public class BigQuerySchemaTransformWriteProviderTest {
 
     PCollectionRowTupleTransform transform =
         transformFrom(
-            BigQuerySchemaTransformWriteConfiguration.builder()
+            BigQueryFileLoadsWriteSchemaTransformConfiguration.builder()
                 .setTableSpec(BigQueryHelpers.toTableSpec(TABLE_REFERENCE))
                 
.setCreateDisposition(CreateDisposition.CREATE_IF_NEEDED.name())
                 .setWriteDisposition(WriteDisposition.WRITE_APPEND.name())
@@ -247,8 +255,8 @@ public class BigQuerySchemaTransformWriteProviderTest {
   }
 
   private PCollectionRowTupleTransform transformFrom(
-      BigQuerySchemaTransformWriteConfiguration configuration) {
-    SchemaTransformProvider provider = new 
BigQuerySchemaTransformWriteProvider();
+      BigQueryFileLoadsWriteSchemaTransformConfiguration configuration) {
+    SchemaTransformProvider provider = new 
BigQueryFileLoadsWriteSchemaTransformProvider();
     PCollectionRowTupleTransform transform =
         (PCollectionRowTupleTransform) 
provider.from(configuration.toBeamRow()).buildTransform();
 


Reply via email to