Skip to content

Commit

Permalink
Add linkUrl to TextIO DisplayData
Browse files Browse the repository at this point in the history
  • Loading branch information
swegner committed May 4, 2016
1 parent f4c7c1b commit cd77d1b
Show file tree
Hide file tree
Showing 2 changed files with 52 additions and 7 deletions.
30 changes: 27 additions & 3 deletions sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import org.apache.beam.sdk.runners.DirectPipelineRunner;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.display.DisplayData;
import org.apache.beam.sdk.util.IOChannelFactory;
import org.apache.beam.sdk.util.IOChannelUtils;
import org.apache.beam.sdk.util.MimeTypes;
import org.apache.beam.sdk.values.PCollection;
Expand Down Expand Up @@ -343,10 +344,14 @@ public PCollection<T> apply(PInput input) {
public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);

if (filepattern != null) {
builder.add(DisplayData.item("filePattern", filepattern)
.withLinkUrl(getBrowseUrl(filepattern)));
}

builder
.add(DisplayData.item("compressionType", compressionType.toString()))
.addIfNotDefault(DisplayData.item("validation", validate), true)
.addIfNotNull(DisplayData.item("filePattern", filepattern));
.addIfNotDefault(DisplayData.item("validation", validate), true);
}

@Override
Expand Down Expand Up @@ -648,8 +653,15 @@ public PDone apply(PCollection<T> input) {
public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);

if (filenamePrefix != null) {
// Append wildcard to browseUrl input since this is a filename prefix
String browseUrl = getBrowseUrl(filenamePrefix + "*");

builder.add(DisplayData.item("filePrefix", filenamePrefix)
.withLinkUrl(browseUrl));
}

builder
.addIfNotNull(DisplayData.item("filePrefix", filenamePrefix))
.addIfNotDefault(DisplayData.item("fileSuffix", filenameSuffix), "")
.addIfNotDefault(
DisplayData.item("shardNameTemplate", shardTemplate),
Expand Down Expand Up @@ -744,6 +756,18 @@ private static void validateOutputComponent(String partialFilePattern) {
+ partialFilePattern);
}

private static String getBrowseUrl(String filePattern) {
IOChannelFactory factory;
try {
factory = IOChannelUtils.getFactory(filePattern);
} catch (IOException e) {
throw new IllegalStateException(
String.format("Invalid filePattern: %s", filePattern), e);
}

return factory.getBrowseUrl(filePattern);
}

//////////////////////////////////////////////////////////////////////////////

/** Disable construction of utility class. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,8 +22,12 @@
import static org.apache.beam.sdk.TestUtils.NO_INTS_ARRAY;
import static org.apache.beam.sdk.TestUtils.NO_LINES_ARRAY;
import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasKey;
import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasLinkUrl;
import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasValue;

import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.hamcrest.core.AllOf.allOf;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThat;
Expand All @@ -36,6 +40,7 @@
import org.apache.beam.sdk.coders.VoidCoder;
import org.apache.beam.sdk.io.TextIO.CompressionType;
import org.apache.beam.sdk.io.TextIO.TextSource;
import org.apache.beam.sdk.options.GcsOptions;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.SourceTestUtils;
Expand All @@ -45,11 +50,13 @@
import org.apache.beam.sdk.transforms.display.DisplayData;
import org.apache.beam.sdk.util.CoderUtils;
import org.apache.beam.sdk.util.IOChannelUtils;
import org.apache.beam.sdk.util.gcsfs.GcsPath;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PDone;

import com.google.common.collect.ImmutableList;

import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
Expand Down Expand Up @@ -79,6 +86,11 @@ public class TextIOTest {
@Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
@Rule public ExpectedException expectedException = ExpectedException.none();

@BeforeClass
public static void setUpClass() {
IOChannelUtils.registerStandardIOFactories(PipelineOptionsFactory.as(GcsOptions.class));
}

<T> void runTestRead(T[] expected, Coder<T> coder) throws Exception {
File tmpFile = tmpFolder.newFile("file.txt");
String filename = tmpFile.getPath();
Expand Down Expand Up @@ -159,14 +171,19 @@ public void testReadNamed() throws Exception {

@Test
public void testReadDisplayData() {
String filePattern = "gs://bucket/foo/bar*";

TextIO.Read.Bound<?> read = TextIO.Read
.from("foo.*")
.from(filePattern)
.withCompressionType(CompressionType.BZIP2)
.withoutValidation();

DisplayData displayData = DisplayData.from(read);

assertThat(displayData, hasDisplayItem("filePattern", "foo.*"));
assertThat(displayData, hasDisplayItem(allOf(
hasKey("filePattern"),
hasValue(filePattern),
hasLinkUrl(GcsPath.fromUri(filePattern).getBrowseUrl()))));
assertThat(displayData, hasDisplayItem("compressionType", CompressionType.BZIP2.toString()));
assertThat(displayData, hasDisplayItem("validation", false));
}
Expand Down Expand Up @@ -292,16 +309,20 @@ public void testShardedWrite() throws Exception {

@Test
public void testWriteDisplayData() {
String filePrefix = "gs://bucket/foo/thing";
TextIO.Write.Bound<?> write = TextIO.Write
.to("foo")
.to(filePrefix)
.withSuffix("bar")
.withShardNameTemplate("-SS-of-NN-")
.withNumShards(100)
.withoutValidation();

DisplayData displayData = DisplayData.from(write);

assertThat(displayData, hasDisplayItem("filePrefix", "foo"));
assertThat(displayData, hasDisplayItem(allOf(
hasKey("filePrefix"),
hasValue(filePrefix),
hasLinkUrl(GcsPath.fromUri(filePrefix + "*").getBrowseUrl()))));
assertThat(displayData, hasDisplayItem("fileSuffix", "bar"));
assertThat(displayData, hasDisplayItem("shardNameTemplate", "-SS-of-NN-"));
assertThat(displayData, hasDisplayItem("numShards", 100));
Expand Down

0 comments on commit cd77d1b

Please sign in to comment.