Repository: beam Updated Branches: refs/heads/master 93020941a -> 6a95e5eec
TFRecordIO: switch to FileSystems from IOChannelUtils Project: http://git-wip-us.apache.org/repos/asf/beam/repo Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/7e3d5723 Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/7e3d5723 Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/7e3d5723 Branch: refs/heads/master Commit: 7e3d5723db6d1118df31e6af496b1aa2240607c9 Parents: 56b1280 Author: Dan Halperin <[email protected]> Authored: Wed May 3 19:34:25 2017 -0700 Committer: Dan Halperin <[email protected]> Committed: Thu May 4 09:32:45 2017 -0700 ---------------------------------------------------------------------- .../src/main/java/org/apache/beam/sdk/io/TFRecordIO.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/beam/blob/7e3d5723/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java index 3198829..6350fb5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java @@ -35,6 +35,7 @@ import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.Read.Bounded; +import org.apache.beam.sdk.io.fs.MatchResult; import org.apache.beam.sdk.io.fs.MatchResult.Metadata; import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.options.PipelineOptions; @@ -43,7 +44,6 @@ import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; @@ -158,12 +158,11 @@ public class TFRecordIO { if (getValidate()) { checkState(getFilepattern().isAccessible(), "Cannot validate with a RVP."); try { + MatchResult matches = FileSystems.match(getFilepattern().get()); checkState( - !IOChannelUtils.getFactory(getFilepattern().get()) - .match(getFilepattern().get()) - .isEmpty(), + !matches.metadata().isEmpty(), "Unable to find any files matching %s", - getFilepattern()); + getFilepattern().get()); } catch (IOException e) { throw new IllegalStateException( String.format("Failed to validate %s", getFilepattern().get()), e);
