You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by ar...@apache.org on 2019/03/18 13:31:26 UTC
[beam] branch master updated: [BEAM-5115] Make ValueProvider API
consistent between XmlIO and XmlSource
This is an automated email from the ASF dual-hosted git repository.
aromanenko 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 7ac0790 [BEAM-5115] Make ValueProvider API consistent between XmlIO and XmlSource
new 3d6315b Merge pull request #8068: [BEAM-5115] Make ValueProvider API consistent between XmlIO and XmlSource
7ac0790 is described below
commit 7ac0790b1d592cbd9bbefe62f8c7048ff8084fe3
Author: Ismaël Mejía <ie...@gmail.com>
AuthorDate: Fri Mar 15 14:23:43 2019 +0100
[BEAM-5115] Make ValueProvider API consistent between XmlIO and XmlSource
---
.../src/main/java/org/apache/beam/sdk/io/xml/XmlIO.java | 17 ++++++++++++-----
.../test/java/org/apache/beam/sdk/io/xml/XmlIOTest.java | 4 +++-
2 files changed, 15 insertions(+), 6 deletions(-)
diff --git a/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlIO.java b/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlIO.java
index 5b2dbc7..17fdcf4 100644
--- a/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlIO.java
+++ b/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlIO.java
@@ -41,6 +41,7 @@ import org.apache.beam.sdk.io.FileSystems;
import org.apache.beam.sdk.io.OffsetBasedSource;
import org.apache.beam.sdk.io.ReadAllViaFileBasedSource;
import org.apache.beam.sdk.io.fs.ResourceId;
+import org.apache.beam.sdk.options.ValueProvider;
import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.SerializableFunction;
@@ -232,7 +233,7 @@ public class XmlIO {
abstract MappingConfiguration<T> getConfiguration();
@Nullable
- abstract String getFileOrPatternSpec();
+ abstract ValueProvider<String> getFileOrPatternSpec();
abstract Compression getCompression();
@@ -244,7 +245,7 @@ public class XmlIO {
abstract static class Builder<T> {
abstract Builder<T> setConfiguration(MappingConfiguration<T> configuration);
- abstract Builder<T> setFileOrPatternSpec(String fileOrPatternSpec);
+ abstract Builder<T> setFileOrPatternSpec(ValueProvider<String> fileOrPatternSpec);
abstract Builder<T> setCompression(Compression compression);
@@ -291,6 +292,14 @@ public class XmlIO {
* file should be of the form defined in {@link #read}.
*/
public Read<T> from(String fileOrPatternSpec) {
+ return from(StaticValueProvider.of(fileOrPatternSpec));
+ }
+
+ /**
+ * Reads a single XML file or a set of XML files defined by a Java "glob" file pattern. Each XML
+ * file should be of the form defined in {@link #read}. Using ValueProviders.
+ */
+ public Read<T> from(ValueProvider<String> fileOrPatternSpec) {
return toBuilder().setFileOrPatternSpec(fileOrPatternSpec).build();
}
@@ -371,9 +380,7 @@ public class XmlIO {
@VisibleForTesting
BoundedSource<T> createSource() {
- return CompressedSource.from(
- new XmlSource<>(
- StaticValueProvider.of(getFileOrPatternSpec()), getConfiguration(), 1L))
+ return CompressedSource.from(new XmlSource<>(getFileOrPatternSpec(), getConfiguration(), 1L))
.withCompression(getCompression());
}
diff --git a/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/XmlIOTest.java b/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/XmlIOTest.java
index 102b38a..ba778a2 100644
--- a/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/XmlIOTest.java
+++ b/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/XmlIOTest.java
@@ -128,7 +128,9 @@ public class XmlIOTest {
PCollection<Bird> readBack =
readPipeline.apply(
XmlIO.<Bird>read()
- .from(new File(tmpFolder.getRoot(), "birds").getAbsolutePath() + "*")
+ .from(
+ readPipeline.newProvider(
+ new File(tmpFolder.getRoot(), "birds").getAbsolutePath() + "*"))
.withRecordClass(Bird.class)
.withRootElement("birds")
.withRecordElement("bird")