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


##########
sdks/java/io/file-schema-transform/src/main/java/org/apache/beam/sdk/io/fileschematransform/XmlWriteSchemaTransformFormatProvider.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.fileschematransform;
+
+import static 
org.apache.beam.sdk.io.fileschematransform.FileWriteSchemaTransformFormatProviders.XML;
+import static 
org.apache.beam.sdk.io.fileschematransform.FileWriteSchemaTransformFormatProviders.applyCommonFileIOWriteFeatures;
+import static 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+import static 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState;
+
+import com.google.auto.service.AutoService;
+import java.nio.charset.Charset;
+import java.util.Optional;
+import org.apache.beam.sdk.io.FileIO;
+import 
org.apache.beam.sdk.io.fileschematransform.FileWriteSchemaTransformConfiguration.XmlConfiguration;
+import org.apache.beam.sdk.io.xml.XmlIO;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.Values;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+/** A {@link FileWriteSchemaTransformFormatProvider} for XML format. */
+@AutoService(FileWriteSchemaTransformFormatProvider.class)
+public class XmlWriteSchemaTransformFormatProvider
+    implements FileWriteSchemaTransformFormatProvider {
+
+  private static final String SUFFIX = String.format(".%s", XML);
+
+  @Override
+  public String identifier() {
+    return XML;
+  }
+
+  /**
+   * Builds a {@link PTransform} that transforms a {@link Row} {@link 
PCollection} into result
+   * {@link PCollection} file names written using {@link XmlIO.Sink} and 
{@link FileIO.Write}.
+   */
+  @Override
+  public PTransform<PCollection<Row>, PCollection<String>> buildTransform(
+      FileWriteSchemaTransformConfiguration configuration, Schema schema) {
+    return new PTransform<PCollection<Row>, PCollection<String>>() {
+      @Override
+      public PCollection<String> expand(PCollection<Row> input) {
+
+        PCollection<XmlRowAdapter> xml =
+            input.apply(
+                "Row to XML",
+                
MapElements.into(TypeDescriptor.of(XmlRowAdapter.class)).via(new RowToXmlFn()));
+
+        XmlConfiguration xmlConfig = xmlConfiguration(configuration);
+
+        checkArgument(xmlConfig.getCharset() != null, "charset must be 
specified");
+        checkArgument(xmlConfig.getRootElement() != null, "rootElement must be 
specified");

Review Comment:
   ```suggestion
           checkArgument(!Strings.isNullOrEmpty(xmlConfig.getCharset()), 
"charset must be specified");
           checkArgument(!Strings.isNullOrEmpty(xmlConfig.getRootElement()), 
"rootElement must be specified");
   ```



##########
sdks/java/io/file-schema-transform/src/main/java/org/apache/beam/sdk/io/fileschematransform/XmlWriteSchemaTransformFormatProvider.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.fileschematransform;
+
+import static 
org.apache.beam.sdk.io.fileschematransform.FileWriteSchemaTransformFormatProviders.XML;
+import static 
org.apache.beam.sdk.io.fileschematransform.FileWriteSchemaTransformFormatProviders.applyCommonFileIOWriteFeatures;
+import static 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+import static 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState;
+
+import com.google.auto.service.AutoService;
+import java.nio.charset.Charset;
+import java.util.Optional;
+import org.apache.beam.sdk.io.FileIO;
+import 
org.apache.beam.sdk.io.fileschematransform.FileWriteSchemaTransformConfiguration.XmlConfiguration;
+import org.apache.beam.sdk.io.xml.XmlIO;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.Values;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+/** A {@link FileWriteSchemaTransformFormatProvider} for XML format. */
+@AutoService(FileWriteSchemaTransformFormatProvider.class)
+public class XmlWriteSchemaTransformFormatProvider
+    implements FileWriteSchemaTransformFormatProvider {
+
+  private static final String SUFFIX = String.format(".%s", XML);
+
+  @Override
+  public String identifier() {
+    return XML;
+  }
+
+  /**
+   * Builds a {@link PTransform} that transforms a {@link Row} {@link 
PCollection} into result
+   * {@link PCollection} file names written using {@link XmlIO.Sink} and 
{@link FileIO.Write}.
+   */
+  @Override
+  public PTransform<PCollection<Row>, PCollection<String>> buildTransform(
+      FileWriteSchemaTransformConfiguration configuration, Schema schema) {
+    return new PTransform<PCollection<Row>, PCollection<String>>() {
+      @Override
+      public PCollection<String> expand(PCollection<Row> input) {
+
+        PCollection<XmlRowAdapter> xml =
+            input.apply(
+                "Row to XML",
+                
MapElements.into(TypeDescriptor.of(XmlRowAdapter.class)).via(new RowToXmlFn()));
+
+        XmlConfiguration xmlConfig = xmlConfiguration(configuration);
+
+        checkArgument(xmlConfig.getCharset() != null, "charset must be 
specified");
+        checkArgument(xmlConfig.getRootElement() != null, "rootElement must be 
specified");

Review Comment:
   In Python SDK, it's currently not supported to send non-null values over 
through expansion service (see 
[here](https://github.com/chamikaramj/beam/blob/d2342ae120ee386e03701008cdda053303d468c1/sdks/python/apache_beam/transforms/external.py#L198-L201)).
 For this reason, it's unfortunately not enough to use a `!= null` check. 
`null` values sent over Python may be defaulted to `""` for Strings and `0` for 
numbers, etc.
   
   There's other checks in this PR that does this but I just wanted to show one 
as an example. Extra checks should be placed to see if there really is a value 
placed in that config field. It's still important to check for nullness though.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to