From fb4f259a3ce69ee24370d3366e88742e8a26d047 Mon Sep 17 00:00:00 2001 From: Vikas Kedigehalli Date: Fri, 28 Apr 2017 12:12:43 -0700 Subject: [PATCH 1/2] Remove IOChannelUtils from a few ITs --- .../apache/beam/examples/WindowedWordCountIT.java | 13 +++++++------ .../java/org/apache/beam/examples/WordCountIT.java | 14 +++++++------- 2 files changed, 14 insertions(+), 13 deletions(-) diff --git a/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java b/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java index a53a1513f7e5..b5eddb5d17e7 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java +++ b/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java @@ -34,6 +34,8 @@ import org.apache.beam.examples.common.ExampleUtils; import org.apache.beam.examples.common.WriteOneFilePerWindow.PerWindowFiles; import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.StreamingOptions; import org.apache.beam.sdk.testing.FileChecksumMatcher; @@ -44,7 +46,6 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.util.ExplicitShardedFile; import org.apache.beam.sdk.util.FluentBackoff; -import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.NumberedShardedFile; import org.apache.beam.sdk.util.ShardedFile; import org.hamcrest.Description; @@ -107,13 +108,13 @@ private WindowedWordCountITOptions defaultOptions() throws Exception { options.setWindowSize(10); options.setOutput( - IOChannelUtils.resolve( - options.getTempRoot(), - String.format( + FileSystems.matchNewResource(options.getTempRoot(), true) + .resolve(String.format( "WindowedWordCountIT.%s-%tFT% Date: Fri, 28 Apr 2017 12:14:24 -0700 Subject: [PATCH 2/2] Use getFileName in NumberedShardedFile --- .../java/org/apache/beam/sdk/util/NumberedShardedFile.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java index fa22586deff0..e90438242409 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java @@ -195,8 +195,7 @@ List readLines(Collection files) throws IOException { @VisibleForTesting boolean checkTotalNumOfFiles(Collection files) { for (Metadata fileMedadata : files) { - String fileName = fileMedadata.resourceId().toString().substring( - fileMedadata.resourceId().getCurrentDirectory().toString().length()); + String fileName = fileMedadata.resourceId().getFilename(); if (fileName == null) { // this path has zero elements