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

ahmedabualsaud 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 a96fa74f0cc Pass Java SchemaTransform descriptions to Python SDK 
(#29606)
a96fa74f0cc is described below

commit a96fa74f0ccb8d0c5490ba2872bfd52675a15558
Author: Ahmed Abualsaud <65791736+ahmedab...@users.noreply.github.com>
AuthorDate: Mon Dec 11 00:35:32 2023 +0300

    Pass Java SchemaTransform descriptions to Python SDK (#29606)
    
    * pipe thru schematransform descriptions
---
 .../beam/sdk/schemas/transforms/SchemaTransformProvider.java   |  6 +++++-
 .../schemas/transforms/TypedSchemaTransformProviderTest.java   |  6 ++++++
 .../BigQueryStorageWriteApiSchemaTransformProvider.java        | 10 ++++++++++
 3 files changed, 21 insertions(+), 1 deletion(-)

diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/SchemaTransformProvider.java
 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/SchemaTransformProvider.java
index e73ec5d870c..c76d7a25e69 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/SchemaTransformProvider.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/SchemaTransformProvider.java
@@ -36,7 +36,11 @@ public interface SchemaTransformProvider {
   /** Returns an id that uniquely represents this transform. */
   String identifier();
 
-  /** Returns a description of this transform to be used for documentation. */
+  /**
+   * Returns a description regarding the {@link SchemaTransform} represented 
by the {@link
+   * SchemaTransformProvider}. Please keep the language generic (i.e. not 
specific to any
+   * programming language). The description may be markdown formatted.
+   */
   default String description() {
     return "";
   }
diff --git 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/TypedSchemaTransformProviderTest.java
 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/TypedSchemaTransformProviderTest.java
index db7b1436a12..2b698f4f67b 100644
--- 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/TypedSchemaTransformProviderTest.java
+++ 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/TypedSchemaTransformProviderTest.java
@@ -61,6 +61,11 @@ public class TypedSchemaTransformProviderTest {
       return "fake:v1";
     }
 
+    @Override
+    public String description() {
+      return "Description of fake provider";
+    }
+
     @Override
     protected Class<Configuration> configurationClass() {
       return Configuration.class;
@@ -115,6 +120,7 @@ public class TypedSchemaTransformProviderTest {
     Configuration outputConfig = ((FakeSchemaTransform) 
provider.from(inputConfig)).config;
     assertEquals("field1", outputConfig.getField1());
     assertEquals(13, outputConfig.getField2().intValue());
+    assertEquals("Description of fake provider", provider.description());
   }
 
   @Test
diff --git 
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProvider.java
 
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProvider.java
index 98cc246ce0d..8c4edd2244b 100644
--- 
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProvider.java
+++ 
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProvider.java
@@ -98,6 +98,16 @@ public class BigQueryStorageWriteApiSchemaTransformProvider
     return 
String.format("beam:schematransform:org.apache.beam:bigquery_storage_write:v2");
   }
 
+  @Override
+  public String description() {
+    return String.format(
+        "Writes data to BigQuery using the Storage Write API 
(https://cloud.google.com/bigquery/docs/write-api)."
+            + "\n\nThis expects a single PCollection of Beam Rows and outputs 
two dead-letter queues (DLQ) that "
+            + "contain failed rows. The first DLQ has tag [%s] and contains 
the failed rows. The second DLQ has "
+            + "tag [%s] and contains failed rows and along with their 
respective errors.",
+        FAILED_ROWS_TAG, FAILED_ROWS_WITH_ERRORS_TAG);
+  }
+
   @Override
   public List<String> inputCollectionNames() {
     return Collections.singletonList(INPUT_ROWS_TAG);

Reply via email to