You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by lc...@apache.org on 2017/04/27 16:39:07 UTC
[1/2] beam git commit: XmlIO and XmlSource now take an optional
validationEventHandler to throw a runtime exception when validations failed
(by example)
Repository: beam
Updated Branches:
refs/heads/master 634bf4e3a -> 0c26d024d
XmlIO and XmlSource now take an optional validationEventHandler to throw a runtime exception when validations failed (by example)
Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/26798fb5
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/26798fb5
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/26798fb5
Branch: refs/heads/master
Commit: 26798fb5a02795e6512f1414c1714c48bd64ae52
Parents: 634bf4e
Author: dgouyette <da...@gmail.com>
Authored: Thu Apr 27 14:33:37 2017 +0200
Committer: Luke Cwik <lc...@google.com>
Committed: Thu Apr 27 09:37:41 2017 -0700
----------------------------------------------------------------------
.../java/org/apache/beam/sdk/io/xml/XmlIO.java | 15 ++++
.../org/apache/beam/sdk/io/xml/XmlSource.java | 13 +--
.../apache/beam/sdk/io/xml/XmlSourceTest.java | 83 ++++++++++++++++++--
3 files changed, 93 insertions(+), 18 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/beam/blob/26798fb5/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlIO.java
----------------------------------------------------------------------
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 ce36abe..47715e9 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
@@ -27,6 +27,8 @@ import java.nio.charset.Charset;
import javax.annotation.Nullable;
import javax.xml.bind.JAXBContext;
import javax.xml.bind.JAXBException;
+import javax.xml.bind.ValidationEventHandler;
+
import org.apache.beam.sdk.io.BoundedSource;
import org.apache.beam.sdk.io.CompressedSource;
import org.apache.beam.sdk.io.FileBasedSink;
@@ -253,6 +255,9 @@ public class XmlIO {
abstract Builder<T> toBuilder();
+ @Nullable
+ abstract ValidationEventHandler getValidationEventHandler();
+
@AutoValue.Builder
abstract static class Builder<T> {
abstract Builder<T> setFileOrPatternSpec(String fileOrPatternSpec);
@@ -269,6 +274,8 @@ public class XmlIO {
abstract Builder<T> setCharset(String charset);
+ abstract Builder<T> setValidationEventHandler(ValidationEventHandler validationEventHandler);
+
abstract Read<T> build();
}
@@ -365,6 +372,14 @@ public class XmlIO {
return toBuilder().setCharset(charset.name()).build();
}
+ /**
+ * Sets the {@link ValidationEventHandler} to use with JAXB. Calling this with a {@code null}
+ * parameter will cause the JAXB unmarshaller event handler to be unspecified.
+ */
+ public Read<T> withValidationEventHandler(ValidationEventHandler validationEventHandler) {
+ return toBuilder().setValidationEventHandler(validationEventHandler).build();
+ }
+
@Override
public void validate(PBegin input) {
checkNotNull(
http://git-wip-us.apache.org/repos/asf/beam/blob/26798fb5/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlSource.java b/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlSource.java
index 1eb0e06..7aa42c5 100644
--- a/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlSource.java
+++ b/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlSource.java
@@ -33,8 +33,6 @@ import javax.xml.bind.JAXBContext;
import javax.xml.bind.JAXBElement;
import javax.xml.bind.JAXBException;
import javax.xml.bind.Unmarshaller;
-import javax.xml.bind.ValidationEvent;
-import javax.xml.bind.ValidationEventHandler;
import javax.xml.stream.FactoryConfigurationError;
import javax.xml.stream.XMLInputFactory;
import javax.xml.stream.XMLStreamConstants;
@@ -141,14 +139,9 @@ public class XmlSource<T> extends FileBasedSource<T> {
try {
JAXBContext jaxbContext = JAXBContext.newInstance(getCurrentSource().spec.getRecordClass());
jaxbUnmarshaller = jaxbContext.createUnmarshaller();
-
- // Throw errors if validation fails. JAXB by default ignores validation errors.
- jaxbUnmarshaller.setEventHandler(new ValidationEventHandler() {
- @Override
- public boolean handleEvent(ValidationEvent event) {
- throw new RuntimeException(event.getMessage(), event.getLinkedException());
- }
- });
+ if (getCurrentSource().spec.getValidationEventHandler() != null) {
+ jaxbUnmarshaller.setEventHandler(getCurrentSource().spec.getValidationEventHandler());
+ }
} catch (JAXBException e) {
throw new RuntimeException(e);
}
http://git-wip-us.apache.org/repos/asf/beam/blob/26798fb5/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/XmlSourceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/XmlSourceTest.java b/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/XmlSourceTest.java
index 3deee3e..abddcf9 100644
--- a/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/XmlSourceTest.java
+++ b/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/XmlSourceTest.java
@@ -21,9 +21,7 @@ import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionE
import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionFails;
import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionSucceedsAndConsistent;
import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
-import static org.hamcrest.Matchers.both;
import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.containsString;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
@@ -38,6 +36,8 @@ import java.nio.file.Files;
import java.util.ArrayList;
import java.util.List;
import java.util.Random;
+import javax.xml.bind.ValidationEvent;
+import javax.xml.bind.ValidationEventHandler;
import javax.xml.bind.annotation.XmlAttribute;
import javax.xml.bind.annotation.XmlRootElement;
import org.apache.beam.sdk.io.BoundedSource;
@@ -49,7 +49,6 @@ import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.transforms.display.DisplayData;
import org.apache.beam.sdk.values.PCollection;
-import org.hamcrest.Matchers;
import org.junit.Ignore;
import org.junit.Rule;
import org.junit.Test;
@@ -164,6 +163,29 @@ public class XmlSourceTest {
+ "</trains>";
@XmlRootElement
+ static class TinyTrain {
+ public TinyTrain(String name) {
+ this.name = name;
+ }
+
+ public TinyTrain() {
+ }
+
+ public String name = null;
+
+ @Override
+ public String toString() {
+ String str = "Train[";
+ if (name != null) {
+ str = str + "name=" + name;
+ }
+ str = str + "]";
+ return str;
+ }
+ }
+
+
+ @XmlRootElement
static class Train {
public static final int TRAIN_NUMBER_UNDEFINED = -1;
public String name = null;
@@ -277,10 +299,10 @@ public class XmlSourceTest {
return file;
}
- private List<Train> readEverythingFromReader(Reader<Train> reader) throws IOException {
- List<Train> results = new ArrayList<>();
+ private <T> List<T> readEverythingFromReader(Reader<T> reader) throws IOException {
+ List<T> results = new ArrayList<>();
for (boolean available = reader.start(); available; available = reader.advance()) {
- Train train = reader.getCurrent();
+ T train = reader.getCurrent();
results.add(train);
}
return results;
@@ -403,6 +425,14 @@ public class XmlSourceTest {
return strings;
}
+ List<String> tinyTrainsToStrings(List<TinyTrain> input) {
+ List<String> strings = new ArrayList<>();
+ for (Object data : input) {
+ strings.add(data.toString());
+ }
+ return strings;
+ }
+
@Test
public void testReadXMLSmall() throws IOException {
File file = tempFolder.newFile("trainXMLSmall");
@@ -522,22 +552,30 @@ public class XmlSourceTest {
}
@Test
- public void testReadXMLInvalidRecordClass() throws IOException {
+ public void testReadXMLInvalidRecordClassWithCustomEventHandler() throws IOException {
File file = tempFolder.newFile("trainXMLSmall");
Files.write(file.toPath(), trainXML.getBytes(StandardCharsets.UTF_8));
+ ValidationEventHandler validationEventHandler = new ValidationEventHandler() {
+ @Override
+ public boolean handleEvent(ValidationEvent event) {
+ throw new RuntimeException("MyCustomValidationEventHandler failure mesage");
+ }
+ };
+
BoundedSource<WrongTrainType> source =
XmlIO.<WrongTrainType>read()
.from(file.toPath().toString())
.withRootElement("trains")
.withRecordElement("train")
.withRecordClass(WrongTrainType.class)
+ .withValidationEventHandler(validationEventHandler)
.createSource();
exception.expect(RuntimeException.class);
// JAXB internationalizes the error message. So this is all we can match for.
- exception.expectMessage(both(containsString("name")).and(Matchers.containsString("something")));
+ exception.expectMessage("MyCustomValidationEventHandler failure mesage");
try (Reader<WrongTrainType> reader = source.createReader(null)) {
List<WrongTrainType> results = new ArrayList<>();
@@ -549,6 +587,35 @@ public class XmlSourceTest {
}
@Test
+ public void testReadXmlWithAdditionalFieldsShouldNotThrowException() throws IOException{
+ File file = tempFolder.newFile("trainXMLSmall");
+ Files.write(file.toPath(), trainXML.getBytes(StandardCharsets.UTF_8));
+
+ BoundedSource<TinyTrain> source =
+ XmlIO.<TinyTrain>read()
+ .from(file.toPath().toString())
+ .withRootElement("trains")
+ .withRecordElement("train")
+ .withRecordClass(TinyTrain.class)
+ .createSource();
+
+ List<TinyTrain> expectedResults =
+ ImmutableList.of(
+ new TinyTrain("Thomas"),
+ new TinyTrain("Henry"),
+ new TinyTrain("Toby"),
+ new TinyTrain("Gordon"),
+ new TinyTrain("Emily"),
+ new TinyTrain("Percy")
+ );
+
+ assertThat(
+ tinyTrainsToStrings(expectedResults),
+ containsInAnyOrder(
+ tinyTrainsToStrings(readEverythingFromReader(source.createReader(null))).toArray()));
+ }
+
+ @Test
public void testReadXMLNoBundleSize() throws IOException {
File file = tempFolder.newFile("trainXMLSmall");
Files.write(file.toPath(), trainXML.getBytes(StandardCharsets.UTF_8));
[2/2] beam git commit: [BEAM-2062] XmlIO and XmlSource now take an
optional validationEventHandler
Posted by lc...@apache.org.
[BEAM-2062] XmlIO and XmlSource now take an optional validationEventHandler
This closes #2736
Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/0c26d024
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/0c26d024
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/0c26d024
Branch: refs/heads/master
Commit: 0c26d024dc210b706343fbc5f17753fc5809466b
Parents: 634bf4e 26798fb
Author: Luke Cwik <lc...@google.com>
Authored: Thu Apr 27 09:38:57 2017 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Thu Apr 27 09:38:57 2017 -0700
----------------------------------------------------------------------
.../java/org/apache/beam/sdk/io/xml/XmlIO.java | 15 ++++
.../org/apache/beam/sdk/io/xml/XmlSource.java | 13 +--
.../apache/beam/sdk/io/xml/XmlSourceTest.java | 83 ++++++++++++++++++--
3 files changed, 93 insertions(+), 18 deletions(-)
----------------------------------------------------------------------