diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/AvroIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/AvroIO.java index f2c094d759..0ef2e07d85 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/AvroIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/AvroIO.java @@ -329,8 +329,10 @@ public PCollection apply(PInput input) { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .addIfNotNull("filePattern", filepattern) - .addIfNotDefault("validation", validate, true); + .addIfNotNull(DisplayData.item("filePattern", filepattern) + .withLabel("Input File Pattern")) + .addIfNotDefault(DisplayData.item("validation", validate) + .withLabel("Validation Enabled"), true); } @Override @@ -692,12 +694,22 @@ public PDone apply(PCollection input) { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .add("schema", type) - .addIfNotNull("filePrefix", filenamePrefix) - .addIfNotDefault("shardNameTemplate", shardTemplate, DEFAULT_SHARD_TEMPLATE) - .addIfNotDefault("fileSuffix", filenameSuffix, "") - .addIfNotDefault("numShards", numShards, 0) - .addIfNotDefault("validation", validate, true); + .add(DisplayData.item("schema", type) + .withLabel("Record Schema")) + .addIfNotNull(DisplayData.item("filePrefix", filenamePrefix) + .withLabel("Output File Prefix")) + .addIfNotDefault(DisplayData.item("shardNameTemplate", shardTemplate) + .withLabel("Output Shard Name Template"), + DEFAULT_SHARD_TEMPLATE) + .addIfNotDefault(DisplayData.item("fileSuffix", filenameSuffix) + .withLabel("Output File Suffix"), + "") + .addIfNotDefault(DisplayData.item("numShards", numShards) + .withLabel("Maximum Output Shards"), + 0) + .addIfNotDefault(DisplayData.item("validation", validate) + .withLabel("Validation Enabled"), + true); } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java index dbe5f493e3..d39bc874a8 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BigQueryIO.java @@ -629,13 +629,18 @@ public void populateDisplayData(DisplayData.Builder builder) { TableReference table = getTable(); if (table != null) { - builder.add("table", toTableSpec(table)); + builder.add(DisplayData.item("table", toTableSpec(table)) + .withLabel("Table")); } builder - .addIfNotNull("query", query) - .addIfNotNull("flattenResults", flattenResults) - .addIfNotDefault("validation", validate, true); + .addIfNotNull(DisplayData.item("query", query) + .withLabel("Query")) + .addIfNotNull(DisplayData.item("flattenResults", flattenResults) + .withLabel("Flatten Query Results")) + .addIfNotDefault(DisplayData.item("validation", validate) + .withLabel("Validation Enabled"), + true); } /** @@ -800,6 +805,12 @@ public synchronized long getEstimatedSizeBytes(PipelineOptions options) throws E protected void cleanupTempResource(BigQueryOptions bqOptions) throws Exception { // Do nothing. } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder.add(DisplayData.item("table", jsonTable)); + } } /** @@ -891,6 +902,11 @@ protected void cleanupTempResource(BigQueryOptions bqOptions) throws Exception { tableService.deleteDataset(tableToRemove.getProjectId(), tableToRemove.getDatasetId()); } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder.add(DisplayData.item("query", query)); + } private synchronized JobStatistics dryRunQueryIfNeeded(BigQueryOptions bqOptions) throws InterruptedException, IOException { if (dryRunJobStats.get() == null) { @@ -1740,17 +1756,23 @@ public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .addIfNotNull("table", jsonTableRef) - .addIfNotNull("schema", jsonSchema); + .addIfNotNull(DisplayData.item("table", jsonTableRef) + .withLabel("Table Reference")) + .addIfNotNull(DisplayData.item("schema", jsonSchema) + .withLabel("Table Schema")); if (tableRefFunction != null) { - builder.add("tableFn", tableRefFunction.getClass()); + builder.add(DisplayData.item("tableFn", tableRefFunction.getClass()) + .withLabel("Table Reference Function")); } builder - .add("createDisposition", createDisposition.toString()) - .add("writeDisposition", writeDisposition.toString()) - .addIfNotDefault("validation", validate, true); + .add(DisplayData.item("createDisposition", createDisposition.toString()) + .withLabel("Table CreateDisposition")) + .add(DisplayData.item("writeDisposition", writeDisposition.toString()) + .withLabel("Table WriteDisposition")) + .addIfNotDefault(DisplayData.item("validation", validate) + .withLabel("Validation Enabled"), true); } /** Returns the create disposition. */ @@ -1837,6 +1859,17 @@ public FileBasedSink.FileBasedWriteOperation createWriteOperation( return new BigQueryWriteOperation(this); } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + + builder + .addIfNotNull(DisplayData.item("schema", jsonSchema) + .withLabel("Table Schema")) + .addIfNotNull(DisplayData.item("tableSpec", jsonTable) + .withLabel("Table Specification")); + } + private static class BigQueryWriteOperation extends FileBasedWriteOperation { // The maximum number of retry load jobs. private static final int MAX_RETRY_LOAD_JOBS = 3; @@ -2070,6 +2103,14 @@ public void finishBundle(Context context) throws Exception { uniqueIdsForTableRows.clear(); } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + + builder.addIfNotNull(DisplayData.item("schema", jsonTableSchema) + .withLabel("Table Schema")); + } + public TableReference getOrCreateTable(BigQueryOptions options, String tableSpec) throws IOException { TableReference tableReference = parseTableSpec(tableSpec); @@ -2277,6 +2318,17 @@ public void processElement(ProcessContext context) throws IOException { new TableRowInfo(context.element(), uniqueId))); } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + + builder.addIfNotNull(DisplayData.item("table", tableSpec)); + if (tableRefFunction != null) { + builder.add(DisplayData.item("tableFn", tableRefFunction.getClass()) + .withLabel("Table Reference Function")); + } + } + private String tableSpecFromWindow(BigQueryOptions options, BoundedWindow window) { if (tableSpec != null) { return tableSpec; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BoundedReadFromUnboundedSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BoundedReadFromUnboundedSource.java index 84e7d82639..8388f1a39a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BoundedReadFromUnboundedSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BoundedReadFromUnboundedSource.java @@ -110,9 +110,12 @@ public String getKindString() { public void populateDisplayData(DisplayData.Builder builder) { // We explicitly do not register base-class data, instead we use the delegate inner source. builder - .add("source", source.getClass()) - .addIfNotDefault("maxRecords", maxNumRecords, Long.MAX_VALUE) - .addIfNotNull("maxReadTime", maxReadTime) + .add(DisplayData.item("source", source.getClass()) + .withLabel("Read Source")) + .addIfNotDefault(DisplayData.item("maxRecords", maxNumRecords) + .withLabel("Maximum Read Records"), Long.MAX_VALUE) + .addIfNotNull(DisplayData.item("maxReadTime", maxReadTime) + .withLabel("Maximum Read Time")) .include(source); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/CompressedSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/CompressedSource.java index 05f33d6ba1..be57eed11b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/CompressedSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/CompressedSource.java @@ -324,14 +324,17 @@ public void populateDisplayData(DisplayData.Builder builder) { // We explicitly do not register base-class data, instead we use the delegate inner source. builder .include(sourceDelegate) - .add("source", sourceDelegate.getClass()); + .add(DisplayData.item("source", sourceDelegate.getClass()) + .withLabel("Read Source")); if (channelFactory instanceof Enum) { // GZIP and BZIP are implemented as enums; Enum classes are anonymous, so use the .name() // value instead - builder.add("compressionMode", ((Enum) channelFactory).name()); + builder.add(DisplayData.item("compressionMode", ((Enum) channelFactory).name()) + .withLabel("Compression Mode")); } else { - builder.add("compressionMode", channelFactory.getClass()); + builder.add(DisplayData.item("compressionMode", channelFactory.getClass()) + .withLabel("Compression Mode")); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/CountingInput.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/CountingInput.java index 85c3300401..91e3f13113 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/CountingInput.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/CountingInput.java @@ -113,7 +113,8 @@ public PCollection apply(PBegin begin) { @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - builder.add("upTo", numElements); + builder.add(DisplayData.item("upTo", numElements) + .withLabel("Count Up To")); } } @@ -199,14 +200,17 @@ public PCollection apply(PBegin begin) { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - builder.add("timestampFn", timestampFn.getClass()); + builder.add(DisplayData.item("timestampFn", timestampFn.getClass()) + .withLabel("Timestamp Function")); if (maxReadTime.isPresent()) { - builder.add("maxReadTime", maxReadTime.get()); + builder.add(DisplayData.item("maxReadTime", maxReadTime.get()) + .withLabel("Maximum Read Time")); } if (maxNumRecords.isPresent()) { - builder.add("maxRecords", maxNumRecords.get()); + builder.add(DisplayData.item("maxRecords", maxNumRecords.get()) + .withLabel("Maximum Read Records")); } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java index 482cdb57ba..9bd54944cb 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java @@ -401,12 +401,16 @@ public long getEstimatedSizeBytes(PipelineOptions options) throws Exception { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .addIfNotDefault("host", host, DEFAULT_HOST) - .addIfNotNull("dataset", datasetId) - .addIfNotNull("namespace", namespace); + .addIfNotDefault(DisplayData.item("host", host) + .withLabel("Datastore Service"), DEFAULT_HOST) + .addIfNotNull(DisplayData.item("dataset", datasetId) + .withLabel("Input Dataset")) + .addIfNotNull(DisplayData.item("namespace", namespace) + .withLabel("App Engine Namespace")); if (query != null) { - builder.add("query", query.toString()); + builder.add(DisplayData.item("query", query.toString()) + .withLabel("Query")); } } @@ -614,9 +618,12 @@ public DatastoreWriteOperation createWriteOperation(PipelineOptions options) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder - .addIfNotDefault("host", host, DEFAULT_HOST) - .addIfNotNull("dataset", datasetId); + .addIfNotDefault(DisplayData.item("host", host) + .withLabel("Datastore Service"), DEFAULT_HOST) + .addIfNotNull(DisplayData.item("dataset", datasetId) + .withLabel("Output Dataset")); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSink.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSink.java index e9d280d707..166ade611d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSink.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSink.java @@ -127,7 +127,8 @@ public void populateDisplayData(DisplayData.Builder builder) { String fileNamePattern = String.format("%s%s%s", baseOutputFilename, fileNamingTemplate, getFileExtension(extension)); - builder.add("fileNamePattern", fileNamePattern); + builder.add(DisplayData.item("fileNamePattern", fileNamePattern) + .withLabel("File Name Pattern")); } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSource.java index 468a276b43..35cd40d7c8 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSource.java @@ -261,7 +261,8 @@ private static long getEstimatedSizeOfFilesBySampling( @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - builder.add("filePattern", getFileOrPatternSpec()); + builder.add(DisplayData.item("filePattern", getFileOrPatternSpec()) + .withLabel("File Pattern")); } private ListenableFuture>> createFutureForFileSplit( diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/OffsetBasedSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/OffsetBasedSource.java index 04b61b8771..d9f3ffcd92 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/OffsetBasedSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/OffsetBasedSource.java @@ -203,9 +203,12 @@ public boolean allowsDynamicSplitting() { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .add("minBundleSize", minBundleSize) - .addIfNotDefault("startOffset", startOffset, 0) - .addIfNotDefault("endOffset", endOffset, Long.MAX_VALUE); + .addIfNotDefault(DisplayData.item("minBundleSize", minBundleSize) + .withLabel("Minimum Bundle Size"), 1L) + .addIfNotDefault(DisplayData.item("startOffset", startOffset) + .withLabel("Start Read Offset"), 0L) + .addIfNotDefault(DisplayData.item("endOffset", endOffset) + .withLabel("End Read Offset"), Long.MAX_VALUE); } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java index b286125600..c75e501c97 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java @@ -183,6 +183,23 @@ protected static Instant assignMessageTimestamp( return new Instant(millisSinceEpoch); } + /** + * Populate common {@link DisplayData} between Pubsub source and sink. + */ + private static void populateCommonDisplayData(DisplayData.Builder builder, + String timestampLabel, String idLabel, PubsubTopic topic) { + builder + .addIfNotNull(DisplayData.item("timestampLabel", timestampLabel) + .withLabel("Timestamp Label Attribute")) + .addIfNotNull(DisplayData.item("idLabel", idLabel) + .withLabel("ID Label Attribute")); + + if (topic != null) { + builder.add(DisplayData.item("topic", topic.asPath()) + .withLabel("Pubsub Topic")); + } + } + /** * Class representing a Cloud Pub/Sub Subscription. */ @@ -691,19 +708,17 @@ public PCollection apply(PInput input) { @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); + populateCommonDisplayData(builder, timestampLabel, idLabel, topic); builder - .addIfNotNull("timestampLabel", timestampLabel) - .addIfNotNull("idLabel", idLabel) - .addIfNotNull("maxReadTime", maxReadTime) - .addIfNotDefault("maxNumRecords", maxNumRecords, 0); - - if (topic != null) { - builder.add("topic", topic.asPath()); - } + .addIfNotNull(DisplayData.item("maxReadTime", maxReadTime) + .withLabel("Maximum Read Time")) + .addIfNotDefault(DisplayData.item("maxNumRecords", maxNumRecords) + .withLabel("Maximum Read Records"), 0); if (subscription != null) { - builder.add("subscription", subscription.asPath()); + builder.add(DisplayData.item("subscription", subscription.asPath()) + .withLabel("Pubsub Subscription")); } } @@ -995,14 +1010,7 @@ public PDone apply(PCollection input) { @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - - builder - .addIfNotNull("timestampLabel", timestampLabel) - .addIfNotNull("idLabel", idLabel); - - if (topic != null) { - builder.add("topic", topic.asPath()); - } + populateCommonDisplayData(builder, timestampLabel, idLabel, topic); } @Override @@ -1072,6 +1080,11 @@ private void publish() throws IOException { .execute(); output.clear(); } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + Bound.this.populateDisplayData(builder); + } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Read.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Read.java index 803423b474..ef77a6371f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Read.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Read.java @@ -146,8 +146,10 @@ public String getKindString() { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder - .add("source", source.getClass()) + .add(DisplayData.item("source", source.getClass()) + .withLabel("Read Source")) .include(source); } @@ -260,8 +262,10 @@ public String getKindString() { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder - .add("source", source.getClass()) + .add(DisplayData.item("source", source.getClass()) + .withLabel("Read Source")) .include(source); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Sink.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Sink.java index 62bfbc1ad2..c7bb68beb1 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Sink.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Sink.java @@ -136,7 +136,7 @@ public abstract class Sink implements Serializable, HasDisplayData { * {@inheritDoc} * *

By default, does not register any display data. Implementors may override this method - * to provide their own display metadata. + * to provide their own display data. */ @Override public void populateDisplayData(DisplayData.Builder builder) {} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Source.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Source.java index 05a86ca4a5..81d5dfdebc 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Source.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Source.java @@ -71,7 +71,7 @@ public abstract class Source implements Serializable, HasDisplayData { * {@inheritDoc} * *

By default, does not register any display data. Implementors may override this method - * to provide their own display metadata. + * to provide their own display data. */ @Override public void populateDisplayData(DisplayData.Builder builder) {} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java index 2509f6f518..5089993437 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/TextIO.java @@ -344,9 +344,12 @@ public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .add("compressionType", compressionType.toString()) - .addIfNotDefault("validation", validate, true) - .addIfNotNull("filePattern", filepattern); + .add(DisplayData.item("compressionType", compressionType.toString()) + .withLabel("Compression Type")) + .addIfNotDefault(DisplayData.item("validation", validate) + .withLabel("Validation Enabled"), true) + .addIfNotNull(DisplayData.item("filePattern", filepattern) + .withLabel("File Pattern")); } @Override @@ -649,11 +652,17 @@ public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .addIfNotNull("filePrefix", filenamePrefix) - .addIfNotDefault("fileSuffix", filenameSuffix, "") - .addIfNotDefault("shardNameTemplate", shardTemplate, DEFAULT_SHARD_TEMPLATE) - .addIfNotDefault("validation", validate, true) - .addIfNotDefault("numShards", numShards, 0); + .addIfNotNull(DisplayData.item("filePrefix", filenamePrefix) + .withLabel("Output File Prefix")) + .addIfNotDefault(DisplayData.item("fileSuffix", filenameSuffix) + .withLabel("Output Fix Suffix"), "") + .addIfNotDefault(DisplayData.item("shardNameTemplate", shardTemplate) + .withLabel("Output Shard Name Template"), + DEFAULT_SHARD_TEMPLATE) + .addIfNotDefault(DisplayData.item("validation", validate) + .withLabel("Validation Enabled"), true) + .addIfNotDefault(DisplayData.item("numShards", numShards) + .withLabel("Maximum Output Shards"), 0); } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Write.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Write.java index ccea32ec14..62d0cf639a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Write.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Write.java @@ -81,8 +81,10 @@ public PDone apply(PCollection input) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder - .add("sink", sink.getClass()) + .add(DisplayData.item("sink", sink.getClass()) + .withLabel("Write Sink")) .include(sink); } @@ -200,6 +202,11 @@ public void finishBundle(Context c) throws Exception { c.output(result); } } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + Write.Bound.this.populateDisplayData(builder); + } }).withSideInputs(writeOperationView)) .setCoder(writeOperation.getWriterResultCoder()); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/XmlSink.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/XmlSink.java index 90d3bfbcdd..75326aeeea 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/XmlSink.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/XmlSink.java @@ -225,8 +225,10 @@ public XmlWriteOperation createWriteOperation(PipelineOptions options) { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .addIfNotNull("rootElement", rootElementName) - .addIfNotNull("recordClass", classToBind); + .addIfNotNull(DisplayData.item("rootElement", rootElementName) + .withLabel("XML Root Element")) + .addIfNotNull(DisplayData.item("recordClass", classToBind) + .withLabel("XML Record Class")); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/XmlSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/XmlSource.java index c8781f6e1d..aaac6a83a2 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/XmlSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/XmlSource.java @@ -218,9 +218,12 @@ public void validate() { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .addIfNotNull("rootElement", rootElement) - .addIfNotNull("recordElement", recordElement) - .addIfNotNull("recordClass", recordClass); + .addIfNotNull(DisplayData.item("rootElement", rootElement) + .withLabel("XML Root Element")) + .addIfNotNull(DisplayData.item("recordElement", recordElement) + .withLabel("XML Record Element")) + .addIfNotNull(DisplayData.item("recordClass", recordClass) + .withLabel("XML Record Class")); } @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/bigtable/BigtableIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/bigtable/BigtableIO.java index 9e452ba7a4..bf66c6c52a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/bigtable/BigtableIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/bigtable/BigtableIO.java @@ -264,14 +264,17 @@ public void validate(PBegin input) { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - builder.add("tableId", tableId); + builder.add(DisplayData.item("tableId", tableId) + .withLinkUrl("Table ID")); if (options != null) { - builder.add("bigtableOptions", options.toString()); + builder.add(DisplayData.item("bigtableOptions", options.toString()) + .withLabel("Bigtable Options")); } if (filter != null) { - builder.add("rowFilter", filter.toString()); + builder.add(DisplayData.item("rowFilter", filter.toString()) + .withLabel("Table Row Filter")); } } @@ -446,10 +449,12 @@ Write withBigtableService(BigtableService bigtableService) { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - builder.add("tableId", tableId); + builder.add(DisplayData.item("tableId", tableId) + .withLabel("Table ID")); if (options != null) { - builder.add("bigtableOptions", options.toString()); + builder.add(DisplayData.item("bigtableOptions", options.toString()) + .withLabel("Bigtable Options")); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionSpec.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionSpec.java index 54e9fc7a1b..06fec4e453 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionSpec.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionSpec.java @@ -15,73 +15,40 @@ */ package com.google.cloud.dataflow.sdk.options; -import com.google.common.base.MoreObjects; -import com.google.common.base.Objects; +import com.google.auto.value.AutoValue; + +import com.fasterxml.jackson.annotation.JsonIgnore; import java.lang.reflect.Method; /** * For internal use. Specification for an option defined in a {@link PipelineOptions} interface. */ -class PipelineOptionSpec { - private final Class clazz; - private final String name; - private final Method getter; - +@AutoValue +abstract class PipelineOptionSpec { static PipelineOptionSpec of(Class clazz, String name, Method getter) { - return new PipelineOptionSpec(clazz, name, getter); - } - - private PipelineOptionSpec(Class clazz, String name, Method getter) { - this.clazz = clazz; - this.name = name; - this.getter = getter; + return new AutoValue_PipelineOptionSpec(clazz, name, getter); } /** * The {@link PipelineOptions} interface which defines this {@link PipelineOptionSpec}. */ - Class getDefiningInterface() { - return clazz; - } + abstract Class getDefiningInterface(); /** * Name of the property. */ - String getName() { - return name; - } + abstract String getName(); /** * The getter method for this property. */ - Method getGetterMethod() { - return getter; - } + abstract Method getGetterMethod(); - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("definingInterface", getDefiningInterface()) - .add("name", getName()) - .add("getterMethod", getGetterMethod()) - .toString(); - } - - @Override - public int hashCode() { - return Objects.hashCode(getDefiningInterface(), getName(), getGetterMethod()); - } - - @Override - public boolean equals(Object obj) { - if (!(obj instanceof PipelineOptionSpec)) { - return false; - } - - PipelineOptionSpec that = (PipelineOptionSpec) obj; - return Objects.equal(this.getDefiningInterface(), that.getDefiningInterface()) - && Objects.equal(this.getName(), that.getName()) - && Objects.equal(this.getGetterMethod(), that.getGetterMethod()); + /** + * Whether the option should be serialized. Uses the {@link JsonIgnore} annotation. + */ + boolean shouldSerialize() { + return !getGetterMethod().isAnnotationPresent(JsonIgnore.class); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java index d6ee0d24b2..170db0b1f7 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java @@ -317,13 +317,19 @@ private void populateDisplayData(DisplayData.Builder builder) { HashSet specs = new HashSet<>(optionsMap.get(option.getKey())); for (PipelineOptionSpec optionSpec : specs) { + if (!optionSpec.shouldSerialize()) { + // Options that are excluded for serialization (i.e. those with @JsonIgnore) are also + // excluded from display data. These options are generally not useful for display. + continue; + } + Class pipelineInterface = optionSpec.getDefiningInterface(); if (type != null) { - builder.add(option.getKey(), type, value) - .withNamespace(pipelineInterface); + builder.add(DisplayData.item(option.getKey(), type, value) + .withNamespace(pipelineInterface)); } else { - builder.add(option.getKey(), value.toString()) - .withNamespace(pipelineInterface); + builder.add(DisplayData.item(option.getKey(), value.toString()) + .withNamespace(pipelineInterface)); } } } @@ -336,18 +342,22 @@ private void populateDisplayData(DisplayData.Builder builder) { HashSet specs = new HashSet<>(optionsMap.get(jsonOption.getKey())); if (specs.isEmpty()) { - builder.add(jsonOption.getKey(), jsonOption.getValue().toString()) - .withNamespace(UnknownPipelineOptions.class); + builder.add(DisplayData.item(jsonOption.getKey(), jsonOption.getValue().toString()) + .withNamespace(UnknownPipelineOptions.class)); } else { for (PipelineOptionSpec spec : specs) { + if (!spec.shouldSerialize()) { + continue; + } + Object value = getValueFromJson(jsonOption.getKey(), spec.getGetterMethod()); DisplayData.Type type = DisplayData.inferType(value); if (type != null) { - builder.add(jsonOption.getKey(), type, value) - .withNamespace(spec.getDefiningInterface()); + builder.add(DisplayData.item(jsonOption.getKey(), type, value) + .withNamespace(spec.getDefiningInterface())); } else { - builder.add(jsonOption.getKey(), value.toString()) - .withNamespace(spec.getDefiningInterface()); + builder.add(DisplayData.item(jsonOption.getKey(), value.toString()) + .withNamespace(spec.getDefiningInterface())); } } } @@ -578,7 +588,7 @@ public void serialize(PipelineOptions value, JsonGenerator jgen, SerializerProvi jgen.writeObject(serializableOptions); List> serializedDisplayData = Lists.newArrayList(); - for (DisplayData.Item item : DisplayData.from(value).items()) { + for (DisplayData.Item item : DisplayData.from(value).items()) { @SuppressWarnings("unchecked") Map serializedItem = MAPPER.convertValue(item, Map.class); serializedDisplayData.add(serializedItem); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java index 3690b62c7d..8099f0955e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineRunner.java @@ -94,6 +94,7 @@ import com.google.cloud.dataflow.sdk.transforms.View; import com.google.cloud.dataflow.sdk.transforms.View.CreatePCollectionView; import com.google.cloud.dataflow.sdk.transforms.WithKeys; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.transforms.windowing.AfterPane; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.DefaultTrigger; @@ -2137,6 +2138,11 @@ public PCollection apply(PInput input) { .setCoder(TableRowJsonCoder.of()); } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + transform.populateDisplayData(builder); + } + static { DataflowPipelineTranslator.registerTransformTranslator( BatchBigQueryIONativeRead.class, new BatchBigQueryIONativeReadTranslator()); @@ -2232,6 +2238,11 @@ public PDone apply(PCollection input) { return PDone.in(input.getPipeline()); } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + transform.populateDisplayData(builder); + } + static { DataflowPipelineTranslator.registerTransformTranslator( BatchBigQueryIONativeWrite.class, new BatchBigQueryIONativeWriteTranslator()); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java index 19bc3d82d7..fb477d04c6 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java @@ -28,6 +28,7 @@ import static com.google.cloud.dataflow.sdk.util.Structs.addString; import static com.google.cloud.dataflow.sdk.util.Structs.getString; import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; import com.google.api.services.dataflow.model.AutoscalingSettings; import com.google.api.services.dataflow.model.DataflowPackage; @@ -58,6 +59,7 @@ import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.transforms.View; import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; +import com.google.cloud.dataflow.sdk.transforms.display.HasDisplayData; import com.google.cloud.dataflow.sdk.transforms.windowing.DefaultTrigger; import com.google.cloud.dataflow.sdk.transforms.windowing.Window; import com.google.cloud.dataflow.sdk.util.AppliedCombineFn; @@ -86,6 +88,8 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.io.PrintWriter; +import java.io.StringWriter; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -547,7 +551,7 @@ public void addStep(PTransform transform, String type) { currentStep.setKind(type); steps.add(currentStep); addInput(PropertyNames.USER_NAME, getFullName(transform)); - addDisplayData(PropertyNames.DISPLAY_DATA, DisplayData.from(transform)); + addDisplayData(stepName, transform); } @Override @@ -672,7 +676,7 @@ public void addCollectionToSingletonOutput(String name, PValue inputValue, PValue outputValue) { Coder inputValueCoder = - Preconditions.checkNotNull(outputCoders.get(inputValue)); + checkNotNull(outputCoders.get(inputValue)); // The inputValueCoder for the input PCollection should be some // WindowedValueCoder of the input PCollection's element // coder. @@ -725,9 +729,21 @@ private void addOutput(String name, PValue value, Coder valueCoder) { outputInfoList.add(outputInfo); } - private void addDisplayData(String name, DisplayData displayData) { + private void addDisplayData(String stepName, HasDisplayData hasDisplayData) { + DisplayData displayData; + try { + displayData = DisplayData.from(hasDisplayData); + } catch (Exception e) { + String msg = String.format("Exception thrown while collecting display data for step: %s. " + + "Display data will be not be available for this step.", stepName); + DisplayDataException displayDataException = new DisplayDataException(msg, e); + LOG.warn(msg, displayDataException); + + displayData = displayDataException.asDisplayData(); + } + List> list = MAPPER.convertValue(displayData, List.class); - addList(getProperties(), name, list); + addList(getProperties(), PropertyNames.DISPLAY_DATA, list); } @Override @@ -1089,4 +1105,38 @@ private static void translateOutputs( context.addOutput(tag.getId(), output); } } + + /** + * Wraps exceptions thrown while collecting {@link DisplayData} for the Dataflow pipeline runner. + */ + static class DisplayDataException extends Exception implements HasDisplayData { + public DisplayDataException(String message, Throwable cause) { + super(checkNotNull(message), checkNotNull(cause)); + } + + /** + * Retrieve a display data representation of the exception, which can be submitted to + * the service in place of the actual display data. + */ + public DisplayData asDisplayData() { + return DisplayData.from(this); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + Throwable cause = getCause(); + builder + .add(DisplayData.item("exceptionMessage", getMessage())) + .add(DisplayData.item("exceptionType", cause.getClass())) + .add(DisplayData.item("exceptionCause", cause.getMessage())) + .add(DisplayData.item("stackTrace", stackTraceToString())); + } + + private String stackTraceToString() { + StringWriter stringWriter = new StringWriter(); + PrintWriter printWriter = new PrintWriter(stringWriter); + printStackTrace(printWriter); + return stringWriter.toString(); + } + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java index 0f4e738993..a1c330862b 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java @@ -361,9 +361,12 @@ public Coder> getAccumulatorCoder( @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder - .add("numQuantiles", numQuantiles) - .add("comparer", compareFn.getClass()); + .add(DisplayData.item("numQuantiles", numQuantiles) + .withLabel("Quantile Count")) + .add(DisplayData.item("comparer", compareFn.getClass()) + .withLabel("Record Comparer")); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUnique.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUnique.java index 27176f6f33..ff5e131012 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUnique.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUnique.java @@ -214,6 +214,7 @@ public PCollection apply(PCollection input) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); ApproximateUnique.populateDisplayData(builder, sampleSize, maximumEstimationError); } } @@ -287,6 +288,7 @@ public PCollection> apply(PCollection> input) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); ApproximateUnique.populateDisplayData(builder, sampleSize, maximumEstimationError); } } @@ -457,7 +459,9 @@ static long sampleSizeFromEstimationError(double estimationError) { private static void populateDisplayData( DisplayData.Builder builder, long sampleSize, Double maxEstimationError) { builder - .add("sampleSize", sampleSize) - .addIfNotNull("maximumEstimationError", maxEstimationError); + .add(DisplayData.item("sampleSize", sampleSize) + .withLabel("Sample Size")) + .addIfNotNull(DisplayData.item("maximumEstimationError", maxEstimationError) + .withLabel("Maximum Estimation Error")); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java index 22afcce409..f608988c9e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java @@ -34,7 +34,6 @@ import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.Context; import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext; import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.RequiresContextInternal; -import com.google.cloud.dataflow.sdk.transforms.display.ClassForDisplay; import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.transforms.display.HasDisplayData; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; @@ -100,7 +99,7 @@ private Combine() { */ public static Globally globally( SerializableFunction, V> combiner) { - return globally(IterableCombineFn.of(combiner), ClassForDisplay.fromInstance(combiner)); + return globally(IterableCombineFn.of(combiner), displayDataForFn(combiner)); } @@ -121,12 +120,18 @@ public static Globally globally( */ public static Globally globally( GlobalCombineFn fn) { - return globally(fn, ClassForDisplay.fromInstance(fn)); + return globally(fn, displayDataForFn(fn)); + } + + private static DisplayData.Item> displayDataForFn(T fn) { + return DisplayData.item("combineFn", fn.getClass()) + .withLabel("Combiner"); } private static Globally globally( - GlobalCombineFn fn, ClassForDisplay fnClass) { - return new Globally<>(fn, fnClass, true, 0); + GlobalCombineFn fn, + DisplayData.Item> fnDisplayData) { + return new Globally<>(fn, fnDisplayData, true, 0); } /** @@ -147,7 +152,7 @@ private static Globally globally( */ public static PerKey perKey( SerializableFunction, V> fn) { - return perKey(IterableCombineFn.of(fn).asKeyedFn(), ClassForDisplay.fromInstance(fn)); + return perKey(IterableCombineFn.of(fn).asKeyedFn(), displayDataForFn(fn)); } /** @@ -168,7 +173,7 @@ public static PerKey perKey( */ public static PerKey perKey( GlobalCombineFn fn) { - return perKey(fn.asKeyedFn(), ClassForDisplay.fromInstance(fn)); + return perKey(fn.asKeyedFn(), displayDataForFn(fn)); } /** @@ -189,12 +194,13 @@ public static PerKey perKey( */ public static PerKey perKey( PerKeyCombineFn fn) { - return perKey(fn, ClassForDisplay.fromInstance(fn)); + return perKey(fn, displayDataForFn(fn)); } private static PerKey perKey( - PerKeyCombineFn fn, ClassForDisplay fnClass) { - return new PerKey<>(fn, fnClass, false /*fewKeys*/); + PerKeyCombineFn fn, + DisplayData.Item> fnDisplayData) { + return new PerKey<>(fn, fnDisplayData, false /*fewKeys*/); } /** @@ -202,8 +208,9 @@ private static PerKey perKey( * in {@link GroupByKey}. */ private static PerKey fewKeys( - PerKeyCombineFn fn, ClassForDisplay fnClass) { - return new PerKey<>(fn, fnClass, true /*fewKeys*/); + PerKeyCombineFn fn, + DisplayData.Item> fnDisplayData) { + return new PerKey<>(fn, fnDisplayData, true /*fewKeys*/); } /** @@ -229,7 +236,7 @@ private static PerKey fewKeys( */ public static GroupedValues groupedValues( SerializableFunction, V> fn) { - return groupedValues(IterableCombineFn.of(fn).asKeyedFn(), ClassForDisplay.fromInstance(fn)); + return groupedValues(IterableCombineFn.of(fn).asKeyedFn(), displayDataForFn(fn)); } /** @@ -255,7 +262,7 @@ public static GroupedValues groupedValues( */ public static GroupedValues groupedValues( GlobalCombineFn fn) { - return groupedValues(fn.asKeyedFn(), ClassForDisplay.fromInstance(fn)); + return groupedValues(fn.asKeyedFn(), displayDataForFn(fn)); } /** @@ -281,12 +288,13 @@ public static GroupedValues groupedValu */ public static GroupedValues groupedValues( PerKeyCombineFn fn) { - return groupedValues(fn, ClassForDisplay.fromInstance(fn)); + return groupedValues(fn, displayDataForFn(fn)); } private static GroupedValues groupedValues( - PerKeyCombineFn fn, ClassForDisplay fnClass) { - return new GroupedValues<>(fn, fnClass); + PerKeyCombineFn fn, + DisplayData.Item> fnDisplayData) { + return new GroupedValues<>(fn, fnDisplayData); } ///////////////////////////////////////////////////////////////////////////// @@ -428,7 +436,7 @@ public AccumT compact(AccumT accumulator) { * *

Useful when using a {@code CombineFn} separately from a * {@code Combine} transform. Does not invoke the - * {@link mergeAccumulators} operation. + * {@link #mergeAccumulators} operation. */ public OutputT apply(Iterable inputs) { AccumT accum = createAccumulator(); @@ -512,7 +520,7 @@ public CombineFn forKey(K key, Coder keyCoder) { @Override public void populateDisplayData(DisplayData.Builder builder) { - CombineFn.this.populateDisplayData(builder); + builder.include(CombineFn.this); } }; } @@ -1190,7 +1198,7 @@ public Coder getDefaultOutputCoder( @Override public void populateDisplayData(DisplayData.Builder builder) { - KeyedCombineFn.this.populateDisplayData(builder); + builder.include(KeyedCombineFn.this); } }; } @@ -1257,36 +1265,36 @@ public static class Globally extends PTransform, PCollection> { private final GlobalCombineFn fn; - private final ClassForDisplay fnClass; + private final DisplayData.Item> fnDisplayData; private final boolean insertDefault; private final int fanout; private final List> sideInputs; - private Globally(GlobalCombineFn fn, ClassForDisplay fnClass, - boolean insertDefault, int fanout) { + private Globally(GlobalCombineFn fn, + DisplayData.Item> fnDisplayData, boolean insertDefault, int fanout) { this.fn = fn; - this.fnClass = fnClass; + this.fnDisplayData = fnDisplayData; this.insertDefault = insertDefault; this.fanout = fanout; - this.sideInputs = ImmutableList.>of(); + this.sideInputs = ImmutableList.of(); } private Globally(String name, GlobalCombineFn fn, - ClassForDisplay fnClass, boolean insertDefault, int fanout) { + DisplayData.Item> fnDisplayData, boolean insertDefault, int fanout) { super(name); this.fn = fn; - this.fnClass = fnClass; + this.fnDisplayData = fnDisplayData; this.insertDefault = insertDefault; this.fanout = fanout; - this.sideInputs = ImmutableList.>of(); + this.sideInputs = ImmutableList.of(); } private Globally(String name, GlobalCombineFn fn, - ClassForDisplay fnClass, boolean insertDefault, int fanout, + DisplayData.Item> fnDisplayData, boolean insertDefault, int fanout, List> sideInputs) { super(name); this.fn = fn; - this.fnClass = fnClass; + this.fnDisplayData = fnDisplayData; this.insertDefault = insertDefault; this.fanout = fanout; this.sideInputs = sideInputs; @@ -1297,7 +1305,7 @@ private Globally(String name, GlobalCombineFn fn, * specified name. Does not modify this transform. */ public Globally named(String name) { - return new Globally<>(name, fn, fnClass, insertDefault, fanout); + return new Globally<>(name, fn, fnDisplayData, insertDefault, fanout); } /** @@ -1308,7 +1316,7 @@ public Globally named(String name) { * to an empty input set will be returned. */ public GloballyAsSingletonView asSingletonView() { - return new GloballyAsSingletonView<>(fn, fnClass, insertDefault, fanout); + return new GloballyAsSingletonView<>(fn, fnDisplayData, insertDefault, fanout); } /** @@ -1317,7 +1325,7 @@ public GloballyAsSingletonView asSingletonView() { * is not globally windowed and the output is not being used as a side input. */ public Globally withoutDefaults() { - return new Globally<>(name, fn, fnClass, false, fanout); + return new Globally<>(name, fn, fnDisplayData, false, fanout); } /** @@ -1328,7 +1336,7 @@ public Globally withoutDefaults() { * that will be used. */ public Globally withFanout(int fanout) { - return new Globally<>(name, fn, fnClass, insertDefault, fanout); + return new Globally<>(name, fn, fnDisplayData, insertDefault, fanout); } /** @@ -1338,8 +1346,8 @@ public Globally withFanout(int fanout) { public Globally withSideInputs( Iterable> sideInputs) { Preconditions.checkState(fn instanceof RequiresContextInternal); - return new Globally(name, fn, fnClass, insertDefault, fanout, - ImmutableList.>copyOf(sideInputs)); + return new Globally<>(name, fn, fnDisplayData, insertDefault, fanout, + ImmutableList.copyOf(sideInputs)); } @Override @@ -1349,7 +1357,7 @@ public PCollection apply(PCollection input) { .setCoder(KvCoder.of(VoidCoder.of(), input.getCoder())); Combine.PerKey combine = - Combine.fewKeys(fn.asKeyedFn(), fnClass); + Combine.fewKeys(fn.asKeyedFn(), fnDisplayData); if (!sideInputs.isEmpty()) { combine = combine.withSideInputs(sideInputs); } @@ -1375,7 +1383,9 @@ public PCollection apply(PCollection input) { @Override public void populateDisplayData(DisplayData.Builder builder) { - Combine.populateDisplayData(builder, fn, fnClass); + super.populateDisplayData(builder); + + Combine.populateDisplayData(builder, fn, fnDisplayData); Combine.populateGlobalDisplayData(builder, fanout, insertDefault); } @@ -1406,17 +1416,20 @@ public void processElement(DoFn.ProcessContext c) { } private static void populateDisplayData( - DisplayData.Builder builder, HasDisplayData fn, ClassForDisplay fnClass) { + DisplayData.Builder builder, HasDisplayData fn, + DisplayData.Item> fnDisplayItem) { builder - .include(fn, fnClass) - .add("combineFn", fnClass); + .include(fn) + .add(fnDisplayItem); } private static void populateGlobalDisplayData( DisplayData.Builder builder, int fanout, boolean insertDefault) { builder - .addIfNotDefault("fanout", fanout, 0) - .add("emitDefaultOnEmptyInput", insertDefault); + .addIfNotDefault(DisplayData.item("fanout", fanout) + .withLabel("Key Fanout Size"), 0) + .add(DisplayData.item("emitDefaultOnEmptyInput", insertDefault) + .withLabel("Emit Default On Empty Input")); } /** @@ -1462,15 +1475,15 @@ public static class GloballyAsSingletonView extends PTransform, PCollectionView> { private final GlobalCombineFn fn; - private final ClassForDisplay fnClass; + private final DisplayData.Item> fnDisplayData; private final boolean insertDefault; private final int fanout; private GloballyAsSingletonView( - GlobalCombineFn fn, ClassForDisplay fnClass, - boolean insertDefault, int fanout) { + GlobalCombineFn fn, + DisplayData.Item> fnDisplayData, boolean insertDefault, int fanout) { this.fn = fn; - this.fnClass = fnClass; + this.fnDisplayData = fnDisplayData; this.insertDefault = insertDefault; this.fanout = fanout; } @@ -1504,7 +1517,9 @@ public boolean getInsertDefault() { @Override public void populateDisplayData(DisplayData.Builder builder) { - Combine.populateDisplayData(builder, fn, fnClass); + super.populateDisplayData(builder); + + Combine.populateDisplayData(builder, fn, fnDisplayData); Combine.populateGlobalDisplayData(builder, fanout, insertDefault); } } @@ -1588,7 +1603,9 @@ public List compact(List accumulator) { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("combineFn", combiner.getClass()); + super.populateDisplayData(builder); + builder.add(DisplayData.item("combineFn", combiner.getClass()) + .withLabel("Combiner")); } private List mergeToSingleton(Iterable values) { @@ -1664,35 +1681,36 @@ public static class PerKey extends PTransform>, PCollection>> { private final transient PerKeyCombineFn fn; - private final ClassForDisplay fnClass; + private final DisplayData.Item> fnDisplayData; private final boolean fewKeys; private final List> sideInputs; private PerKey( - PerKeyCombineFn fn, ClassForDisplay fnClass, - boolean fewKeys) { + PerKeyCombineFn fn, + DisplayData.Item> fnDisplayData, boolean fewKeys) { this.fn = fn; - this.fnClass = fnClass; + this.fnDisplayData = fnDisplayData; this.fewKeys = fewKeys; this.sideInputs = ImmutableList.of(); } private PerKey(String name, - PerKeyCombineFn fn, ClassForDisplay fnClass, + PerKeyCombineFn fn, + DisplayData.Item> fnDisplayData, boolean fewKeys, List> sideInputs) { super(name); this.fn = fn; - this.fnClass = fnClass; + this.fnDisplayData = fnDisplayData; this.fewKeys = fewKeys; this.sideInputs = sideInputs; } private PerKey( String name, PerKeyCombineFn fn, - ClassForDisplay fnClass, boolean fewKeys) { + DisplayData.Item> fnDisplayData, boolean fewKeys) { super(name); this.fn = fn; - this.fnClass = fnClass; + this.fnDisplayData = fnDisplayData; this.fewKeys = fewKeys; this.sideInputs = ImmutableList.of(); } @@ -1702,7 +1720,7 @@ private PerKey( * specified name. Does not modify this transform. */ public PerKey named(String name) { - return new PerKey(name, fn, fnClass, fewKeys); + return new PerKey<>(name, fn, fnDisplayData, fewKeys); } /** @@ -1712,8 +1730,8 @@ public PerKey named(String name) { public PerKey withSideInputs( Iterable> sideInputs) { Preconditions.checkState(fn instanceof RequiresContextInternal); - return new PerKey(name, fn, fnClass, fewKeys, - ImmutableList.>copyOf(sideInputs)); + return new PerKey<>(name, fn, fnDisplayData, fewKeys, + ImmutableList.copyOf(sideInputs)); } /** @@ -1729,7 +1747,7 @@ public PerKey withSideInputs( */ public PerKeyWithHotKeyFanout withHotKeyFanout( SerializableFunction hotKeyFanout) { - return new PerKeyWithHotKeyFanout(name, fn, fnClass, hotKeyFanout); + return new PerKeyWithHotKeyFanout<>(name, fn, fnDisplayData, hotKeyFanout); } /** @@ -1737,8 +1755,8 @@ public PerKeyWithHotKeyFanout withHotKeyFanout( * constant value for every key. */ public PerKeyWithHotKeyFanout withHotKeyFanout(final int hotKeyFanout) { - return new PerKeyWithHotKeyFanout(name, fn, fnClass, - new SerializableFunction(){ + return new PerKeyWithHotKeyFanout<>(name, fn, fnDisplayData, + new SerializableFunction() { @Override public Integer apply(K unused) { return hotKeyFanout; @@ -1764,12 +1782,13 @@ public List> getSideInputs() { public PCollection> apply(PCollection> input) { return input .apply(GroupByKey.create(fewKeys)) - .apply(Combine.groupedValues(fn).withSideInputs(sideInputs)); + .apply(Combine.groupedValues(fn, fnDisplayData).withSideInputs(sideInputs)); } @Override public void populateDisplayData(DisplayData.Builder builder) { - Combine.populateDisplayData(builder, fn, fnClass); + super.populateDisplayData(builder); + Combine.populateDisplayData(builder, fn, fnDisplayData); } } @@ -1780,16 +1799,16 @@ public static class PerKeyWithHotKeyFanout extends PTransform>, PCollection>> { private final transient PerKeyCombineFn fn; - private final ClassForDisplay fnClass; + private final DisplayData.Item> fnDisplayData; private final SerializableFunction hotKeyFanout; private PerKeyWithHotKeyFanout(String name, PerKeyCombineFn fn, - ClassForDisplay fnClass, + DisplayData.Item> fnDisplayData, SerializableFunction hotKeyFanout) { super(name); this.fn = fn; - this.fnClass = fnClass; + this.fnDisplayData = fnDisplayData; this.hotKeyFanout = hotKeyFanout; } @@ -2074,8 +2093,11 @@ public void processElement(ProcessContext c) { @Override public void populateDisplayData(DisplayData.Builder builder) { - Combine.populateDisplayData(builder, fn, fnClass); - builder.add("fanoutFn", hotKeyFanout.getClass()); + super.populateDisplayData(builder); + + Combine.populateDisplayData(builder, fn, fnDisplayData); + builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass()) + .withLabel("Fanout Function")); } /** @@ -2219,28 +2241,29 @@ public static class GroupedValues PCollection>> { private final PerKeyCombineFn fn; - private final ClassForDisplay fnClass; + private final DisplayData.Item> fnDisplayData; private final List> sideInputs; private GroupedValues( - PerKeyCombineFn fn, ClassForDisplay fnClass) { + PerKeyCombineFn fn, + DisplayData.Item> fnDisplayData) { this.fn = SerializableUtils.clone(fn); - this.fnClass = fnClass; - this.sideInputs = ImmutableList.>of(); + this.fnDisplayData = fnDisplayData; + this.sideInputs = ImmutableList.of(); } private GroupedValues( PerKeyCombineFn fn, - ClassForDisplay fnClass, + DisplayData.Item> fnDisplayData, List> sideInputs) { this.fn = SerializableUtils.clone(fn); - this.fnClass = fnClass; + this.fnDisplayData = fnDisplayData; this.sideInputs = sideInputs; } public GroupedValues withSideInputs( Iterable> sideInputs) { - return new GroupedValues<>(fn, fnClass, ImmutableList.>copyOf(sideInputs)); + return new GroupedValues<>(fn, fnDisplayData, ImmutableList.copyOf(sideInputs)); } /** @@ -2330,7 +2353,8 @@ public Coder> getDefaultOutputCoder( @Override public void populateDisplayData(DisplayData.Builder builder) { - Combine.populateDisplayData(builder, fn, fnClass); + super.populateDisplayData(builder); + Combine.populateDisplayData(builder, fn, fnDisplayData); } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/CombineFnBase.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/CombineFnBase.java index d278858478..938c5db6eb 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/CombineFnBase.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/CombineFnBase.java @@ -222,7 +222,7 @@ public TypeVariable getOutputTVariable() { * {@inheritDoc} * *

By default, does not register any display data. Implementors may override this method - * to provide their own display metadata. + * to provide their own display data. */ @Override public void populateDisplayData(DisplayData.Builder builder) { @@ -297,7 +297,7 @@ public TypeVariable getOutputTVariable() { * {@inheritDoc} * *

By default, does not register any display data. Implementors may override this method - * to provide their own display metadata. + * to provide their own display data. */ @Override public void populateDisplayData(DisplayData.Builder builder) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/CombineFns.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/CombineFns.java index 120dbbf573..44004660d9 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/CombineFns.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/CombineFns.java @@ -460,6 +460,7 @@ public Coder getAccumulatorCoder(CoderRegistry registry, Coder @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); CombineFns.populateDisplayData(builder, combineFns); } } @@ -598,6 +599,7 @@ public Coder getAccumulatorCoder(CoderRegistry registry, Coder @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); CombineFns.populateDisplayData(builder, combineFnWithContexts); } } @@ -784,6 +786,7 @@ public Coder getAccumulatorCoder( @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); CombineFns.populateDisplayData(builder, keyedCombineFns); } } @@ -935,6 +938,7 @@ public Coder getAccumulatorCoder( @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); CombineFns.populateDisplayData(builder, keyedCombineFns); } } @@ -1138,7 +1142,8 @@ private static void populateDisplayData( for (int i = 0; i < combineFns.size(); i++) { HasDisplayData combineFn = combineFns.get(i); - builder.add("combineFn" + (i + 1), combineFn.getClass()); + builder.add(DisplayData.item("combineFn" + (i + 1), combineFn.getClass()) + .withLabel("Combine Function")); combineFnMap.put(combineFn.getClass(), combineFn); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/CombineWithContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/CombineWithContext.java index b26636ee20..e9f29fccb4 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/CombineWithContext.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/CombineWithContext.java @@ -169,6 +169,7 @@ public CombineFnWithContext forKey(K key, Coder keyC @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); CombineFnWithContext.this.populateDisplayData(builder); } }; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java index 5ba9992143..3947c632dd 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFn.java @@ -372,7 +372,7 @@ public void finishBundle(Context c) throws Exception { * {@inheritDoc} * *

By default, does not register any display data. Implementors may override this method - * to provide their own display metadata. + * to provide their own display data. */ @Override public void populateDisplayData(DisplayData.Builder builder) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnReflector.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnReflector.java index 559603aecf..a5ab9be334 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnReflector.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnReflector.java @@ -652,7 +652,7 @@ protected TypeDescriptor getOutputTypeDescriptor() { @Override public void populateDisplayData(DisplayData.Builder builder) { - fn.populateDisplayData(builder); + builder.include(fn); } private void readObject(java.io.ObjectInputStream in) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnWithContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnWithContext.java index 452de2d10e..355a100518 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnWithContext.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnWithContext.java @@ -420,7 +420,7 @@ void prepareForProcessing() { * {@inheritDoc} * *

By default, does not register any display data. Implementors may override this method - * to provide their own display metadata. + * to provide their own display data. */ @Override public void populateDisplayData(DisplayData.Builder builder) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Filter.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Filter.java index f0860d841f..cbd58cc071 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Filter.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Filter.java @@ -102,6 +102,7 @@ public void processElement(ProcessContext c) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); Filter.populateDisplayData(builder, String.format("x < %s", value)); } }); @@ -140,6 +141,7 @@ public void processElement(ProcessContext c) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); Filter.populateDisplayData(builder, String.format("x > %s", value)); } }); @@ -177,6 +179,7 @@ public void processElement(ProcessContext c) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); Filter.populateDisplayData(builder, String.format("x ≤ %s", value)); } }); @@ -214,6 +217,7 @@ public void processElement(ProcessContext c) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); Filter.populateDisplayData(builder, String.format("x ≥ %s", value)); } }); @@ -256,6 +260,7 @@ protected Coder getDefaultOutputCoder(PCollection input) { private static void populateDisplayData( DisplayData.Builder builder, String predicateDescription) { - builder.add("predicate", predicateDescription); + builder.add(DisplayData.item("predicate", predicateDescription) + .withLabel("Filter Predicate")); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java index 1125ea5e50..2bf1b97f79 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java @@ -318,8 +318,10 @@ static KvCoder> getOutputKvCoder(Coder> inputCode @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); if (fewKeys) { - builder.add("fewKeys", true); + builder.add(DisplayData.item("fewKeys", true) + .withLabel("Has Few Keys")); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/IntraBundleParallelization.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/IntraBundleParallelization.java index 45b7c8afaf..0a62cd65ec 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/IntraBundleParallelization.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/IntraBundleParallelization.java @@ -174,9 +174,12 @@ public PCollection apply(PCollection input) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder - .add("maxParallelism", maxParallelism) - .add("fn", doFn.getClass()) + .add(DisplayData.item("maxParallelism", maxParallelism) + .withLabel("Maximum Parallelism")) + .add(DisplayData.item("fn", doFn.getClass()) + .withLabel("Function")) .include(doFn); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/MapElements.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/MapElements.java index 8997050864..d0733933c3 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/MapElements.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/MapElements.java @@ -16,6 +16,7 @@ package com.google.cloud.dataflow.sdk.transforms; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.TypeDescriptor; @@ -107,6 +108,18 @@ public PCollection apply(PCollection input) { public void processElement(ProcessContext c) { c.output(fn.apply(c.element())); } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + MapElements.this.populateDisplayData(builder); + } })).setTypeDescriptorInternal(outputType); } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder.add(DisplayData.item("mapFn", fn.getClass()) + .withLabel("Map Function")); + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Max.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Max.java index 70e541c42e..04bd360173 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Max.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Max.java @@ -207,7 +207,9 @@ public T apply(T left, T right) { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("comparer", comparator.getClass()); + super.populateDisplayData(builder); + builder.add(DisplayData.item("comparer", comparator.getClass()) + .withLabel("Record Comparer")); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Min.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Min.java index 5e9d7041c8..7194d9d3aa 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Min.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Min.java @@ -207,7 +207,9 @@ public T apply(T left, T right) { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("comparer", comparator.getClass()); + super.populateDisplayData(builder); + builder.add(DisplayData.item("comparer", comparator.getClass()) + .withLabel("Record Comparer")); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java index 21355e17ba..8499bcb141 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/PTransform.java @@ -316,7 +316,7 @@ public Coder getDefaultOutputCoder( * {@inheritDoc} * *

By default, does not register any display data. Implementors may override this method - * to provide their own display metadata. + * to provide their own display data. */ @Override public void populateDisplayData(Builder builder) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java index 2ab61ba5ee..29bfb7f549 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java @@ -806,12 +806,13 @@ protected String getKindString() { /** * {@inheritDoc} * - *

{@link ParDo} registers its internal {@link DoFn} as a subcomponent for display metadata. + *

{@link ParDo} registers its internal {@link DoFn} as a subcomponent for display data. * {@link DoFn} implementations can register display data by overriding * {@link DoFn#populateDisplayData}. */ @Override public void populateDisplayData(Builder builder) { + super.populateDisplayData(builder); ParDo.populateDisplayData(builder, fn, fnClass); } @@ -1049,6 +1050,7 @@ protected String getKindString() { @Override public void populateDisplayData(Builder builder) { + super.populateDisplayData(builder); ParDo.populateDisplayData(builder, fn, fnClass); } @@ -1261,8 +1263,9 @@ private static SideInputReader makeSideInputReader( private static void populateDisplayData( DisplayData.Builder builder, DoFn fn, Class fnClass) { builder - .include(fn, fnClass) - .add("fn", fnClass); + .include(fn) + .add(DisplayData.item("fn", fnClass) + .withLabel("Transform Function")); } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Partition.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Partition.java index 623875caa8..1962a8e7e4 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Partition.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Partition.java @@ -123,6 +123,7 @@ public PCollectionList apply(PCollection in) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder.include(partitionDoFn); } @@ -178,9 +179,12 @@ public void processElement(ProcessContext c) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder - .add("numPartitions", numPartitions) - .add("partitionFn", partitionFn.getClass()); + .add(DisplayData.item("numPartitions", numPartitions) + .withLabel("Partition Count")) + .add(DisplayData.item("partitionFn", partitionFn.getClass()) + .withLabel("Partition Function")); } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sample.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sample.java index a72443ce49..0de14f661f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sample.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sample.java @@ -155,7 +155,9 @@ public PCollection apply(PCollection in) { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("sampleSize", limit); + super.populateDisplayData(builder); + builder.add(DisplayData.item("sampleSize", limit) + .withLabel("Sample Size")); } } @@ -254,7 +256,9 @@ public Coder> getDefaultOutputCoder( @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("sampleSize", sampleSize); + super.populateDisplayData(builder); + builder.add(DisplayData.item("sampleSize", sampleSize) + .withLabel("Sample Size")); } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java index e46da47539..6e2c1c91aa 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java @@ -393,9 +393,12 @@ public Coder> getAccumulatorCoder( @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder - .add("count", count) - .add("comparer", compareFn.getClass()); + .add(DisplayData.item("count", count) + .withLabel("Top Count")) + .add(DisplayData.item("comparer", compareFn.getClass()) + .withLabel("Record Comparer")); } @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/ClassForDisplay.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/ClassForDisplay.java deleted file mode 100644 index 7316c2a272..0000000000 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/ClassForDisplay.java +++ /dev/null @@ -1,91 +0,0 @@ -/* - * Copyright (C) 2016 Google Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ -package com.google.cloud.dataflow.sdk.transforms.display; - -import static com.google.common.base.Preconditions.checkNotNull; - -import java.io.Serializable; -import java.util.Objects; - -/** - * Display metadata representing a Java class. - *

- *

Java classes can be registered as display metadata via - * {@link DisplayData.Builder#add(String, ClassForDisplay)}. {@link ClassForDisplay} is - * serializable, unlike {@link Class} which can fail to serialize for Java 8 lambda functions. - */ -public class ClassForDisplay implements Serializable { - private final String simpleName; - private final String name; - - private ClassForDisplay(Class clazz) { - name = clazz.getName(); - simpleName = clazz.getSimpleName(); - } - - /** - * Create a {@link ClassForDisplay} instance representing the specified class. - */ - public static ClassForDisplay of(Class clazz) { - return new ClassForDisplay(checkNotNull(clazz)); - } - - /** - * Create a {@link ClassForDisplay} from the class of the specified object instance. - */ - public static ClassForDisplay fromInstance(Object obj) { - checkNotNull(obj); - return new ClassForDisplay(obj.getClass()); - } - - /** - * Retrieve the fully-qualified name of the class. - * - * @see Class#getName() - */ - public String getName() { - return name; - } - - /** - * Retrieve a simple representation of the class name. - * - * @see Class#getSimpleName() - */ - public String getSimpleName() { - return simpleName; - } - - @Override - public String toString() { - return name; - } - - @Override - public int hashCode() { - return name.hashCode(); - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof ClassForDisplay) { - ClassForDisplay that = (ClassForDisplay) obj; - return Objects.equals(this.name, that.name); - } - - return false; - } -} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayData.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayData.java index eefbfb6078..05485e4f7a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayData.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayData.java @@ -16,12 +16,10 @@ package com.google.cloud.dataflow.sdk.transforms.display; -import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.PTransform; -import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.common.collect.Sets; @@ -36,26 +34,27 @@ import org.joda.time.format.DateTimeFormatter; import org.joda.time.format.ISODateTimeFormat; +import java.io.Serializable; import java.util.Collection; import java.util.Map; import java.util.Objects; import java.util.Set; /** - * Static display metadata associated with a pipeline component. Display data is useful for + * Static display data associated with a pipeline component. Display data is useful for * pipeline runner UIs and diagnostic dashboards to display details about * {@link PTransform PTransforms} that make up a pipeline. * *

Components specify their display data by implementing the {@link HasDisplayData} * interface. */ -public class DisplayData { - private static final DisplayData EMPTY = new DisplayData(Maps.newHashMap()); +public class DisplayData implements Serializable { + private static final DisplayData EMPTY = new DisplayData(Maps.>newHashMap()); private static final DateTimeFormatter TIMESTAMP_FORMATTER = ISODateTimeFormat.dateTime(); - private final ImmutableMap entries; + private final ImmutableMap> entries; - private DisplayData(Map entries) { + private DisplayData(Map> entries) { this.entries = ImmutableMap.copyOf(entries); } @@ -70,9 +69,13 @@ public static DisplayData none() { * Collect the {@link DisplayData} from a component. This will traverse all subcomponents * specified via {@link Builder#include} in the given component. Data in this component will be in * a namespace derived from the component. + * + *

Pipeline runners should call this method in order to collect display data. While it should + * be safe to call {@code DisplayData.from} on any component which implements it, runners should + * be resilient to exceptions thrown while collecting display data. */ public static DisplayData from(HasDisplayData component) { - checkNotNull(component); + checkNotNull(component, "component argument cannot be null"); return InternalBuilder.forRoot(component).build(); } @@ -87,7 +90,7 @@ public static DisplayData from(HasDisplayData component) { * public void populateDisplayData(DisplayData.Builder builder) { * Optional type = DisplayData.inferType(foo); * if (type.isPresent()) { - * builder.add("foo", type.get(), foo); + * builder.add(DisplayData.item("foo", type.get(), foo)); * } * } * } @@ -101,11 +104,11 @@ public static Type inferType(@Nullable Object value) { } @JsonValue - public Collection items() { + public Collection> items() { return entries.values(); } - public Map asMap() { + public Map> asMap() { return entries; } @@ -128,7 +131,7 @@ public boolean equals(Object obj) { public String toString() { StringBuilder builder = new StringBuilder(); boolean isFirstLine = true; - for (Item entry : entries.values()) { + for (Item entry : entries.values()) { if (isFirstLine) { isFirstLine = false; } else { @@ -141,345 +144,141 @@ public String toString() { return builder.toString(); } - private static String namespaceOf(ClassForDisplay clazz) { + private static String namespaceOf(Class clazz) { return clazz.getName(); } /** - * Utility to build up display metadata from a component and its included + * Utility to build up display data from a component and its included * subcomponents. */ public interface Builder { /** - * Register display metadata from the specified subcomponent. - * - * @see #include(HasDisplayData, String) - */ - Builder include(HasDisplayData subComponent); - - /** - * Register display metadata from the specified subcomponent, using the specified namespace. - * - * @see #include(HasDisplayData, String) - */ - Builder include(HasDisplayData subComponent, Class namespace); - - /** - * Register display metadata from the specified subcomponent, using the specified namespace. - * - * @see #include(HasDisplayData, String) - */ - Builder include(HasDisplayData subComponent, ClassForDisplay namespace); - - /** - * Register display metadata from the specified subcomponent, using the specified namespace. - * - *

For example, a {@link ParDo} transform includes display metadata from the encapsulated - * {@link DoFn}. - */ - Builder include(HasDisplayData subComponent, String namespace); - - /** - * Register the given string display metadata. The metadata item will be registered with type - * {@link DisplayData.Type#STRING}, and is identified by the specified key and namespace from - * the current transform or component. - */ - ItemBuilder add(String key, String value); - - /** - * Register the given string display data if the value is not null. - * - * @see DisplayData.Builder#add(String, String) - */ - ItemBuilder addIfNotNull(String key, @Nullable String value); - - /** - * Register the given string display data if the value is different than the specified default. - * - * @see DisplayData.Builder#add(String, String) - */ - ItemBuilder addIfNotDefault(String key, @Nullable String value, @Nullable String defaultValue); - - /** - * Register the given numeric display metadata. The metadata item will be registered with type - * {@link DisplayData.Type#INTEGER}, and is identified by the specified key and namespace from - * the current transform or component. - */ - ItemBuilder add(String key, long value); - - /** - * Register the given numeric display data if the value is not null. - * - * @see DisplayData.Builder#add(String, long) - */ - ItemBuilder addIfNotNull(String key, @Nullable Long value); - - /** - * Register the given numeric display data if the value is different than the specified default. - * - * @see DisplayData.Builder#add(String, long) - */ - ItemBuilder addIfNotDefault(String key, long value, long defaultValue); - - /** - * Register the given floating point display metadata. The metadata item will be registered with - * type {@link DisplayData.Type#FLOAT}, and is identified by the specified key and namespace - * from the current transform or component. - */ - ItemBuilder add(String key, double value); - - /** - * Register the given floating point display data if the value is not null. - * - * @see DisplayData.Builder#add(String, double) - */ - ItemBuilder addIfNotNull(String key, @Nullable Double value); - - /** - * Register the given floating point display data if the value is different than the specified - * default. - * - * @see DisplayData.Builder#add(String, double) - */ - ItemBuilder addIfNotDefault(String key, double value, double defaultValue); - - /** - * Register the given boolean display metadata. The metadata item will be registered with - * type {@link DisplayData.Type#BOOLEAN}, and is identified by the specified key and namespace - * from the current transform or component. - */ - ItemBuilder add(String key, boolean value); - - /** - * Register the given boolean display data if the value is not null. - * - * @see DisplayData.Builder#add(String, boolean) - */ - ItemBuilder addIfNotNull(String key, @Nullable Boolean value); - - /** - * Register the given boolean display data if the value is different than the specified default. - * - * @see DisplayData.Builder#add(String, boolean) - */ - ItemBuilder addIfNotDefault(String key, boolean value, boolean defaultValue); - - /** - * Register the given timestamp display metadata. The metadata item will be registered with type - * {@link DisplayData.Type#TIMESTAMP}, and is identified by the specified key and namespace from - * the current transform or component. - */ - ItemBuilder add(String key, Instant value); - - /** - * Register the given timestamp display data if the value is not null. - * - * @see DisplayData.Builder#add(String, Instant) - */ - ItemBuilder addIfNotNull(String key, @Nullable Instant value); - - /** - * Register the given timestamp display data if the value is different than the specified - * default. - * - * @see DisplayData.Builder#add(String, Instant) - */ - ItemBuilder addIfNotDefault( - String key, @Nullable Instant value, @Nullable Instant defaultValue); - - /** - * Register the given duration display metadata. The metadata item will be registered with type - * {@link DisplayData.Type#DURATION}, and is identified by the specified key and namespace from - * the current transform or component. - */ - ItemBuilder add(String key, Duration value); - - /** - * Register the given duration display data if the value is not null. - * - * @see DisplayData.Builder#add(String, Duration) - */ - ItemBuilder addIfNotNull(String key, @Nullable Duration value); - - /** - * Register the given duration display data if the value is different than the specified - * default. + * Register display data from the specified subcomponent. For example, a {@link PTransform} + * which delegates to a user-provided function can implement {@link HasDisplayData} on the + * function and include it from the {@link PTransform}: * - * @see DisplayData.Builder#add(String, Duration) - */ - ItemBuilder addIfNotDefault( - String key, @Nullable Duration value, @Nullable Duration defaultValue); - - /** - * Register the given class display metadata. The metadata item will be registered with type - * {@link DisplayData.Type#JAVA_CLASS}, and is identified by the specified key and namespace - * from the current transform or component. - */ - ItemBuilder add(String key, Class value); - - /** - * Register the given class display metadata. The metadata item will be registered with type - * {@link DisplayData.Type#JAVA_CLASS}, and is identified by the specified key and namespace - * from the current transform or component. - */ - ItemBuilder add(String key, ClassForDisplay value); - - /** - * Register the given class display data if the value is not null. + *

{@literal @Override}
+     * public void populateDisplayData(DisplayData.Builder builder) {
+     *   super.populateDisplayData(builder);
      *
-     * @see DisplayData.Builder#add(String, Class)
-     */
-    ItemBuilder addIfNotNull(String key, @Nullable Class value);
-
-    /**
-     * Register the given class display data if the value is not null.
+     *   builder
+     *     .add(DisplayData.item("userFn", userFn)) // To register the class name of the userFn
+     *     .include(userFn); // To allow the userFn to register additional display data
+     * }
+     * 
* - * @see DisplayData.Builder#add(String, ClassForDisplay) - */ - ItemBuilder addIfNotNull(String key, @Nullable ClassForDisplay value); - - /** - * Register the given class display data if the value is different than the specified default. + * Using {@code include(subcomponent)} will associate each of the registered items with the + * namespace of the {@code subcomponent} being registered. To register display data in the + * current namespace, such as from a base class implementation, use + * {@code subcomponent.populateDisplayData(builder)} instead. * - * @see DisplayData.Builder#add(String, Class) + * @see HasDisplayData#populateDisplayData(DisplayData.Builder) */ - ItemBuilder addIfNotDefault( - String key, @Nullable Class value, @Nullable Class defaultValue); + Builder include(HasDisplayData subComponent); /** - * Register the given class display data if the value is different than the specified default. + * Register display data from the specified subcomponent, overriding the namespace of + * subcomponent display items with the specified namespace. * - * @see DisplayData.Builder#add(String, ClassForDisplay) + * @see #include(HasDisplayData) */ - ItemBuilder addIfNotDefault( - String key, @Nullable ClassForDisplay value, @Nullable ClassForDisplay defaultValue); - /** - * Register the given display metadata with the specified type. - * - *

The added display data is identified by the specified key and namespace from the current - * transform or component. - * - * @throws ClassCastException if the value cannot be safely cast to the specified type. - * - * @see DisplayData#inferType(Object) - */ - ItemBuilder add(String key, Type type, Object value); - } + Builder include(HasDisplayData subComponent, Class namespace); - /** - * Utility to append optional fields to display metadata, or register additional display metadata - * items. - */ - public interface ItemBuilder extends Builder { /** - * Add a human-readable label to describe the most-recently added metadata field. - * A label is optional; if unspecified, UIs should display the metadata key to identify the - * display item. + * Register display data from the specified subcomponent, overriding the namespace of + * subcomponent display items with the specified namespace. * - *

Specifying a null value will clear the label if it was previously defined. + * @see #include(HasDisplayData) */ - ItemBuilder withLabel(@Nullable String label); + Builder include(HasDisplayData subComponent, String namespace); /** - * Add a link URL to the most-recently added display metadata. A link URL is optional and - * can be provided to point the reader to additional details about the metadata. - * - *

Specifying a null value will clear the URL if it was previously defined. + * Register the given display item. */ - ItemBuilder withLinkUrl(@Nullable String url); + Builder add(Item item); /** - * Adds an explicit namespace to the most-recently added display metadata. The namespace - * and key uniquely identify the display metadata. - * - *

Leaving the namespace unspecified will default to the registering instance's class. + * Register the given display item if the value is not null. */ - ItemBuilder withNamespace(Class namespace); + Builder addIfNotNull(Item item); /** - * Adds an explicit namespace to the most-recently added display metadata. The namespace - * and key uniquely identify the display metadata. - * - *

Leaving the namespace unspecified will default to the registering instance's class. + * Register the given display item if the value is different than the specified default. */ - ItemBuilder withNamespace(ClassForDisplay namespace); + Builder addIfNotDefault(Item item, @Nullable T defaultValue); } /** - * A display metadata item. DisplayData items are registered via {@link Builder#add} within - * {@link HasDisplayData#populateDisplayData} implementations. Each metadata item is uniquely - * identified by the specified key and namespace generated from the registering component's - * class name. + * {@link Item Items} are the unit of display data. Each item is identified by a given key + * and namespace from the component the display item belongs to. + * + *

{@link Item Items} are registered via {@link DisplayData.Builder#add} + * within {@link HasDisplayData#populateDisplayData} implementations. */ - public static class Item { + public static final class Item implements Serializable { + @Nullable private final String namespace; private final String key; - private final String ns; private final Type type; - private final Object value; - private final Object shortValue; - private final String label; - private final String url; - - private static Item create(String nsClass, String key, Type type, Object value) { - FormattedItemValue formatted = type.format(value); - return new Item( - nsClass, key, type, formatted.getLongValue(), formatted.getShortValue(), null, null); - } - - private Item( - String namespace, - String key, - Type type, - Object value, - Object shortValue, - String url, - String label) { - this.ns = namespace; - this.key = key; - this.type = type; - this.value = value; - this.shortValue = shortValue; - this.url = url; - this.label = label; - } + @Nullable private final Object value; + @Nullable private final Object shortValue; + @Nullable private final String label; + @Nullable private final String linkUrl; + /** + * The namespace for the display item. The namespace defaults to the component which + * the display item belongs to. + */ + @Nullable @JsonGetter("namespace") public String getNamespace() { - return ns; + return namespace; } + /** + * The key for the display item. Each display item is created with a key and value + * via {@link DisplayData#item). + */ @JsonGetter("key") public String getKey() { return key; } /** - * Retrieve the {@link DisplayData.Type} of display metadata. All metadata conforms to a + * Retrieve the {@link DisplayData.Type} of display data. All metadata conforms to a * predefined set of allowed types. */ @JsonGetter("type") public Type getType() { - return type; - } + return type; + } /** - * Retrieve the value of the metadata item. + * Retrieve the value of the display item. The value is translated from the input to + * {@link DisplayData#item} into a format suitable for display. Translation is based on the + * item's {@link #getType() type}. + * + *

The value will only be {@literal null} if the input value during creation was null. */ @JsonGetter("value") + @Nullable public Object getValue() { return value; } /** - * Return the optional short value for an item. Types may provide a short-value to displayed - * instead of or in addition to the full {@link Item#value}. + * Return the optional short value for an item, or null if none is provided. + * + *

The short value is an alternative display representation for items having a long display + * value. For example, the {@link #getValue() value} for {@link Type#JAVA_CLASS} items contains + * the full class name with package, while the short value contains just the class name. * - *

Some display data types will not provide a short value, in which case the return value - * will be null. + * A {@link #getValue() value} will be provided for each display item, and some types may also + * provide a short-value. If a short value is provided, display data consumers may + * choose to display it instead of or in addition to the {@link #getValue() value}. */ @JsonGetter("shortValue") @JsonInclude(JsonInclude.Include.NON_NULL) - @Nullable public Object getShortValue() { return shortValue; } @@ -499,7 +298,7 @@ public String getLabel() { /** * Retrieve the optional link URL for an item. The URL points to an address where the reader - * can find additional context for the display metadata. + * can find additional context for the display data. * *

If no URL was specified, this will return {@code null}. */ @@ -507,63 +306,129 @@ public String getLabel() { @JsonInclude(JsonInclude.Include.NON_NULL) @Nullable public String getLinkUrl() { - return url; + return linkUrl; } - @Override - public String toString() { - return String.format("%s:%s=%s", ns, key, value); + private static Item create(String key, Type type, @Nullable T value) { + FormattedItemValue formatted = type.safeFormat(value); + return new Item<>( + null, key, type, formatted.getLongValue(), formatted.getShortValue(), null, null); } - @Override - public boolean equals(Object obj) { - if (obj instanceof Item) { - Item that = (Item) obj; - return Objects.equals(this.ns, that.ns) - && Objects.equals(this.key, that.key) - && Objects.equals(this.type, that.type) - && Objects.equals(this.value, that.value) - && Objects.equals(this.shortValue, that.shortValue) - && Objects.equals(this.label, that.label) - && Objects.equals(this.url, that.url); - } + /** + * Set the item {@link Item#getNamespace() namespace} from the given {@link Class}. + * + *

This method does not alter the current instance, but instead returns a new {@link Item} + * with the namespace set. + */ + public Item withNamespace(Class namespace) { + checkNotNull(namespace, "namespace argument cannot be null"); + return withNamespace(namespaceOf(namespace)); + } - return false; + /** @see #withNamespace(Class) */ + public Item withNamespace(String namespace) { + checkNotNull(namespace, "namespace argument cannot be null"); + return new Item<>( + namespace, getKey(), getType(), getValue(), getShortValue(), getLabel(), getLinkUrl()); + } + + /** + * Set the item {@link Item#getLabel() label}. + * + *

Specifying a null value will clear the label if it was previously defined. + * + *

This method does not alter the current instance, but instead returns a new {@link Item} + * with the label set. + */ + public Item withLabel(String label) { + return new Item<>( + getNamespace(), getKey(), getType(), getValue(), getShortValue(), label, getLinkUrl()); + } + + /** + * Set the item {@link Item#getLinkUrl() link url}. + * + *

Specifying a null value will clear the link url if it was previously defined. + * + *

This method does not alter the current instance, but instead returns a new {@link Item} + * with the link url set. + */ + public Item withLinkUrl(String url) { + return new Item<>( + getNamespace(), getKey(), getType(), getValue(), getShortValue(), getLabel(), url); + } + + /** + * Creates a similar item to the current instance but with the specified value. + * + *

This should only be used internally. It is useful to compare the value of a + * {@link DisplayData.Item} to the value derived from a specified input. + */ + private Item withValue(Object value) { + FormattedItemValue formatted = getType().safeFormat(value); + return new Item<>(getNamespace(), getKey(), getType(), formatted.getLongValue(), + formatted.getShortValue(), getLabel(), getLinkUrl()); + } + + private Item( + @Nullable String namespace, + String key, + Type type, + @Nullable Object value, + @Nullable Object shortValue, + @Nullable String label, + @Nullable String linkUrl) { + this.namespace = namespace; + this.key = checkNotNull(key); + this.type = checkNotNull(type); + this.value = value; + this.shortValue = shortValue; + this.label = label; + this.linkUrl = linkUrl; } @Override public int hashCode() { return Objects.hash( - this.ns, - this.key, - this.type, - this.value, - this.shortValue, - this.label, - this.url); + namespace, + key, + type, + value, + shortValue, + label, + linkUrl + ); } - private Item withLabel(String label) { - return new Item(this.ns, this.key, this.type, this.value, this.shortValue, this.url, label); - } + @Override + public boolean equals(Object obj) { + if (!(obj instanceof Item)) { + return false; + } - private Item withUrl(String url) { - return new Item(this.ns, this.key, this.type, this.value, this.shortValue, url, this.label); + Item that = (Item) obj; + return Objects.equals(this.namespace, that.namespace) + && Objects.equals(this.key, that.key) + && Objects.equals(this.type, that.type) + && Objects.equals(this.value, that.value) + && Objects.equals(this.shortValue, that.shortValue) + && Objects.equals(this.label, that.label) + && Objects.equals(this.linkUrl, that.linkUrl); } - private Item withNamespace(ClassForDisplay nsClass) { - String namespace = namespaceOf(nsClass); - return new Item( - namespace, this.key, this.type, this.value, this.shortValue, this.url, this.label); + @Override + public String toString() { + return String.format("%s:%s=%s", getNamespace(), getKey(), getValue()); } } /** - * Unique identifier for a display metadata item within a component. + * Unique identifier for a display data item within a component. * Identifiers are composed of the key they are registered with and a namespace generated from * the class of the component which registered the item. * - *

Display metadata registered with the same key from different components will have different + *

Display data registered with the same key from different components will have different * namespaces and thus will both be represented in the composed {@link DisplayData}. If a * single component registers multiple metadata items with the same key, only the most recent * item will be retained; previous versions are discarded. @@ -572,7 +437,7 @@ public static class Identifier { private final String ns; private final String key; - public static Identifier of(ClassForDisplay namespace, String key) { + public static Identifier of(Class namespace, String key) { return of(namespaceOf(namespace), key); } @@ -616,7 +481,7 @@ public String toString() { } /** - * Display metadata type. + * Display data type. */ public enum Type { STRING { @@ -665,12 +530,7 @@ FormattedItemValue format(Object value) { JAVA_CLASS { @Override FormattedItemValue format(Object value) { - if (value instanceof Class) { - ClassForDisplay classForDisplay = ClassForDisplay.of((Class) value); - return format(classForDisplay); - } - - ClassForDisplay clazz = checkType(value, ClassForDisplay.class, JAVA_CLASS); + Class clazz = checkType(value, Class.class, JAVA_CLASS); return new FormattedItemValue(clazz.getName(), clazz.getSimpleName()); } }; @@ -687,13 +547,27 @@ private static T checkType(Object value, Class clazz, DisplayData.Type ex } /** - * Format the display metadata value into a long string representation, and optionally + * Format the display data value into a long string representation, and optionally * a shorter representation for display. * *

Internal-only. Value objects can be safely cast to the expected Java type. */ abstract FormattedItemValue format(Object value); + /** + * Safe version of {@link Type#format(Object)}, which checks for null input value and if so + * returns a {@link FormattedItemValue} with null value properties. + * + * @see #format(Object) + */ + FormattedItemValue safeFormat(@Nullable Object value) { + if (value == null) { + return FormattedItemValue.NULL_VALUES; + } + + return format(value); + } + @Nullable private static Type tryInferFrom(@Nullable Object value) { if (value instanceof Integer || value instanceof Long) { @@ -706,7 +580,7 @@ private static Type tryInferFrom(@Nullable Object value) { return TIMESTAMP; } else if (value instanceof Duration) { return DURATION; - } else if (value instanceof Class || value instanceof ClassForDisplay) { + } else if (value instanceof Class) { return JAVA_CLASS; } else if (value instanceof String) { return STRING; @@ -717,6 +591,11 @@ private static Type tryInferFrom(@Nullable Object value) { } static class FormattedItemValue { + /** + * Default instance which contains null values. + */ + private static final FormattedItemValue NULL_VALUES = new FormattedItemValue(null); + private final Object shortValue; private final Object longValue; @@ -738,15 +617,12 @@ Object getShortValue() { } } - private static class InternalBuilder implements ItemBuilder { - private final Map entries; + private static class InternalBuilder implements Builder { + private final Map> entries; private final Set visited; private String latestNs; - @Nullable - private Item latestItem; - private InternalBuilder() { this.entries = Maps.newHashMap(); this.visited = Sets.newIdentityHashSet(); @@ -760,28 +636,21 @@ private static InternalBuilder forRoot(HasDisplayData instance) { @Override public Builder include(HasDisplayData subComponent) { - checkNotNull(subComponent); + checkNotNull(subComponent, "subComponent argument cannot be null"); return include(subComponent, subComponent.getClass()); } @Override public Builder include(HasDisplayData subComponent, Class namespace) { - checkNotNull(namespace); - return include(subComponent, ClassForDisplay.of(namespace)); - } - - @Override - public Builder include(HasDisplayData subComponent, ClassForDisplay namespace) { - checkNotNull(namespace); + checkNotNull(namespace, "Input namespace override cannot be null"); return include(subComponent, namespaceOf(namespace)); } @Override public Builder include(HasDisplayData subComponent, String namespace) { - checkNotNull(subComponent); - checkNotNull(namespace); + checkNotNull(subComponent, "subComponent argument cannot be null"); + checkNotNull(namespace, "Input namespace override cannot be null"); - commitLatest(); boolean newComponent = visited.add(subComponent); if (newComponent) { String prevNs = this.latestNs; @@ -794,202 +663,126 @@ public Builder include(HasDisplayData subComponent, String namespace) { } @Override - public ItemBuilder add(String key, String value) { - checkNotNull(value); - return addItemIf(true, key, Type.STRING, value); - } - - @Override - public ItemBuilder addIfNotNull(String key, @Nullable String value) { - return addItemIf(value != null, key, Type.STRING, value); - } - - @Override - public ItemBuilder addIfNotDefault( - String key, @Nullable String value, @Nullable String defaultValue) { - return addItemIf(!Objects.equals(value, defaultValue), key, Type.STRING, value); - } - - @Override - public ItemBuilder add(String key, long value) { - return addItemIf(true, key, Type.INTEGER, value); - } - - @Override - public ItemBuilder addIfNotNull(String key, @Nullable Long value) { - return addItemIf(value != null, key, Type.INTEGER, value); - } - - @Override - public ItemBuilder addIfNotDefault(String key, long value, long defaultValue) { - return addItemIf(!Objects.equals(value, defaultValue), key, Type.INTEGER, value); - } - - @Override - public ItemBuilder add(String key, double value) { - return addItemIf(true, key, Type.FLOAT, value); - } - - @Override - public ItemBuilder addIfNotNull(String key, @Nullable Double value) { - return addItemIf(value != null, key, Type.FLOAT, value); - } - - @Override - public ItemBuilder addIfNotDefault(String key, double value, double defaultValue) { - return addItemIf(!Objects.equals(value, defaultValue), key, Type.FLOAT, value); - } - - @Override - public ItemBuilder add(String key, boolean value) { - return addItemIf(true, key, Type.BOOLEAN, value); - } - - @Override - public ItemBuilder addIfNotNull(String key, @Nullable Boolean value) { - return addItemIf(value != null, key, Type.BOOLEAN, value); - } - - @Override - public ItemBuilder addIfNotDefault(String key, boolean value, boolean defaultValue) { - return addItemIf(!Objects.equals(value, defaultValue), key, Type.BOOLEAN, value); - } - - @Override - public ItemBuilder add(String key, Instant value) { - return addItemIf(true, key, Type.TIMESTAMP, value); - } - - @Override - public ItemBuilder addIfNotNull(String key, @Nullable Instant value) { - return addItemIf(value != null, key, Type.TIMESTAMP, value); - } - - @Override - public ItemBuilder addIfNotDefault( - String key, @Nullable Instant value, @Nullable Instant defaultValue) { - return addItemIf(!Objects.equals(value, defaultValue), key, Type.TIMESTAMP, value); + public Builder add(Item item) { + checkNotNull(item, "Input display item cannot be null"); + return addItemIf(true, item); } @Override - public ItemBuilder add(String key, Duration value) { - return addItemIf(true, key, Type.DURATION, value); + public Builder addIfNotNull(Item item) { + checkNotNull(item, "Input display item cannot be null"); + return addItemIf(item.getValue() != null, item); } @Override - public ItemBuilder addIfNotNull(String key, @Nullable Duration value) { - return addItemIf(value != null, key, Type.DURATION, value); + public Builder addIfNotDefault(Item item, @Nullable T defaultValue) { + checkNotNull(item, "Input display item cannot be null"); + Item defaultItem = item.withValue(defaultValue); + return addItemIf(!Objects.equals(item, defaultItem), item); } - @Override - public ItemBuilder addIfNotDefault( - String key, @Nullable Duration value, @Nullable Duration defaultValue) { - return addItemIf(!Objects.equals(value, defaultValue), key, Type.DURATION, value); - } - - @Override - public ItemBuilder add(String key, Class value) { - return addItemIf(true, key, Type.JAVA_CLASS, value); - } - - @Override - public ItemBuilder add(String key, ClassForDisplay value) { - checkNotNull(value); - return addItemIf(true, key, Type.JAVA_CLASS, value); - } - - @Override - public ItemBuilder addIfNotNull(String key, @Nullable Class value) { - return addItemIf(value != null, key, Type.JAVA_CLASS, value); - } - - @Override - public ItemBuilder addIfNotNull(String key, @Nullable ClassForDisplay value) { - return addItemIf(value != null, key, Type.JAVA_CLASS, value); - } + private Builder addItemIf(boolean condition, Item item) { + if (!condition) { + return this; + } - @Override - public ItemBuilder addIfNotDefault( - String key, @Nullable Class value, @Nullable Class defaultValue) { - return addItemIf(!Objects.equals(value, defaultValue), key, Type.JAVA_CLASS, value); - } + checkNotNull(item, "Input display item cannot be null"); + checkNotNull(item.getValue(), "Input display value cannot be null"); + if (item.getNamespace() == null) { + item = item.withNamespace(latestNs); + } - @Override - public ItemBuilder addIfNotDefault( - String key, @Nullable ClassForDisplay value, @Nullable ClassForDisplay defaultValue) { - return addItemIf(!Objects.equals(value, defaultValue), key, Type.JAVA_CLASS, value); - } + Identifier id = Identifier.of(item.getNamespace(), item.getKey()); + Preconditions.checkArgument(!entries.containsKey(id), + "Display data key (%s) is not unique within the specified namespace (%s).", + item.getKey(), item.getNamespace()); - @Override - public ItemBuilder add(String key, Type type, Object value) { - checkNotNull(type); - return addItemIf(true, key, type, value); + entries.put(id, item); + return this; } - private ItemBuilder addItemIf(boolean condition, String key, Type type, Object value) { - checkNotNull(key, "Display data keys cannot be null or empty."); - checkArgument(!key.isEmpty(), "Display data keys cannot be null or empty."); - commitLatest(); - - if (condition) { - checkNotNull(value, "Display data values cannot be null. Key: [%s]", key); - latestItem = Item.create(latestNs, key, type, value); - } - - return this; + private DisplayData build() { + return new DisplayData(this.entries); } + } - private void commitLatest() { - if (latestItem == null) { - return; - } + /** + * Create a display item for the specified key and string value. + */ + public static Item item(String key, @Nullable String value) { + return item(key, Type.STRING, value); + } - Identifier id = Identifier.of(latestItem.getNamespace(), latestItem.getKey()); - if (entries.containsKey(id)) { - throw new IllegalArgumentException("DisplayData key already exists. All display data " - + "for a component must be registered with a unique key.\nKey: " + id); - } + /** + * Create a display item for the specified key and integer value. + */ + public static Item item(String key, @Nullable Integer value) { + return item(key, Type.INTEGER, value); + } - entries.put(id, latestItem); - latestItem = null; - } + /** + * Create a display item for the specified key and integer value. + */ + public static Item item(String key, @Nullable Long value) { + return item(key, Type.INTEGER, value); + } - @Override - public ItemBuilder withLabel(@Nullable String label) { - if (latestItem != null) { - latestItem = latestItem.withLabel(label); - } + /** + * Create a display item for the specified key and floating point value. + */ + public static Item item(String key, @Nullable Float value) { + return item(key, Type.FLOAT, value); + } - return this; - } + /** + * Create a display item for the specified key and floating point value. + */ + public static Item item(String key, @Nullable Double value) { + return item(key, Type.FLOAT, value); + } - @Override - public ItemBuilder withLinkUrl(@Nullable String url) { - if (latestItem != null) { - latestItem = latestItem.withUrl(url); - } + /** + * Create a display item for the specified key and boolean value. + */ + public static Item item(String key, @Nullable Boolean value) { + return item(key, Type.BOOLEAN, value); + } - return this; - } + /** + * Create a display item for the specified key and timestamp value. + */ + public static Item item(String key, @Nullable Instant value) { + return item(key, Type.TIMESTAMP, value); + } - @Override - public ItemBuilder withNamespace(Class namespace) { - checkNotNull(namespace); - return withNamespace(ClassForDisplay.of(namespace)); - } + /** + * Create a display item for the specified key and duration value. + */ + public static Item item(String key, @Nullable Duration value) { + return item(key, Type.DURATION, value); + } - @Override - public ItemBuilder withNamespace(ClassForDisplay namespace) { - if (latestItem != null) { - latestItem = latestItem.withNamespace(namespace); - } + /** + * Create a display item for the specified key and class value. + */ + public static Item> item(String key, @Nullable Class value) { + return item(key, Type.JAVA_CLASS, value); + } - return this; - } + /** + * Create a display item for the specified key, type, and value. This method should be used + * if the type of the input value can only be determined at runtime. Otherwise, + * {@link HasDisplayData} implementors should call one of the typed factory methods, such as + * {@link #item(String, String)} or {@link #item(String, Integer)}. + * + * @throws ClassCastException if the value cannot be formatted as the given type. + * + * @see Type#inferType(Object) + */ + public static Item item(String key, Type type, @Nullable T value) { + checkNotNull(key, "key argument cannot be null"); + checkNotNull(type, "type argument cannot be null"); - private DisplayData build() { - commitLatest(); - return new DisplayData(this.entries); - } + return Item.create(key, type, value); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/HasDisplayData.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/HasDisplayData.java index 825b19267d..04727a1caf 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/HasDisplayData.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/HasDisplayData.java @@ -19,36 +19,48 @@ import com.google.cloud.dataflow.sdk.transforms.PTransform; /** - * Marker interface for {@link PTransform PTransforms} and components used within - * {@link PTransform PTransforms} to specify display metadata to be used within UIs and diagnostic - * tools. + * Marker interface for {@link PTransform PTransforms} and components to specify display data used + * within UIs and diagnostic tools. * - *

Display metadata is optional and may be collected during pipeline construction. It should - * only be used to informational purposes. Tools and components should not assume that display data + *

Display data is registered by overriding + * {@link #populateDisplayData(DisplayData.Builder)} in a component which implements + * {@code HasDisplayData}. Display data is available for {@link PipelineOptions} and + * {@link PTransform} implementations. + * + *

{@literal @Override}
+ * public void populateDisplayData(DisplayData.Builder builder) {
+ *  super.populateDisplayData(builder);
+ *
+ *  builder
+ *     .include(subComponent)
+ *     .add(DisplayData.item("minFilter", 42))
+ *     .addIfNotDefault(DisplayData.item("useTransactions", this.txn), false)
+ *     .add(DisplayData.item("topic", "projects/myproject/topics/mytopic")
+ *       .withLabel("Pub/Sub Topic"))
+ *     .add(DisplayData.item("serviceInstance", "myservice.com/fizzbang")
+ *       .withLinkUrl("http://www.myservice.com/fizzbang"));
+ * }
+ * 
+ * + *

Display data is optional and may be collected during pipeline construction. It should + * only be used for informational purposes. Tools and components should not assume that display data * will always be collected, or that collected display data will always be displayed. + * + * @see #populateDisplayData(DisplayData.Builder) */ public interface HasDisplayData { /** - * Register display metadata for the given transform or component. Metadata can be registered - * directly on the provided builder, as well as via included sub-components. + * Register display data for the given transform or component. + * + *

{@code populateDisplayData(DisplayData.Builder)} is invoked by Pipeline runners to collect + * display data via {@link DisplayData#from(HasDisplayData)}. Implementations may call + * {@code super.populateDisplayData(builder)} in order to register display data in the current + * namespace, but should otherwise use {@code subcomponent.populateDisplayData(builder)} to use + * the namespace of the subcomponent. * - *

-   * {@code
-   * @Override
-   * public void populateDisplayData(DisplayData.Builder builder) {
-   *  builder
-   *     .include(subComponent)
-   *     .add("minFilter", 42)
-   *     .addIfNotDefault("useTransactions", this.txn, false)
-   *     .add("topic", "projects/myproject/topics/mytopic")
-   *       .withLabel("Pub/Sub Topic")
-   *     .add("serviceInstance", "myservice.com/fizzbang")
-   *       .withLinkUrl("http://www.myservice.com/fizzbang");
-   * }
-   * }
-   * 
+ * @param builder The builder to populate with display data. * - * @param builder The builder to populate with display metadata. + * @see HasDisplayData */ void populateDisplayData(DisplayData.Builder builder); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindows.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindows.java index 42c97b8370..a0e8357066 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindows.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindows.java @@ -146,9 +146,14 @@ public boolean isCompatible(WindowFn other) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder - .add("numDays", number) - .addIfNotDefault("startDate", new DateTime(startDate, timeZone).toInstant(), + .add(DisplayData.item("numDays", number) + .withLabel("Windows Days")) + .addIfNotDefault( + DisplayData.item("startDate", new DateTime(startDate, timeZone).toInstant()) + .withLabel("Window Start Date"), new DateTime(DEFAULT_START_DATE, DateTimeZone.UTC).toInstant()); } @@ -241,9 +246,14 @@ public boolean isCompatible(WindowFn other) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder - .add("numMonths", number) - .addIfNotDefault("startDate", new DateTime(startDate, timeZone).toInstant(), + .add(DisplayData.item("numMonths", number) + .withLabel("Window Months")) + .addIfNotDefault( + DisplayData.item("startDate", new DateTime(startDate, timeZone).toInstant()) + .withLabel("Window Start Date"), new DateTime(DEFAULT_START_DATE, DateTimeZone.UTC).toInstant()); } @@ -345,9 +355,14 @@ public boolean isCompatible(WindowFn other) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder - .add("numYears", number) - .addIfNotDefault("startDate", new DateTime(startDate, timeZone).toInstant(), + .add(DisplayData.item("numYears", number) + .withLabel("Window Years")) + .addIfNotDefault( + DisplayData.item("startDate", new DateTime(startDate, timeZone).toInstant()) + .withLabel("Window Start Date"), new DateTime(DEFAULT_START_DATE, DateTimeZone.UTC).toInstant()); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindows.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindows.java index 67d226102f..2218628f25 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindows.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindows.java @@ -84,9 +84,12 @@ public IntervalWindow assignWindow(Instant timestamp) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder - .add("size", size) - .addIfNotDefault("offset", offset, Duration.ZERO); + .add(DisplayData.item("size", size) + .withLabel("Window Duration")) + .addIfNotDefault(DisplayData.item("offset", offset) + .withLabel("Window Start Offset"), Duration.ZERO); } @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Sessions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Sessions.java index 0915ae7ea4..8ffe82fc42 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Sessions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Sessions.java @@ -99,7 +99,9 @@ public Duration getGapDuration() { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("gapDuration", gapDuration); + super.populateDisplayData(builder); + builder.add(DisplayData.item("gapDuration", gapDuration) + .withLabel("Session Gap Duration")); } @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java index ddb453b133..e2a9f59ba3 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java @@ -141,10 +141,14 @@ public boolean isCompatible(WindowFn other) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder - .add("size", size) - .add("period", period) - .add("offset", offset); + .add(DisplayData.item("size", size) + .withLabel("Window Size")) + .add(DisplayData.item("period", period) + .withLabel("Window Period")) + .add(DisplayData.item("offset", offset) + .withLabel("Window Start Offset")); } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java index b22457800a..60b9e5d904 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java @@ -606,25 +606,39 @@ private PCollection assignWindows( @Override public void populateDisplayData(DisplayData.Builder builder) { - builder - .add("windowFn", windowFn.getClass()) - .include(windowFn) - .addIfNotNull("allowedLateness", allowedLateness); + super.populateDisplayData(builder); + + if (windowFn != null) { + builder + .add(DisplayData.item("windowFn", windowFn.getClass()) + .withLabel("Windowing Function")) + .include(windowFn); + } + + if (allowedLateness != null) { + builder.addIfNotDefault(DisplayData.item("allowedLateness", allowedLateness) + .withLabel("Allowed Lateness"), + Duration.millis(BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis())); + } if (trigger != null && !(trigger instanceof DefaultTrigger)) { - builder.add("trigger", trigger.toString()); + builder.add(DisplayData.item("trigger", trigger.toString()) + .withLabel("Trigger")); } if (mode != null) { - builder.add("accumulationMode", mode.toString()); + builder.add(DisplayData.item("accumulationMode", mode.toString()) + .withLabel("Accumulation Mode")); } if (closingBehavior != null) { - builder.add("closingBehavior", closingBehavior.toString()); + builder.add(DisplayData.item("closingBehavior", closingBehavior.toString()) + .withLabel("Window Closing Behavior")); } if (outputTimeFn != null) { - builder.add("outputTimeFn", outputTimeFn.getClass()); + builder.add(DisplayData.item("outputTimeFn", outputTimeFn.getClass()) + .withLabel("Output Time Function")); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowFn.java index 3d43372103..fd9ee0df9c 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowFn.java @@ -182,7 +182,7 @@ public boolean assignsToSingleWindow() { * {@inheritDoc} * *

By default, does not register any display data. Implementors may override this method - * to provide their own display metadata. + * to provide their own display data. */ @Override public void populateDisplayData(DisplayData.Builder builder) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CombineFnUtil.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CombineFnUtil.java index b895fa35e0..15c6272e21 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CombineFnUtil.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CombineFnUtil.java @@ -102,6 +102,7 @@ public Coder getDefaultOutputCoder( } @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); combineFn.populateDisplayData(builder); } }; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/AvroIOTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/AvroIOTest.java index 509e464436..9c155e2515 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/AvroIOTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/AvroIOTest.java @@ -18,6 +18,7 @@ import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.hasItem; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; @@ -30,13 +31,16 @@ import com.google.cloud.dataflow.sdk.testing.DataflowAssert; import com.google.cloud.dataflow.sdk.testing.TestPipeline; import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.display.DataflowDisplayDataEvaluator; import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayDataEvaluator; import com.google.cloud.dataflow.sdk.util.IOChannelUtils; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.common.base.MoreObjects; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterators; +import org.apache.avro.Schema; import org.apache.avro.file.DataFileReader; import org.apache.avro.generic.GenericRecord; import org.apache.avro.reflect.Nullable; @@ -51,6 +55,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Objects; +import java.util.Set; /** * Tests for AvroIO Read and Write transforms. @@ -287,4 +292,31 @@ public void testWriteDisplayData() { assertThat(displayData, hasDisplayItem("numShards", 100)); assertThat(displayData, hasDisplayItem("validation", false)); } + + @Test + public void testPrimitiveWriteDisplayData() { + DisplayDataEvaluator evaluator = DataflowDisplayDataEvaluator.create(); + + AvroIO.Write.Bound write = AvroIO.Write + .to("foo") + .withSchema(Schema.create(Schema.Type.STRING)) + .withoutValidation(); + + Set displayData = evaluator.displayDataForPrimitiveTransforms(write); + assertThat("AvroIO.Write should include the file pattern in its primitive transform", + displayData, hasItem(hasDisplayItem("fileNamePattern"))); + } + + @Test + public void testPrimitiveReadDisplayData() { + DisplayDataEvaluator evaluator = DataflowDisplayDataEvaluator.create(); + + AvroIO.Read.Bound read = AvroIO.Read.from("foo.*") + .withSchema(Schema.create(Schema.Type.STRING)) + .withoutValidation(); + + Set displayData = evaluator.displayDataForPrimitiveTransforms(read); + assertThat("AvroIO.Read should include the file pattern in its primitive transform", + displayData, hasItem(hasDisplayItem("filePattern"))); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/BigQueryIOTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/BigQueryIOTest.java index 54066ff7da..f7530e99e0 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/BigQueryIOTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/BigQueryIOTest.java @@ -19,8 +19,8 @@ import static com.google.cloud.dataflow.sdk.io.BigQueryIO.fromJsonString; import static com.google.cloud.dataflow.sdk.io.BigQueryIO.toJsonString; import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; -import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasKey; +import static org.hamcrest.Matchers.hasItem; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; @@ -57,6 +57,7 @@ import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.CreateDisposition; import com.google.cloud.dataflow.sdk.io.BigQueryIO.Write.WriteDisposition; import com.google.cloud.dataflow.sdk.options.BigQueryOptions; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.testing.DataflowAssert; @@ -69,7 +70,9 @@ import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.sdk.transforms.display.DataflowDisplayDataEvaluator; import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayDataEvaluator; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.util.BigQueryServices; import com.google.cloud.dataflow.sdk.util.BigQueryServices.DatasetService; @@ -104,6 +107,7 @@ import java.util.List; import java.util.Map; import java.util.NoSuchElementException; +import java.util.Set; import javax.annotation.Nullable; @@ -583,6 +587,30 @@ public void testBuildSourceDisplayData() { assertThat(displayData, hasDisplayItem("validation", false)); } + @Test + public void testTableSourcePrimitiveDisplayData() { + DisplayDataEvaluator evaluator = DataflowDisplayDataEvaluator.create(); + BigQueryIO.Read.Bound read = BigQueryIO.Read + .from("project:dataset.tableId") + .withoutValidation(); + + Set displayData = evaluator.displayDataForPrimitiveTransforms(read); + assertThat("BigQueryIO.Read should include the table spec in its primitive display data", + displayData, hasItem(hasDisplayItem("table"))); + } + + @Test + public void testQuerySourcePrimitiveDisplayData() { + DisplayDataEvaluator evaluator = DataflowDisplayDataEvaluator.create(); + BigQueryIO.Read.Bound read = BigQueryIO.Read + .fromQuery("foobar") + .withoutValidation(); + + Set displayData = evaluator.displayDataForPrimitiveTransforms(read); + assertThat("BigQueryIO.Read should include the query in its primitive display data", + displayData, hasItem(hasDisplayItem("query"))); + } + @Test public void testBuildSink() { BigQueryIO.Write.Bound bound = BigQueryIO.Write.named("WriteMyTable") @@ -709,8 +737,8 @@ public void testBuildSinkDisplayData() { DisplayData displayData = DisplayData.from(write); - assertThat(displayData, hasDisplayItem(hasKey("table"))); - assertThat(displayData, hasDisplayItem(hasKey("schema"))); + assertThat(displayData, hasDisplayItem("table")); + assertThat(displayData, hasDisplayItem("schema")); assertThat(displayData, hasDisplayItem("createDisposition", CreateDisposition.CREATE_IF_NEEDED.toString())); assertThat(displayData, @@ -718,6 +746,36 @@ public void testBuildSinkDisplayData() { assertThat(displayData, hasDisplayItem("validation", false)); } + @Test + public void testBatchSinkPrimitiveDisplayData() { + DataflowPipelineOptions options = DataflowDisplayDataEvaluator.getDefaultOptions(); + options.setStreaming(false); + testSinkPrimitiveDisplayData(options); + } + + @Test + public void testStreamingSinkPrimitiveDisplayData() { + DataflowPipelineOptions options = DataflowDisplayDataEvaluator.getDefaultOptions(); + options.setStreaming(true); + testSinkPrimitiveDisplayData(options); + } + + private void testSinkPrimitiveDisplayData(DataflowPipelineOptions options) { + DisplayDataEvaluator evaluator = DataflowDisplayDataEvaluator.create(options); + + BigQueryIO.Write.Bound write = BigQueryIO.Write + .to("project:dataset.table") + .withSchema(new TableSchema().set("col1", "type1").set("col2", "type2")) + .withoutValidation(); + + Set displayData = evaluator.displayDataForPrimitiveTransforms(write); + assertThat("BigQueryIO.Write should include the table spec in its primitive display data", + displayData, hasItem(hasDisplayItem("table"))); + + assertThat("BigQueryIO.Write should include the table schema in its primitive display data", + displayData, hasItem(hasDisplayItem("schema"))); + } + private void testWriteValidatesDataset(boolean streaming) { BigQueryOptions options = PipelineOptionsFactory.as(BigQueryOptions.class); options.setProject("someproject"); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/BoundedReadFromUnboundedSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/BoundedReadFromUnboundedSourceTest.java index 963ceb9044..f604c0bd9e 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/BoundedReadFromUnboundedSourceTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/BoundedReadFromUnboundedSourceTest.java @@ -16,7 +16,7 @@ package com.google.cloud.dataflow.sdk.io; -import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includes; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertEquals; @@ -74,12 +74,12 @@ public void testForwardsDisplayData() { TestCountingSource src = new TestCountingSource(1234) { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("foo", "bar"); + builder.add(DisplayData.item("foo", "bar")); } }; BoundedReadFromUnboundedSource> read = Read.from(src).withMaxNumRecords(5); - assertThat(DisplayData.from(read), includes(src)); + assertThat(DisplayData.from(read), includesDisplayDataFrom(src)); } private static class Checker diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/CompressedSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/CompressedSourceTest.java index fe0e59ad70..57a0f8aec3 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/CompressedSourceTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/CompressedSourceTest.java @@ -17,8 +17,7 @@ package com.google.cloud.dataflow.sdk.io; import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; -import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasKey; -import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includes; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.instanceOf; @@ -341,7 +340,7 @@ public void testDisplayData() { ByteSource inputSource = new ByteSource("foobar.txt", 1) { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("foo", "bar"); + builder.add(DisplayData.item("foo", "bar")); } }; @@ -351,10 +350,10 @@ public void populateDisplayData(DisplayData.Builder builder) { DisplayData compressedSourceDisplayData = DisplayData.from(compressedSource); DisplayData gzipDisplayData = DisplayData.from(gzipSource); - assertThat(compressedSourceDisplayData, hasDisplayItem(hasKey("compressionMode"))); + assertThat(compressedSourceDisplayData, hasDisplayItem("compressionMode")); assertThat(gzipDisplayData, hasDisplayItem("compressionMode", CompressionMode.GZIP.toString())); assertThat(compressedSourceDisplayData, hasDisplayItem("source", inputSource.getClass())); - assertThat(compressedSourceDisplayData, includes(inputSource)); + assertThat(compressedSourceDisplayData, includesDisplayDataFrom(inputSource)); } /** diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/DatastoreIOTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/DatastoreIOTest.java index edf7c243f1..f1ae3cc70f 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/DatastoreIOTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/DatastoreIOTest.java @@ -21,6 +21,7 @@ import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -36,6 +37,7 @@ import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; +import com.google.api.services.datastore.DatastoreV1; import com.google.api.services.datastore.DatastoreV1.Entity; import com.google.api.services.datastore.DatastoreV1.EntityResult; import com.google.api.services.datastore.DatastoreV1.Key; @@ -56,8 +58,13 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.testing.ExpectedLogs; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.display.DataflowDisplayDataEvaluator; import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayDataEvaluator; import com.google.cloud.dataflow.sdk.util.TestCredential; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PInput; import com.google.common.collect.Lists; import org.junit.Before; @@ -76,6 +83,7 @@ import java.util.Collections; import java.util.List; import java.util.NoSuchElementException; +import java.util.Set; import javax.annotation.Nullable; @@ -197,7 +205,7 @@ public void testSourceValidationSucceedsNamespace() throws Exception { } @Test - public void testSourceDipslayData() { + public void testSourceDisplayData() { DatastoreIO.Source source = DatastoreIO.source() .withDataset(DATASET) .withQuery(QUERY) @@ -212,6 +220,17 @@ public void testSourceDipslayData() { assertThat(displayData, hasDisplayItem("namespace", NAMESPACE)); } + @Test + public void testSourcePrimitiveDisplayData() { + DisplayDataEvaluator evaluator = DataflowDisplayDataEvaluator.create(); + PTransform read = DatastoreIO.readFrom( + "myDataset", DatastoreV1.Query.newBuilder().build()); + + Set displayData = evaluator.displayDataForPrimitiveTransforms(read); + assertThat("DatastoreIO read should include the dataset in its primitive display data", + displayData, hasItem(hasDisplayItem("dataset"))); + } + @Test public void testSinkDoesNotAllowNullHost() throws Exception { thrown.expect(NullPointerException.class); @@ -245,7 +264,7 @@ public void testSinkValidationSucceedsWithDataset() throws Exception { } @Test - public void testSinkDipslayData() { + public void testSinkDisplayData() { DatastoreIO.Sink sink = DatastoreIO.sink() .withDataset(DATASET) .withHost(HOST); @@ -256,6 +275,17 @@ public void testSinkDipslayData() { assertThat(displayData, hasDisplayItem("host", HOST)); } + @Test + public void testSinkPrimitiveDisplayData() { + DisplayDataEvaluator evaluator = DataflowDisplayDataEvaluator.create(); + PTransform, ?> write = DatastoreIO.writeTo("myDataset"); + + Set displayData = evaluator.displayDataForPrimitiveTransforms(write); + assertThat("DatastoreIO write should include the dataset in its primitive display data", + displayData, hasItem(hasDisplayItem("dataset"))); + } + + @Test public void testQuerySplitBasic() throws Exception { KindExpression mykind = KindExpression.newBuilder().setName("mykind").build(); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/PubsubIOTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/PubsubIOTest.java index 0eb6ab302b..dfe5da3457 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/PubsubIOTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/PubsubIOTest.java @@ -18,13 +18,16 @@ import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; +import static org.hamcrest.Matchers.hasItem; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; import com.google.api.client.testing.http.FixedClock; import com.google.api.client.util.Clock; import com.google.api.services.pubsub.model.PubsubMessage; +import com.google.cloud.dataflow.sdk.transforms.display.DataflowDisplayDataEvaluator; import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayDataEvaluator; import org.joda.time.Duration; import org.joda.time.Instant; @@ -35,6 +38,7 @@ import org.junit.runners.JUnit4; import java.util.HashMap; +import java.util.Set; import javax.annotation.Nullable; @@ -273,4 +277,25 @@ public void testWriteDisplayData() { assertThat(displayData, hasDisplayItem("timestampLabel", "myTimestamp")); assertThat(displayData, hasDisplayItem("idLabel", "myId")); } + + @Test + public void testPrimitiveWriteDisplayData() { + DisplayDataEvaluator evaluator = DataflowDisplayDataEvaluator.create(); + PubsubIO.Write.Bound write = PubsubIO.Write + .topic("projects/project/topics/topic"); + + Set displayData = evaluator.displayDataForPrimitiveTransforms(write); + assertThat("PubsubIO.Write should include the topic in its primitive display data", + displayData, hasItem(hasDisplayItem("topic"))); + } + + @Test + public void testPrimitiveReadDisplayData() { + DisplayDataEvaluator evaluator = DataflowDisplayDataEvaluator.create(); + PubsubIO.Read.Bound read = PubsubIO.Read.topic("projects/project/topics/topic"); + + Set displayData = evaluator.displayDataForPrimitiveTransforms(read); + assertThat("PubsubIO.Read should include the topic in its primitive display data", + displayData, hasItem(hasDisplayItem("topic"))); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/ReadTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/ReadTest.java index 530687be14..25afc07f00 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/ReadTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/ReadTest.java @@ -17,7 +17,7 @@ package com.google.cloud.dataflow.sdk.io; import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; -import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includes; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; import static org.hamcrest.MatcherAssert.assertThat; @@ -73,13 +73,13 @@ public void testDisplayData() { SerializableBoundedSource boundedSource = new SerializableBoundedSource() { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("foo", "bar"); + builder.add(DisplayData.item("foo", "bar")); } }; SerializableUnboundedSource unboundedSource = new SerializableUnboundedSource() { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("foo", "bar"); + builder.add(DisplayData.item("foo", "bar")); } }; Duration maxReadTime = Duration.standardMinutes(2345); @@ -91,11 +91,11 @@ public void populateDisplayData(DisplayData.Builder builder) { DisplayData boundedDisplayData = DisplayData.from(bounded); assertThat(boundedDisplayData, hasDisplayItem("source", boundedSource.getClass())); - assertThat(boundedDisplayData, includes(boundedSource)); + assertThat(boundedDisplayData, includesDisplayDataFrom(boundedSource)); DisplayData unboundedDisplayData = DisplayData.from(unbounded); assertThat(unboundedDisplayData, hasDisplayItem("source", unboundedSource.getClass())); - assertThat(unboundedDisplayData, includes(unboundedSource)); + assertThat(unboundedDisplayData, includesDisplayDataFrom(unboundedSource)); assertThat(unboundedDisplayData, hasDisplayItem("maxRecords", 1234)); assertThat(unboundedDisplayData, hasDisplayItem("maxReadTime", maxReadTime)); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java index d6ed939ba7..1d2fa06e96 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/TextIOTest.java @@ -21,8 +21,11 @@ import static com.google.cloud.dataflow.sdk.TestUtils.NO_INTS_ARRAY; import static com.google.cloud.dataflow.sdk.TestUtils.NO_LINES_ARRAY; import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasValue; import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.hasItem; +import static org.hamcrest.Matchers.startsWith; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; @@ -42,7 +45,10 @@ import com.google.cloud.dataflow.sdk.testing.TestPipeline; import com.google.cloud.dataflow.sdk.transforms.Create; import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.display.DataflowDisplayDataEvaluator; import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayDataEvaluator; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers; import com.google.cloud.dataflow.sdk.util.CoderUtils; import com.google.cloud.dataflow.sdk.util.GcsUtil; import com.google.cloud.dataflow.sdk.util.IOChannelUtils; @@ -76,6 +82,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Set; import java.util.zip.GZIPOutputStream; /** @@ -352,6 +359,31 @@ public void testWriteDisplayData() { assertThat(displayData, hasDisplayItem("validation", false)); } + @Test + public void testPrimitiveWriteDisplayData() { + DisplayDataEvaluator evaluator = DataflowDisplayDataEvaluator.create(); + + TextIO.Write.Bound write = TextIO.Write.to("foobar"); + + Set displayData = evaluator.displayDataForPrimitiveTransforms(write); + assertThat("TextIO.Write should include the file prefix in its primitive display data", + displayData, hasItem(hasDisplayItem(hasValue(startsWith("foobar"))))); + } + + @Test + public void testPrimitiveReadDisplayData() { + DisplayDataEvaluator evaluator = DataflowDisplayDataEvaluator.create(); + + TextIO.Read.Bound read = TextIO.Read + .from("foobar") + .withoutValidation(); + + Set displayData = evaluator.displayDataForPrimitiveTransforms(read); + assertThat("TextIO.Read should include the file prefix in its primitive display data", + displayData, + hasItem(hasDisplayItem(DisplayDataMatchers.hasValue(startsWith("foobar"))))); + } + @Test public void testUnsupportedFilePattern() throws IOException { File outFolder = tmpFolder.newFolder(); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/WriteTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/WriteTest.java index ca1c7df00d..d28b1adf93 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/WriteTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/WriteTest.java @@ -15,7 +15,7 @@ package com.google.cloud.dataflow.sdk.io; import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; -import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includes; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -160,14 +160,14 @@ public void testDisplayData() { TestSink sink = new TestSink() { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("foo", "bar"); + builder.add(DisplayData.item("foo", "bar")); } }; Write.Bound write = Write.to(sink); DisplayData displayData = DisplayData.from(write); assertThat(displayData, hasDisplayItem("sink", sink.getClass())); - assertThat(displayData, includes(sink)); + assertThat(displayData, includesDisplayDataFrom(sink)); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/bigtable/BigtableIOTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/bigtable/BigtableIOTest.java index 5878c6403a..ebe53c90b3 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/bigtable/BigtableIOTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/bigtable/BigtableIOTest.java @@ -20,11 +20,10 @@ import static com.google.cloud.dataflow.sdk.testing.SourceTestUtils.assertSplitAtFractionExhaustive; import static com.google.cloud.dataflow.sdk.testing.SourceTestUtils.assertSplitAtFractionFails; import static com.google.cloud.dataflow.sdk.testing.SourceTestUtils.assertSplitAtFractionSucceedsAndConsistent; - import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; -import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasKey; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Verify.verifyNotNull; + import static org.hamcrest.Matchers.hasSize; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; @@ -424,7 +423,7 @@ public void testReadingDisplayData() { assertThat(displayData, hasDisplayItem("rowFilter", rowFilter.toString())); // BigtableIO adds user-agent to options; assert only on key and not value. - assertThat(displayData, hasDisplayItem(hasKey("bigtableOptions"))); + assertThat(displayData, hasDisplayItem("bigtableOptions")); } /** Tests that a record gets written to the service and messages are logged. */ @@ -490,10 +489,7 @@ public void testWritingDisplayData() { .withBigtableOptions(BIGTABLE_OPTIONS); DisplayData displayData = DisplayData.from(write); - assertThat(displayData, hasDisplayItem("tableId", "fooTable")); - // BigtableIO adds user-agent to options; assert only on key and not value. - assertThat(displayData, hasDisplayItem(hasKey("bigtableOptions"))); } //////////////////////////////////////////////////////////////////////////////////////////// diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsReflectorTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsReflectorTest.java index 560fef1b38..cc0c861065 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsReflectorTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsReflectorTest.java @@ -17,12 +17,15 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.allOf; +import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.isOneOf; import static org.hamcrest.Matchers.not; import com.google.common.collect.ImmutableSet; + +import com.fasterxml.jackson.annotation.JsonIgnore; import org.hamcrest.FeatureMatcher; import org.hamcrest.Matcher; import org.hamcrest.Matchers; @@ -126,6 +129,24 @@ interface HiddenOptions extends PipelineOptions { void setFoo(String value); } + @Test + public void testShouldSerialize() { + Set properties = + PipelineOptionsReflector.getOptionSpecs(JsonIgnoreOptions.class); + + assertThat(properties, hasItem(allOf(hasName("notIgnored"), shouldSerialize()))); + assertThat(properties, hasItem(allOf(hasName("ignored"), not(shouldSerialize())))); + } + + interface JsonIgnoreOptions extends PipelineOptions { + String getNotIgnored(); + void setNotIgnored(String value); + + @JsonIgnore + String getIgnored(); + void setIgnored(String value); + } + @Test public void testMultipleInputInterfaces() { Set> interfaces = @@ -190,4 +211,15 @@ protected String featureValueOf(PipelineOptionSpec actual) { } }; } + + private static Matcher shouldSerialize() { + return new FeatureMatcher(equalTo(true), + "should serialize", "shouldSerialize") { + + @Override + protected Boolean featureValueOf(PipelineOptionSpec actual) { + return actual.shouldSerialize(); + } + }; + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java index 70e685d500..017bcf37e5 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java @@ -833,7 +833,7 @@ public void testDisplayDataExcludesDefaultValues() { PipelineOptions options = PipelineOptionsFactory.as(HasDefaults.class); DisplayData data = DisplayData.from(options); - assertThat(data, not(hasDisplayItem(hasKey("foo")))); + assertThat(data, not(hasDisplayItem("foo"))); } interface HasDefaults extends PipelineOptions { @@ -848,7 +848,7 @@ public void testDisplayDataExcludesValuesAccessedButNeverSet() { assertEquals("bar", options.getFoo()); DisplayData data = DisplayData.from(options); - assertThat(data, not(hasDisplayItem(hasKey("foo")))); + assertThat(data, not(hasDisplayItem("foo"))); } @Test @@ -950,4 +950,13 @@ private T serializeDeserialize(Class kls, Pipelin String value = MAPPER.writeValueAsString(options); return MAPPER.readValue(value, PipelineOptions.class).as(kls); } + + @Test + public void testDisplayDataExcludesJsonIgnoreOptions() { + IgnoredProperty options = PipelineOptionsFactory.as(IgnoredProperty.class); + options.setValue("foobar"); + + DisplayData data = DisplayData.from(options); + assertThat(data, not(hasDisplayItem("value"))); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java index 699f5022c2..d28bcdf815 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java @@ -20,8 +20,11 @@ import static com.google.cloud.dataflow.sdk.util.Structs.getDictionary; import static com.google.cloud.dataflow.sdk.util.Structs.getString; +import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.hasEntry; import static org.hamcrest.Matchers.hasKey; +import static org.hamcrest.Matchers.is; import static org.hamcrest.core.IsInstanceOf.instanceOf; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; @@ -49,6 +52,7 @@ import com.google.cloud.dataflow.sdk.options.DataflowPipelineWorkerPoolOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator.TranslationContext; +import com.google.cloud.dataflow.sdk.testing.ExpectedLogs; import com.google.cloud.dataflow.sdk.transforms.Count; import com.google.cloud.dataflow.sdk.transforms.Create; import com.google.cloud.dataflow.sdk.transforms.DoFn; @@ -101,6 +105,7 @@ public class DataflowPipelineTranslatorTest implements Serializable { @Rule public transient ExpectedException thrown = ExpectedException.none(); + @Rule public transient ExpectedLogs logs = ExpectedLogs.none(DataflowPipelineTranslator.class); // A Custom Mockito matcher for an initial Job that checks that all // expected fields are set. @@ -689,7 +694,7 @@ public void testBadWildcardRecursive() throws Exception { pipeline.apply(TextIO.Read.from("gs://bucket/foo**/baz")); // Check that translation does fail. - thrown.expectCause(Matchers.allOf( + thrown.expectCause(allOf( instanceOf(IllegalArgumentException.class), ThrowableMessageMatcher.hasMessage(containsString("Unsupported wildcard usage")))); t.translate(pipeline, pipeline.getRunner(), Collections.emptyList()); @@ -820,10 +825,10 @@ public void processElement(ProcessContext c) throws Exception { @Override public void populateDisplayData(DisplayData.Builder builder) { builder - .add("foo", "bar") - .add("foo2", DataflowPipelineTranslatorTest.class) - .withLabel("Test Class") - .withLinkUrl("http://www.google.com"); + .add(DisplayData.item("foo", "bar")) + .add(DisplayData.item("foo2", DataflowPipelineTranslatorTest.class) + .withLabel("Test Class") + .withLinkUrl("http://www.google.com")); } }; @@ -835,7 +840,7 @@ public void processElement(ProcessContext c) throws Exception { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("foo3", 1234); + builder.add(DisplayData.item("foo3", 1234)); } }; @@ -872,6 +877,7 @@ public void populateDisplayData(DisplayData.Builder builder) { .build(), ImmutableMap.builder() .put("key", "fn") + .put("label", "Transform Function") .put("type", "JAVA_CLASS") .put("value", fn1.getClass().getName()) .put("shortValue", fn1.getClass().getSimpleName()) @@ -891,6 +897,7 @@ public void populateDisplayData(DisplayData.Builder builder) { ImmutableSet> expectedFn2DisplayData = ImmutableSet.of( ImmutableMap.builder() .put("key", "fn") + .put("label", "Transform Function") .put("type", "JAVA_CLASS") .put("value", fn2.getClass().getName()) .put("shortValue", fn2.getClass().getSimpleName()) @@ -907,4 +914,62 @@ public void populateDisplayData(DisplayData.Builder builder) { assertEquals(expectedFn1DisplayData, ImmutableSet.copyOf(fn1displayData)); assertEquals(expectedFn2DisplayData, ImmutableSet.copyOf(fn2displayData)); } + + @Test + public void testCapturesDisplayDataExceptions() throws IOException { + DataflowPipelineOptions options = buildPipelineOptions(); + options.setRunner(DataflowPipelineRunner.class); + DataflowPipelineTranslator translator = DataflowPipelineTranslator.fromOptions(options); + Pipeline pipeline = Pipeline.create(options); + + final RuntimeException displayDataException = new RuntimeException("foobar"); + pipeline + .apply(Create.of(1, 2, 3)) + .apply(ParDo.of(new DoFn() { + @Override + public void processElement(ProcessContext c) throws Exception { + c.output(c.element()); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + throw displayDataException; + } + })); + + Job job = translator.translate( + pipeline, + (DataflowPipelineRunner) pipeline.getRunner(), + Collections.emptyList()).getJob(); + + String expectedMessage = "Display data will be not be available for this step"; + logs.verifyWarn(expectedMessage); + + List steps = job.getSteps(); + assertEquals("Job should have 2 steps", 2, steps.size()); + + @SuppressWarnings("unchecked") + Iterable> displayData = (Collection>) steps.get(1) + .getProperties().get("display_data"); + + String namespace = DataflowPipelineTranslator.DisplayDataException.class.getName(); + Assert.assertThat(displayData, Matchers.>hasItem(allOf( + hasEntry("namespace", namespace), + hasEntry("key", "exceptionType"), + hasEntry("value", RuntimeException.class.getName())))); + + Assert.assertThat(displayData, Matchers.>hasItem(allOf( + hasEntry("namespace", namespace), + hasEntry("key", "exceptionMessage"), + hasEntry(is("value"), Matchers.containsString(expectedMessage))))); + + Assert.assertThat(displayData, Matchers.>hasItem(allOf( + hasEntry("namespace", namespace), + hasEntry("key", "exceptionCause"), + hasEntry("value", "foobar")))); + + Assert.assertThat(displayData, Matchers.>hasItem(allOf( + hasEntry("namespace", namespace), + hasEntry("key", "stackTrace")))); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUniqueTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUniqueTest.java index 5f008079e9..4426db5dcc 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUniqueTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUniqueTest.java @@ -17,7 +17,6 @@ package com.google.cloud.dataflow.sdk.transforms; import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; -import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasKey; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; @@ -304,6 +303,6 @@ public void testDisplayData() { DisplayData maxErrorDisplayData = DisplayData.from(specifiedMaxError); assertThat(maxErrorDisplayData, hasDisplayItem("maximumEstimationError", 0.1234)); assertThat("calculated sampleSize should be included", maxErrorDisplayData, - hasDisplayItem(hasKey("sampleSize"))); + hasDisplayItem("sampleSize")); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineFnsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineFnsTest.java index aa6e035f6b..abffe4055e 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineFnsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineFnsTest.java @@ -16,7 +16,7 @@ package com.google.cloud.dataflow.sdk.transforms; import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; -import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includes; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; import static org.junit.Assert.assertThat; @@ -300,8 +300,8 @@ public String apply(String input) { assertThat(displayData, hasDisplayItem("combineFn2", combineFn2.getClass())); String nsBase = DisplayDataCombineFn.class.getName(); - assertThat(displayData, includes(combineFn1, nsBase + "#1")); - assertThat(displayData, includes(combineFn2, nsBase + "#2")); + assertThat(displayData, includesDisplayDataFrom(combineFn1, nsBase + "#1")); + assertThat(displayData, includesDisplayDataFrom(combineFn2, nsBase + "#2")); } private static class DisplayDataCombineFn extends Combine.CombineFn { @@ -337,8 +337,8 @@ public String extractOutput(String accumulator) { @Override public void populateDisplayData(DisplayData.Builder builder) { builder - .add("uniqueKey" + id, value) - .add("sharedKey", value); + .add(DisplayData.item("uniqueKey" + id, value)) + .add(DisplayData.item("sharedKey", value)); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java index 6dd838ad87..bd4cead138 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java @@ -18,8 +18,11 @@ import static com.google.cloud.dataflow.sdk.TestUtils.checkCombineFn; import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; -import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includes; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasNamespace; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; import static com.google.common.base.Preconditions.checkNotNull; + +import static org.hamcrest.Matchers.hasItem; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; @@ -36,6 +39,7 @@ import com.google.cloud.dataflow.sdk.coders.SerializableCoder; import com.google.cloud.dataflow.sdk.coders.StandardCoder; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.coders.VarIntCoder; import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; import com.google.cloud.dataflow.sdk.testing.DataflowAssert; import com.google.cloud.dataflow.sdk.testing.RunnableOnService; @@ -43,7 +47,9 @@ import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn; import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.Context; import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext; +import com.google.cloud.dataflow.sdk.transforms.display.DataflowDisplayDataEvaluator; import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayDataEvaluator; import com.google.cloud.dataflow.sdk.transforms.windowing.AfterPane; import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; @@ -58,6 +64,7 @@ import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PCollectionView; +import com.google.cloud.dataflow.sdk.values.POutput; import com.google.common.base.MoreObjects; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableSet; @@ -686,7 +693,7 @@ public void testDisplayData() { UniqueInts combineFn = new UniqueInts() { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("fnMetadata", "foobar"); + builder.add(DisplayData.item("fnMetadata", "foobar")); } }; Combine.Globally combine = Combine.globally(combineFn) @@ -696,7 +703,37 @@ public void populateDisplayData(DisplayData.Builder builder) { assertThat(displayData, hasDisplayItem("combineFn", combineFn.getClass())); assertThat(displayData, hasDisplayItem("emitDefaultOnEmptyInput", true)); assertThat(displayData, hasDisplayItem("fanout", 1234)); - assertThat(displayData, includes(combineFn)); + assertThat(displayData, includesDisplayDataFrom(combineFn)); + } + + @Test + public void testDisplayDataForWrappedFn() { + UniqueInts combineFn = new UniqueInts() { + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.add(DisplayData.item("foo", "bar")); + } + }; + Combine.PerKey combine = Combine.perKey(combineFn); + DisplayData displayData = DisplayData.from(combine); + + assertThat(displayData, hasDisplayItem("combineFn", combineFn.getClass())); + assertThat(displayData, hasDisplayItem(hasNamespace(combineFn.getClass()))); + } + + @Test + public void testCombinePerKeyPrimitiveDisplayData() { + DisplayDataEvaluator evaluator = DataflowDisplayDataEvaluator.create(); + + CombineTest.UniqueInts combineFn = new CombineTest.UniqueInts(); + PTransform>, ? extends POutput> combine = + Combine.perKey(combineFn); + + Set displayData = evaluator.displayDataForPrimitiveTransforms(combine, + KvCoder.of(VarIntCoder.of(), VarIntCoder.of())); + + assertThat("Combine.perKey should include the combineFn in its primitive transform", + displayData, hasItem(hasDisplayItem("combineFn", combineFn.getClass()))); } //////////////////////////////////////////////////////////////////////////// diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/IntraBundleParallelizationTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/IntraBundleParallelizationTest.java index 8a562c3eff..012ffe327d 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/IntraBundleParallelizationTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/IntraBundleParallelizationTest.java @@ -18,7 +18,7 @@ import static com.google.cloud.dataflow.sdk.testing.SystemNanoTimeSleeper.sleepMillis; import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; -import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includes; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; import static org.hamcrest.Matchers.both; import static org.hamcrest.Matchers.containsString; @@ -227,7 +227,7 @@ public void processElement(ProcessContext c) throws Exception { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("foo", "bar"); + builder.add(DisplayData.item("foo", "bar")); } }; @@ -236,7 +236,7 @@ public void populateDisplayData(DisplayData.Builder builder) { .of(fn); DisplayData displayData = DisplayData.from(transform); - assertThat(displayData, includes(fn)); + assertThat(displayData, includesDisplayDataFrom(fn)); assertThat(displayData, hasDisplayItem("fn", fn.getClass())); assertThat(displayData, hasDisplayItem("maxParallelism", 1234)); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/MapElementsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/MapElementsTest.java index be3e720646..aba79d9abb 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/MapElementsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/MapElementsTest.java @@ -16,12 +16,18 @@ package com.google.cloud.dataflow.sdk.transforms; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; + import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasItem; import static org.junit.Assert.assertThat; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.testing.DataflowAssert; import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.display.DataflowDisplayDataEvaluator; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayDataEvaluator; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.TypeDescriptor; @@ -33,6 +39,7 @@ import org.junit.runners.JUnit4; import java.io.Serializable; +import java.util.Set; /** * Tests for {@link MapElements}. @@ -102,7 +109,7 @@ public String apply(String input) { assertThat(pipeline.getCoderRegistry().getDefaultCoder(output.getTypeDescriptor()), equalTo(pipeline.getCoderRegistry().getDefaultCoder(new TypeDescriptor() {}))); - // Make sure the pipelien runs too + // Make sure the pipeline runs too pipeline.run(); } @@ -117,12 +124,58 @@ public void testVoidValues() throws Exception { pipeline.run(); } + @Test + public void testSerializableFunctionDisplayData() { + SerializableFunction serializableFn = + new SerializableFunction() { + @Override + public Integer apply(Integer input) { + return input; + } + }; + + MapElements serializableMap = MapElements.via(serializableFn) + .withOutputType(TypeDescriptor.of(Integer.class)); + assertThat(DisplayData.from(serializableMap), + hasDisplayItem("mapFn", serializableFn.getClass())); + } + + @Test + public void testSimpleFunctionDisplayData() { + SimpleFunction simpleFn = new SimpleFunction() { + @Override + public Integer apply(Integer input) { + return input; + } + }; + + MapElements simpleMap = MapElements.via(simpleFn); + assertThat(DisplayData.from(simpleMap), hasDisplayItem("mapFn", simpleFn.getClass())); + } + + @Test + public void testPrimitiveDisplayData() { + SimpleFunction mapFn = new SimpleFunction() { + @Override + public Integer apply(Integer input) { + return input; + } + }; + + MapElements map = MapElements.via(mapFn); + DisplayDataEvaluator evaluator = DataflowDisplayDataEvaluator.create(); + + Set displayData = evaluator.displayDataForPrimitiveTransforms(map); + assertThat("MapElements should include the mapFn in its primitive display data", + displayData, hasItem(hasDisplayItem("mapFn", mapFn.getClass()))); + } + static class VoidValues extends PTransform>, PCollection>> { @Override public PCollection> apply(PCollection> input) { - return input.apply(MapElements., KV>via( + return input.apply(MapElements.via( new SimpleFunction, KV>() { @Override public KV apply(KV input) { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java index 04baade57c..ac24706f4f 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java @@ -19,7 +19,7 @@ import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasKey; import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasType; -import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includes; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; import static com.google.cloud.dataflow.sdk.util.SerializableUtils.serializeToByteArray; import static com.google.cloud.dataflow.sdk.util.StringUtils.byteArrayToJsonString; import static com.google.cloud.dataflow.sdk.util.StringUtils.jsonStringToByteArray; @@ -1534,7 +1534,7 @@ public void processElement(ProcessContext c) { @Override public void populateDisplayData(Builder builder) { - builder.add("doFnMetadata", "bar"); + builder.add(DisplayData.item("doFnMetadata", "bar")); } }; @@ -1546,7 +1546,7 @@ public void populateDisplayData(Builder builder) { hasType(DisplayData.Type.JAVA_CLASS), DisplayDataMatchers.hasValue(fn.getClass().getName())))); - assertThat(displayData, includes(fn)); + assertThat(displayData, includesDisplayDataFrom(fn)); } @Test @@ -1557,14 +1557,14 @@ public void proccessElement(ProcessContext c) {} @Override public void populateDisplayData(Builder builder) { - builder.add("fnMetadata", "foobar"); + builder.add(DisplayData.item("fnMetadata", "foobar")); } }; Bound parDo = ParDo.of(fn); DisplayData displayData = DisplayData.from(parDo); - assertThat(displayData, includes(fn)); + assertThat(displayData, includesDisplayDataFrom(fn)); assertThat(displayData, hasDisplayItem("fn", fn.getClass())); } @@ -1576,7 +1576,7 @@ public void proccessElement(ProcessContext c) {} @Override public void populateDisplayData(Builder builder) { - builder.add("fnMetadata", "foobar"); + builder.add(DisplayData.item("fnMetadata", "foobar")); } }; @@ -1585,7 +1585,7 @@ public void populateDisplayData(Builder builder) { .of(fn); DisplayData displayData = DisplayData.from(parDo); - assertThat(displayData, includes(fn)); + assertThat(displayData, includesDisplayDataFrom(fn)); assertThat(displayData, hasDisplayItem("fn", fn.getClass())); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/ClassForDisplayTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/ClassForDisplayTest.java deleted file mode 100644 index 9c6102b7f9..0000000000 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/ClassForDisplayTest.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Copyright (C) 2016 Google Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ -package com.google.cloud.dataflow.sdk.transforms.display; - -import static org.junit.Assert.assertEquals; - -import com.google.cloud.dataflow.sdk.util.SerializableUtils; -import com.google.common.testing.EqualsTester; - -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** - * Unit tests for {@link ClassForDisplay}. - */ -@RunWith(JUnit4.class) -public class ClassForDisplayTest { - @Rule - public final ExpectedException thrown = ExpectedException.none(); - - @Test - public void testProperties() { - ClassForDisplay thisClass = ClassForDisplay.of(ClassForDisplayTest.class); - assertEquals(ClassForDisplayTest.class.getName(), thisClass.getName()); - assertEquals(ClassForDisplayTest.class.getSimpleName(), thisClass.getSimpleName()); - } - - @Test - public void testInputValidation() { - thrown.expect(NullPointerException.class); - ClassForDisplay.of(null); - } - - @Test - public void testEquality() { - new EqualsTester() - .addEqualityGroup( - ClassForDisplay.of(ClassForDisplayTest.class), ClassForDisplay.fromInstance(this)) - .addEqualityGroup(ClassForDisplay.of(ClassForDisplay.class)) - .addEqualityGroup(ClassForDisplay.of(Class.class)) - .testEquals(); - } - - @Test - public void testSerialization() { - SerializableUtils.ensureSerializable(ClassForDisplay.of(ClassForDisplayTest.class)); - } -} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DataflowDisplayDataEvaluator.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DataflowDisplayDataEvaluator.java new file mode 100644 index 0000000000..4040634a93 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DataflowDisplayDataEvaluator.java @@ -0,0 +1,69 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.sdk.transforms.display; + +import com.google.cloud.dataflow.sdk.options.DataflowPipelineDebugOptions; +import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions; +import com.google.cloud.dataflow.sdk.options.GcpOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner; +import com.google.cloud.dataflow.sdk.util.NoopCredentialFactory; +import com.google.cloud.dataflow.sdk.util.NoopPathValidator; +import com.google.common.collect.Lists; + +/** + * Factory methods for creating {@link DisplayDataEvaluator} instances against the + * {@link DataflowPipelineRunner}. + */ +public final class DataflowDisplayDataEvaluator { + /** Do not instantiate. */ + private DataflowDisplayDataEvaluator() {} + + /** + * Retrieve a set of default {@link DataflowPipelineOptions} which can be used to build + * dataflow pipelines for evaluating display data. + */ + public static DataflowPipelineOptions getDefaultOptions() { + DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); + + options.setRunner(DataflowPipelineRunner.class); + options.setProject("foobar"); + options.setTempLocation("gs://bucket/tmpLocation"); + options.setFilesToStage(Lists.newArrayList()); + + options.as(DataflowPipelineDebugOptions.class).setPathValidatorClass(NoopPathValidator.class); + options.as(GcpOptions.class).setCredentialFactoryClass(NoopCredentialFactory.class); + + return options; + } + + /** + * Create a {@link DisplayDataEvaluator} instance to evaluate pipeline display data against + * the {@link DataflowPipelineRunner}. + */ + public static DisplayDataEvaluator create() { + return create(getDefaultOptions()); + } + + /** + * Create a {@link DisplayDataEvaluator} instance to evaluate pipeline display data against + * the {@link DataflowPipelineRunner} with the specified {@code options}. + */ + public static DisplayDataEvaluator create(DataflowPipelineOptions options) { + return DisplayDataEvaluator.create(options); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataEvaluator.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataEvaluator.java new file mode 100644 index 0000000000..1da3cc8868 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataEvaluator.java @@ -0,0 +1,142 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.sdk.transforms.display; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.runners.PipelineRunner; +import com.google.cloud.dataflow.sdk.runners.TransformTreeNode; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.POutput; +import com.google.cloud.dataflow.sdk.values.PValue; +import com.google.common.collect.Sets; + +import java.util.Objects; +import java.util.Set; + +/** + * Test utilities to evaluate the {@link DisplayData} in the context of a {@link PipelineRunner}. + */ +public class DisplayDataEvaluator { + private final PipelineOptions options; + + /** + * Create a new {@link DisplayDataEvaluator} using {@link TestPipeline#testingPipelineOptions()}. + */ + public static DisplayDataEvaluator create() { + return create(TestPipeline.testingPipelineOptions()); + } + + /** + * Create a new {@link DisplayDataEvaluator} using the specified {@link PipelineOptions}. + */ + public static DisplayDataEvaluator create(PipelineOptions pipelineOptions) { + return new DisplayDataEvaluator(pipelineOptions); + } + + private DisplayDataEvaluator(PipelineOptions options) { + this.options = options; + } + + /** + * Traverse the specified {@link PTransform}, collecting {@link DisplayData} registered on the + * inner primitive {@link PTransform PTransforms}. + * + * @return the set of {@link DisplayData} for primitive {@link PTransform PTransforms}. + */ + public Set displayDataForPrimitiveTransforms( + final PTransform, ? extends POutput> root) { + return displayDataForPrimitiveTransforms(root, null); + } + + /** + * Traverse the specified {@link PTransform}, collecting {@link DisplayData} registered on the + * inner primitive {@link PTransform PTransforms}. + * + * @param root The root {@link PTransform} to traverse + * @param inputCoder The coder to set for the {@link PTransform} input, or null to infer the + * default coder. + * + * @return the set of {@link DisplayData} for primitive {@link PTransform PTransforms}. + */ + public Set displayDataForPrimitiveTransforms( + final PTransform, ? extends POutput> root, + Coder inputCoder) { + + Create.Values input = Create.of(); + if (inputCoder != null) { + input = input.withCoder(inputCoder); + } + + Pipeline pipeline = Pipeline.create(options); + pipeline + .apply(input) + .apply(root); + + PrimitiveDisplayDataPTransformVisitor visitor = new PrimitiveDisplayDataPTransformVisitor(root); + pipeline.traverseTopologically(visitor); + return visitor.getPrimitivesDisplayData(); + } + + /** + * Visits {@link PTransform PTransforms} in a pipeline, and collects {@link DisplayData} for + * each primitive transform under a given composite root. + */ + private static class PrimitiveDisplayDataPTransformVisitor implements Pipeline.PipelineVisitor { + private final PTransform root; + private final Set displayData; + private boolean inCompositeRoot = false; + + PrimitiveDisplayDataPTransformVisitor(PTransform root) { + this.root = root; + this.displayData = Sets.newHashSet(); + } + + Set getPrimitivesDisplayData() { + return displayData; + } + + @Override + public void enterCompositeTransform(TransformTreeNode node) { + if (Objects.equals(root, node.getTransform())) { + inCompositeRoot = true; + } + } + + @Override + public void leaveCompositeTransform(TransformTreeNode node) { + if (Objects.equals(root, node.getTransform())) { + inCompositeRoot = false; + } + } + @Override + public void visitTransform(TransformTreeNode node) { + if (inCompositeRoot || Objects.equals(root, node.getTransform())) { + displayData.add(DisplayData.from(node.getTransform())); + } + } + + @Override + public void visitValue(PValue value, TransformTreeNode producer) { + + } + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataEvaluatorTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataEvaluatorTest.java new file mode 100644 index 0000000000..e1921e95ee --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataEvaluatorTest.java @@ -0,0 +1,91 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.dataflow.sdk.transforms.display; + +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; + +import static org.hamcrest.Matchers.hasItem; +import static org.hamcrest.Matchers.not; +import static org.junit.Assert.assertThat; + +import com.google.cloud.dataflow.sdk.transforms.DoFn; +import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.POutput; + +import org.junit.Test; + +import java.io.Serializable; +import java.util.Set; + +/** + * Unit tests for {@link DisplayDataEvaluator}. + */ +public class DisplayDataEvaluatorTest implements Serializable { + + @Test + public void testCompositeTransform() { + PTransform, ? super POutput> myTransform = + new PTransform, POutput> () { + @Override + public PCollection apply(PCollection input) { + return input.apply(ParDo.of(new DoFn() { + @Override + public void processElement(ProcessContext c) throws Exception { + c.output(c.element()); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.add(DisplayData.item("primitiveKey", "primitiveValue")); + } + })); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.add(DisplayData.item("compositeKey", "compositeValue")); + } + }; + + DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(); + Set displayData = evaluator.displayDataForPrimitiveTransforms(myTransform); + + assertThat(displayData, not(hasItem(hasDisplayItem("compositeKey", "compositeValue")))); + assertThat(displayData, hasItem(hasDisplayItem("primitiveKey", "primitiveValue"))); + } + + @Test + public void testPrimitiveTransform() { + PTransform, ? super PCollection> myTransform = ParDo.of( + new DoFn() { + @Override + public void processElement(ProcessContext c) throws Exception {} + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.add(DisplayData.item("foo", "bar")); + } + }); + + DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(); + Set displayData = evaluator.displayDataForPrimitiveTransforms(myTransform); + + assertThat(displayData, hasItem(hasDisplayItem("foo"))); + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchers.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchers.java index aa7b66f0b6..de0dcb8ba7 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchers.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchers.java @@ -45,7 +45,21 @@ private DisplayDataMatchers() {} * Creates a matcher that matches if the examined {@link DisplayData} contains any items. */ public static Matcher hasDisplayItem() { - return hasDisplayItem(Matchers.any(DisplayData.Item.class)); + return new FeatureMatcher>>( + Matchers.not(Matchers.empty()), "DisplayData", "DisplayData") { + @Override + protected Collection> featureValueOf(DisplayData actual) { + return actual.items(); + } + }; + } + + /** + * Creates a matcher that matches if the examined {@link DisplayData} contains an item with the + * specified key. + */ + public static Matcher hasDisplayItem(String key) { + return hasDisplayItem(hasKey(key)); } /** @@ -117,14 +131,14 @@ private static Matcher hasDisplayItem( * Creates a matcher that matches if the examined {@link DisplayData} contains any item * matching the specified {@code itemMatcher}. */ - public static Matcher hasDisplayItem(Matcher itemMatcher) { + public static Matcher hasDisplayItem(Matcher> itemMatcher) { return new HasDisplayDataItemMatcher(itemMatcher); } private static class HasDisplayDataItemMatcher extends TypeSafeDiagnosingMatcher { - private final Matcher itemMatcher; + private final Matcher> itemMatcher; - private HasDisplayDataItemMatcher(Matcher itemMatcher) { + private HasDisplayDataItemMatcher(Matcher> itemMatcher) { this.itemMatcher = itemMatcher; } @@ -136,7 +150,7 @@ public void describeTo(Description description) { @Override protected boolean matchesSafely(DisplayData data, Description mismatchDescription) { - Collection items = data.items(); + Collection> items = data.items(); boolean isMatch = Matchers.hasItem(itemMatcher).matches(items); if (!isMatch) { mismatchDescription.appendText("found " + items.size() + " non-matching item(s):\n"); @@ -147,22 +161,22 @@ protected boolean matchesSafely(DisplayData data, Description mismatchDescriptio } } - /** @see #includes(HasDisplayData, String) */ - public static Matcher includes(HasDisplayData subComponent) { - return includes(subComponent, subComponent.getClass()); + /** @see #includesDisplayDataFrom(HasDisplayData, String) */ + public static Matcher includesDisplayDataFrom(HasDisplayData subComponent) { + return includesDisplayDataFrom(subComponent, subComponent.getClass()); } - /** @see #includes(HasDisplayData, String) */ - public static Matcher includes( + /** @see #includesDisplayDataFrom(HasDisplayData, String) */ + public static Matcher includesDisplayDataFrom( HasDisplayData subComponent, Class namespace) { - return includes(subComponent, namespace.getName()); + return includesDisplayDataFrom(subComponent, namespace.getName()); } /** * Create a matcher that matches if the examined {@link DisplayData} contains all display data * registered from the specified subcomponent and namespace. */ - public static Matcher includes( + public static Matcher includesDisplayDataFrom( final HasDisplayData subComponent, final String namespace) { return new CustomTypeSafeMatcher("includes subcomponent") { @Override @@ -203,8 +217,8 @@ public void populateDisplayData(DisplayData.Builder builder) { private DisplayDataComparison checkSubset( DisplayData displayData, DisplayData included) { DisplayDataComparison comparison = new DisplayDataComparison(displayData.items()); - for (Item item : included.items()) { - Item matchedItem = displayData.asMap().get( + for (Item item : included.items()) { + Item matchedItem = displayData.asMap().get( DisplayData.Identifier.of(item.getNamespace(), item.getKey())); if (matchedItem != null) { @@ -218,19 +232,19 @@ private DisplayDataComparison checkSubset( } class DisplayDataComparison { - Collection missingItems; - Collection unmatchedItems; + Collection> missingItems; + Collection> unmatchedItems; - DisplayDataComparison(Collection superset) { + DisplayDataComparison(Collection> superset) { missingItems = Sets.newHashSet(); unmatchedItems = Sets.newHashSet(superset); } - void matched(Item supersetItem) { + void matched(Item supersetItem) { unmatchedItems.remove(supersetItem); } - void missing(Item subsetItem) { + void missing(Item subsetItem) { missingItems.add(subsetItem); } } @@ -241,7 +255,7 @@ void missing(Item subsetItem) { * Creates a matcher that matches if the examined {@link DisplayData.Item} contains a key * with the specified value. */ - public static Matcher hasKey(String key) { + public static Matcher> hasKey(String key) { return hasKey(Matchers.is(key)); } @@ -249,10 +263,10 @@ public static Matcher hasKey(String key) { * Creates a matcher that matches if the examined {@link DisplayData.Item} contains a key * matching the specified key matcher. */ - public static Matcher hasKey(Matcher keyMatcher) { - return new FeatureMatcher(keyMatcher, "with key", "key") { + public static Matcher> hasKey(Matcher keyMatcher) { + return new FeatureMatcher, String>(keyMatcher, "with key", "key") { @Override - protected String featureValueOf(DisplayData.Item actual) { + protected String featureValueOf(DisplayData.Item actual) { return actual.getKey(); } }; @@ -262,19 +276,33 @@ protected String featureValueOf(DisplayData.Item actual) { * Creates a matcher that matches if the examined {@link DisplayData.Item} contains the * specified namespace. */ - public static Matcher hasNamespace(Class namespace) { + public static Matcher> hasNamespace(Class namespace) { return hasNamespace(Matchers.>is(namespace)); } + /** + * Creates a matcher that matches if the examined {@link DisplayData.Item} contains the + * specified namespace. + */ + public static Matcher> hasNamespace(String namespace) { + return new FeatureMatcher, String>( + Matchers.is(namespace), "display item with namespace", "namespace") { + @Override + protected String featureValueOf(Item actual) { + return actual.getNamespace(); + } + }; + } + /** * Creates a matcher that matches if the examined {@link DisplayData.Item} contains a namespace * matching the specified namespace matcher. */ - public static Matcher hasNamespace(Matcher> namespaceMatcher) { - return new FeatureMatcher>( + public static Matcher> hasNamespace(Matcher> namespaceMatcher) { + return new FeatureMatcher, Class>( namespaceMatcher, "display item with namespace", "namespace") { @Override - protected Class featureValueOf(DisplayData.Item actual) { + protected Class featureValueOf(DisplayData.Item actual) { try { return Class.forName(actual.getNamespace()); } catch (ClassNotFoundException e) { @@ -288,7 +316,7 @@ protected Class featureValueOf(DisplayData.Item actual) { * Creates a matcher that matches if the examined {@link DisplayData.Item} matches the * specified type. */ - public static Matcher hasType(DisplayData.Type type) { + public static Matcher> hasType(DisplayData.Type type) { return hasType(Matchers.is(type)); } @@ -296,11 +324,11 @@ public static Matcher hasType(DisplayData.Type type) { * Creates a matcher that matches if the examined {@link DisplayData.Item} has a type * matching the specified type matcher. */ - public static Matcher hasType(Matcher typeMatcher) { - return new FeatureMatcher( + public static Matcher> hasType(Matcher typeMatcher) { + return new FeatureMatcher, DisplayData.Type>( typeMatcher, "with type", "type") { @Override - protected DisplayData.Type featureValueOf(DisplayData.Item actual) { + protected DisplayData.Type featureValueOf(DisplayData.Item actual) { return actual.getType(); } }; @@ -311,7 +339,7 @@ protected DisplayData.Type featureValueOf(DisplayData.Item actual) { * value. */ - public static Matcher hasValue(Object value) { + public static Matcher> hasValue(Object value) { return hasValue(Matchers.is(value)); } @@ -319,11 +347,11 @@ public static Matcher hasValue(Object value) { * Creates a matcher that matches if the examined {@link DisplayData.Item} contains a value * matching the specified value matcher. */ - public static Matcher hasValue(Matcher valueMatcher) { - return new FeatureMatcher( + public static Matcher> hasValue(Matcher valueMatcher) { + return new FeatureMatcher, T>( valueMatcher, "with value", "value") { @Override - protected T featureValueOf(DisplayData.Item actual) { + protected T featureValueOf(DisplayData.Item actual) { @SuppressWarnings("unchecked") T value = (T) actual.getValue(); return value; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchersTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchersTest.java index 4ee76d9c2f..d519975898 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchersTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchersTest.java @@ -17,13 +17,12 @@ package com.google.cloud.dataflow.sdk.transforms.display; import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; -import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasKey; import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasNamespace; import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasType; import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasValue; -import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includes; -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.core.StringStartsWith.startsWith; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; + +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; @@ -60,13 +59,13 @@ public void testHasDisplayItemDescription() { matcher.describeTo(desc); matcher.describeMismatch(DisplayData.none(), mismatchDesc); - assertThat(desc.toString(), startsWith("display data with item: ")); - assertThat(mismatchDesc.toString(), containsString("found 0 non-matching item(s)")); + assertEquals("DisplayData not an empty collection", desc.toString()); + assertEquals("DisplayData was <[]>", mismatchDesc.toString()); } @Test public void testHasKey() { - Matcher matcher = hasDisplayItem(hasKey("foo")); + Matcher matcher = hasDisplayItem("foo"); assertFalse(matcher.matches(createDisplayDataWithItem("fooz", "bar"))); @@ -80,7 +79,7 @@ public void testHasType() { DisplayData data = DisplayData.from(new PTransform, PCollection>() { @Override public void populateDisplayData(Builder builder) { - builder.add("foo", DisplayDataMatchersTest.class); + builder.add(DisplayData.item("foo", DisplayDataMatchersTest.class)); } }); @@ -110,7 +109,7 @@ public void testIncludes() { final HasDisplayData subComponent = new HasDisplayData() { @Override public void populateDisplayData(Builder builder) { - builder.add("foo", "bar"); + builder.add(DisplayData.item("foo", "bar")); } }; HasDisplayData hasSubcomponent = new HasDisplayData() { @@ -118,16 +117,16 @@ public void populateDisplayData(Builder builder) { public void populateDisplayData(Builder builder) { builder .include(subComponent) - .add("foo2", "bar2"); + .add(DisplayData.item("foo2", "bar2")); } }; HasDisplayData sameKeyDifferentNamespace = new HasDisplayData() { @Override public void populateDisplayData(Builder builder) { - builder.add("foo", "bar"); + builder.add(DisplayData.item("foo", "bar")); } }; - Matcher matcher = includes(subComponent); + Matcher matcher = includesDisplayDataFrom(subComponent); assertFalse(matcher.matches(DisplayData.from(sameKeyDifferentNamespace))); assertThat(DisplayData.from(hasSubcomponent), matcher); @@ -150,7 +149,7 @@ static class SampleTransform extends PTransform, PCollection @Override public void populateDisplayData(Builder builder) { - builder.add(key, value); + builder.add(DisplayData.item(key, value)); } } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataTest.java index 625bfa168f..4adbb33fb6 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataTest.java @@ -21,7 +21,8 @@ import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasNamespace; import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasType; import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasValue; -import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includes; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; + import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.everyItem; @@ -37,7 +38,14 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.testing.DataflowAssert; +import com.google.cloud.dataflow.sdk.testing.RunnableOnService; +import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.Create; +import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.transforms.display.DisplayData.Builder; import com.google.cloud.dataflow.sdk.transforms.display.DisplayData.Item; import com.google.cloud.dataflow.sdk.values.PCollection; @@ -58,11 +66,13 @@ import org.joda.time.format.ISODateTimeFormat; import org.junit.Rule; import org.junit.Test; +import org.junit.experimental.categories.Category; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import java.io.IOException; +import java.io.Serializable; import java.util.Collection; import java.util.Map; import java.util.regex.Pattern; @@ -71,8 +81,8 @@ * Tests for {@link DisplayData} class. */ @RunWith(JUnit4.class) -public class DisplayDataTest { - @Rule public ExpectedException thrown = ExpectedException.none(); +public class DisplayDataTest implements Serializable { + @Rule public transient ExpectedException thrown = ExpectedException.none(); private static final DateTimeFormatter ISO_FORMATTER = ISODateTimeFormat.dateTime(); private static final ObjectMapper MAPPER = new ObjectMapper(); @@ -82,7 +92,7 @@ public void testTypicalUsage() { new HasDisplayData() { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("ExpectedAnswer", 42); + builder.add(DisplayData.item("ExpectedAnswer", 42)); } }; @@ -90,7 +100,9 @@ public void populateDisplayData(DisplayData.Builder builder) { new HasDisplayData() { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("Location", "Seattle").add("Forecast", "Rain"); + builder + .add(DisplayData.item("Location", "Seattle")) + .add(DisplayData.item("Forecast", "Rain")); } }; @@ -105,14 +117,14 @@ public void populateDisplayData(DisplayData.Builder builder) { builder .include(subComponent1) .include(subComponent2) - .add("minSproggles", 200) - .withLabel("Mimimum Required Sproggles") - .add("fireLazers", true) - .addIfNotDefault("startTime", startTime, defaultStartTime) - .add("timeBomb", Instant.now().plus(Duration.standardDays(1))) - .add("filterLogic", subComponent1.getClass()) - .add("serviceUrl", "google.com/fizzbang") - .withLinkUrl("http://www.google.com/fizzbang"); + .add(DisplayData.item("minSproggles", 200) + .withLabel("Minimum Required Sproggles")) + .add(DisplayData.item("fireLasers", true)) + .addIfNotDefault(DisplayData.item("startTime", startTime), defaultStartTime) + .add(DisplayData.item("timeBomb", Instant.now().plus(Duration.standardDays(1)))) + .add(DisplayData.item("filterLogic", subComponent1.getClass())) + .add(DisplayData.item("serviceUrl", "google.com/fizzbang") + .withLinkUrl("http://www.google.com/fizzbang")); } }; @@ -138,12 +150,12 @@ public void testDefaultInstance() { } @Test - public void testCanBuild() { + public void testCanBuildDisplayData() { DisplayData data = DisplayData.from(new HasDisplayData() { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("foo", "bar"); + builder.add(DisplayData.item("foo", "bar")); } }); @@ -158,11 +170,11 @@ public void testAsMap() { new HasDisplayData() { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("foo", "bar"); + builder.add(DisplayData.item("foo", "bar")); } }); - Map map = data.asMap(); + Map> map = data.asMap(); assertEquals(map.size(), 1); assertThat(data, hasDisplayItem("foo", "bar")); assertEquals(map.values(), data.items()); @@ -174,25 +186,27 @@ public void testItemProperties() { DisplayData data = DisplayData.from(new HasDisplayData() { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("now", value) + builder.add(DisplayData.item("now", value) .withLabel("the current instant") .withLinkUrl("http://time.gov") - .withNamespace(DisplayDataTest.class); + .withNamespace(DisplayDataTest.class)); } }); @SuppressWarnings("unchecked") - DisplayData.Item item = (DisplayData.Item) data.items().toArray()[0]; - assertThat( - item, - Matchers.allOf( - hasNamespace(DisplayDataTest.class), - hasKey("now"), - hasType(DisplayData.Type.TIMESTAMP), - hasValue(ISO_FORMATTER.print(value)), - hasShortValue(nullValue(String.class)), - hasLabel(is("the current instant")), - hasUrl(is("http://time.gov")))); + DisplayData.Item item = (DisplayData.Item) data.items().toArray()[0]; + + @SuppressWarnings("unchecked") + Matcher> matchesAllOf = Matchers.allOf( + hasNamespace(DisplayDataTest.class), + hasKey("now"), + hasType(DisplayData.Type.TIMESTAMP), + hasValue(ISO_FORMATTER.print(value)), + hasShortValue(nullValue(String.class)), + hasLabel(is("the current instant")), + hasUrl(is("http://time.gov"))); + + assertThat(item, matchesAllOf); } @Test @@ -202,7 +216,7 @@ public void testUnspecifiedOptionalProperties() { new HasDisplayData() { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("foo", "bar"); + builder.add(DisplayData.item("foo", "bar")); } }); @@ -217,20 +231,26 @@ public void testAddIfNotDefault() { @Override public void populateDisplayData(Builder builder) { builder - .addIfNotDefault("defaultString", "foo", "foo") - .addIfNotDefault("notDefaultString", "foo", "notFoo") - .addIfNotDefault("defaultInteger", 1, 1) - .addIfNotDefault("notDefaultInteger", 1, 2) - .addIfNotDefault("defaultDouble", 123.4, 123.4) - .addIfNotDefault("notDefaultDouble", 123.4, 234.5) - .addIfNotDefault("defaultBoolean", true, true) - .addIfNotDefault("notDefaultBoolean", true, false) - .addIfNotDefault("defaultInstant", new Instant(0), new Instant(0)) - .addIfNotDefault("notDefaultInstant", new Instant(0), Instant.now()) - .addIfNotDefault("defaultDuration", Duration.ZERO, Duration.ZERO) - .addIfNotDefault("notDefaultDuration", Duration.millis(1234), Duration.ZERO) - .addIfNotDefault("defaultClass", DisplayDataTest.class, DisplayDataTest.class) - .addIfNotDefault("notDefaultClass", DisplayDataTest.class, null); + .addIfNotDefault(DisplayData.item("defaultString", "foo"), "foo") + .addIfNotDefault(DisplayData.item("notDefaultString", "foo"), "notFoo") + .addIfNotDefault(DisplayData.item("defaultInteger", 1), 1) + .addIfNotDefault(DisplayData.item("notDefaultInteger", 1), 2) + .addIfNotDefault(DisplayData.item("defaultDouble", 123.4), 123.4) + .addIfNotDefault(DisplayData.item("notDefaultDouble", 123.4), 234.5) + .addIfNotDefault(DisplayData.item("defaultBoolean", true), true) + .addIfNotDefault(DisplayData.item("notDefaultBoolean", true), false) + .addIfNotDefault(DisplayData.item("defaultInstant", new Instant(0)), new Instant(0)) + .addIfNotDefault(DisplayData.item("notDefaultInstant", new Instant(0)), Instant.now()) + .addIfNotDefault(DisplayData.item("defaultDuration", Duration.ZERO), Duration.ZERO) + .addIfNotDefault( + DisplayData.item("notDefaultDuration", Duration.millis(1234)), + Duration.ZERO) + .addIfNotDefault( + DisplayData.item("defaultClass", DisplayDataTest.class), + DisplayDataTest.class) + .addIfNotDefault( + DisplayData.item("notDefaultClass", DisplayDataTest.class), + null); } }); @@ -238,26 +258,53 @@ public void populateDisplayData(Builder builder) { assertThat(data.items(), everyItem(hasKey(startsWith("notDefault")))); } + @Test + @SuppressWarnings("UnnecessaryBoxing") + public void testInterpolatedTypeDefaults() { + DisplayData data = DisplayData.from(new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder + .addIfNotDefault(DisplayData.item("integer", 123), 123) + .addIfNotDefault(DisplayData.item("Integer", + Integer.valueOf(123)), + Integer.valueOf(123)) + .addIfNotDefault(DisplayData.item("long", 123L), 123L) + .addIfNotDefault(DisplayData.item("Long", Long.valueOf(123)), Long.valueOf(123)) + .addIfNotDefault(DisplayData.item("float", 1.23f), 1.23f) + .addIfNotDefault(DisplayData.item("Float", Float.valueOf(1.23f)), Float.valueOf(1.23f)) + .addIfNotDefault(DisplayData.item("double", 1.23), 1.23) + .addIfNotDefault(DisplayData.item("Double", Double.valueOf(1.23)), Double.valueOf(1.23)) + .addIfNotDefault(DisplayData.item("boolean", true), true) + .addIfNotDefault( + DisplayData.item("Boolean", Boolean.valueOf(true)), + Boolean.valueOf(true)); + } + }); + + assertThat(data.items(), empty()); + } + @Test public void testAddIfNotNull() { DisplayData data = DisplayData.from(new HasDisplayData() { @Override public void populateDisplayData(Builder builder) { builder - .addIfNotNull("nullString", (String) null) - .addIfNotNull("notNullString", "foo") - .addIfNotNull("nullLong", (Long) null) - .addIfNotNull("notNullLong", 1234L) - .addIfNotNull("nullDouble", (Double) null) - .addIfNotNull("notNullDouble", 123.4) - .addIfNotNull("nullBoolean", (Boolean) null) - .addIfNotNull("notNullBoolean", true) - .addIfNotNull("nullInstant", (Instant) null) - .addIfNotNull("notNullInstant", Instant.now()) - .addIfNotNull("nullDuration", (Duration) null) - .addIfNotNull("notNullDuration", Duration.ZERO) - .addIfNotNull("nullClass", (Class) null) - .addIfNotNull("notNullClass", DisplayDataTest.class); + .addIfNotNull(DisplayData.item("nullString", (String) null)) + .addIfNotNull(DisplayData.item("notNullString", "foo")) + .addIfNotNull(DisplayData.item("nullLong", (Long) null)) + .addIfNotNull(DisplayData.item("notNullLong", 1234L)) + .addIfNotNull(DisplayData.item("nullDouble", (Double) null)) + .addIfNotNull(DisplayData.item("notNullDouble", 123.4)) + .addIfNotNull(DisplayData.item("nullBoolean", (Boolean) null)) + .addIfNotNull(DisplayData.item("notNullBoolean", true)) + .addIfNotNull(DisplayData.item("nullInstant", (Instant) null)) + .addIfNotNull(DisplayData.item("notNullInstant", Instant.now())) + .addIfNotNull(DisplayData.item("nullDuration", (Duration) null)) + .addIfNotNull(DisplayData.item("notNullDuration", Duration.ZERO)) + .addIfNotNull(DisplayData.item("nullClass", (Class) null)) + .addIfNotNull(DisplayData.item("notNullClass", DisplayDataTest.class)); } }); @@ -270,10 +317,10 @@ public void testModifyingConditionalItemIsSafe() { HasDisplayData component = new HasDisplayData() { @Override public void populateDisplayData(Builder builder) { - builder.addIfNotNull("nullItem", (Class) null) + builder.addIfNotNull(DisplayData.item("nullItem", (Class) null) .withLinkUrl("http://abc") .withNamespace(DisplayDataTest.class) - .withLabel("Null item shoudl be safe"); + .withLabel("Null item should be safe")); } }; @@ -287,7 +334,7 @@ public void testIncludes() { new HasDisplayData() { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("foo", "bar"); + builder.add(DisplayData.item("foo", "bar")); } }; @@ -300,7 +347,7 @@ public void populateDisplayData(DisplayData.Builder builder) { } }); - assertThat(data, includes(subComponent)); + assertThat(data, includesDisplayDataFrom(subComponent)); } @Test @@ -308,7 +355,7 @@ public void testIncludesNamespaceOverride() { final HasDisplayData subComponent = new HasDisplayData() { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("foo", "bar"); + builder.add(DisplayData.item("foo", "bar")); } }; @@ -325,7 +372,40 @@ public void populateDisplayData(DisplayData.Builder builder) { } }); - assertThat(data, includes(subComponent, namespaceOverride.getClass())); + assertThat(data, includesDisplayDataFrom(subComponent, namespaceOverride.getClass())); + } + + @Test + public void testNamespaceOverrideMultipleLevels() { + final HasDisplayData componentA = new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder.add(DisplayData.item("foo", "bar")); + } + }; + + final HasDisplayData componentB = new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder + .add(DisplayData.item("foo", "bar")) + .include(componentA); + } + }; + + final HasDisplayData componentC = new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder + .add(DisplayData.item("foo", "bar")) + .include(componentB, "overrideB"); + } + }; + + DisplayData data = DisplayData.from(componentC); + assertThat(data, hasDisplayItem(hasNamespace(componentC.getClass()))); + assertThat(data, hasDisplayItem(hasNamespace("overrideB"))); + assertThat(data, hasDisplayItem(hasNamespace(componentA.getClass()))); } @Test @@ -335,8 +415,8 @@ public void testNullNamespaceOverride() { DisplayData.from(new HasDisplayData() { @Override public void populateDisplayData(Builder builder) { - builder.add("foo", "bar") - .withNamespace((Class) null); + builder.add(DisplayData.item("foo", "bar") + .withNamespace((Class) null)); } }); } @@ -345,37 +425,18 @@ public void populateDisplayData(Builder builder) { public void testIdentifierEquality() { new EqualsTester() .addEqualityGroup( - DisplayData.Identifier.of(ClassForDisplay.of(DisplayDataTest.class), "1"), - DisplayData.Identifier.of(ClassForDisplay.of(DisplayDataTest.class), "1")) - .addEqualityGroup(DisplayData.Identifier.of(ClassForDisplay.of(Object.class), "1")) - .addEqualityGroup(DisplayData.Identifier.of(ClassForDisplay.of(DisplayDataTest.class), "2")) + DisplayData.Identifier.of(DisplayDataTest.class, "1"), + DisplayData.Identifier.of(DisplayDataTest.class, "1")) + .addEqualityGroup(DisplayData.Identifier.of(Object.class, "1")) + .addEqualityGroup(DisplayData.Identifier.of(DisplayDataTest.class, "2")) .testEquals(); } @Test public void testItemEquality() { - HasDisplayData component1 = new HasDisplayData() { - @Override - public void populateDisplayData(Builder builder) { - builder.add("foo", "bar"); - } - }; - HasDisplayData component2 = new HasDisplayData() { - @Override - public void populateDisplayData(Builder builder) { - builder.add("foo", "bar"); - } - }; - - DisplayData component1DisplayData1 = DisplayData.from(component1); - DisplayData component1DisplayData2 = DisplayData.from(component1); - DisplayData component2DisplayData = DisplayData.from(component2); - new EqualsTester() - .addEqualityGroup( - component1DisplayData1.items().toArray()[0], - component1DisplayData2.items().toArray()[0]) - .addEqualityGroup(component2DisplayData.items().toArray()[0]) + .addEqualityGroup(DisplayData.item("foo", "bar"), DisplayData.item("foo", "bar")) + .addEqualityGroup(DisplayData.item("foo", "barz")) .testEquals(); } @@ -384,13 +445,13 @@ public void testDisplayDataEquality() { HasDisplayData component1 = new HasDisplayData() { @Override public void populateDisplayData(Builder builder) { - builder.add("foo", "bar"); + builder.add(DisplayData.item("foo", "bar")); } }; HasDisplayData component2 = new HasDisplayData() { @Override public void populateDisplayData(Builder builder) { - builder.add("foo", "bar"); + builder.add(DisplayData.item("foo", "bar")); } }; @@ -411,11 +472,11 @@ public void testAnonymousClassNamespace() { new HasDisplayData() { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("foo", "bar"); + builder.add(DisplayData.item("foo", "bar")); } }); - DisplayData.Item item = (DisplayData.Item) data.items().toArray()[0]; + DisplayData.Item item = (DisplayData.Item) data.items().toArray()[0]; final Pattern anonClassRegex = Pattern.compile( Pattern.quote(DisplayDataTest.class.getName()) + "\\$\\d+$"); assertThat(item.getNamespace(), new CustomTypeSafeMatcher( @@ -436,12 +497,12 @@ public void testAcceptsKeysWithDifferentNamespaces() { @Override public void populateDisplayData(DisplayData.Builder builder) { builder - .add("foo", "bar") + .add(DisplayData.item("foo", "bar")) .include( new HasDisplayData() { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("foo", "bar"); + builder.add(DisplayData.item("foo", "bar")); } }); } @@ -458,8 +519,8 @@ public void testDuplicateKeyThrowsException() { @Override public void populateDisplayData(DisplayData.Builder builder) { builder - .add("foo", "bar") - .add("foo", "baz"); + .add(DisplayData.item("foo", "bar")) + .add(DisplayData.item("foo", "baz")); } }); } @@ -471,9 +532,9 @@ public void testDuplicateKeyWithNamespaceOverrideDoesntThrow() { @Override public void populateDisplayData(DisplayData.Builder builder) { builder - .add("foo", "bar") - .add("foo", "baz") - .withNamespace(DisplayDataTest.class); + .add(DisplayData.item("foo", "bar")) + .add(DisplayData.item("foo", "baz") + .withNamespace(DisplayDataTest.class)); } }); @@ -485,7 +546,7 @@ public void testToString() { HasDisplayData component = new HasDisplayData() { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("foo", "bar"); + builder.add(DisplayData.item("foo", "bar")); } }; @@ -550,7 +611,7 @@ private static class EqualsEverything implements HasDisplayData { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add(key, value); + builder.add(DisplayData.item(key, value)); } @Override @@ -570,7 +631,9 @@ abstract static class IncludeSubComponent implements HasDisplayData { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("id", getId()).include(subComponent); + builder + .add(DisplayData.item("id", getId())) + .include(subComponent); } abstract String getId(); @@ -584,18 +647,17 @@ public void testTypeMappings() { @Override public void populateDisplayData(DisplayData.Builder builder) { builder - .add("string", "foobar") - .add("integer", 123) - .add("float", 3.14) - .add("boolean", true) - .add("java_class", DisplayDataTest.class) - .add("java_class2", ClassForDisplay.of(DisplayDataTest.class)) - .add("timestamp", Instant.now()) - .add("duration", Duration.standardHours(1)); + .add(DisplayData.item("string", "foobar")) + .add(DisplayData.item("integer", 123)) + .add(DisplayData.item("float", 3.14)) + .add(DisplayData.item("boolean", true)) + .add(DisplayData.item("java_class", DisplayDataTest.class)) + .add(DisplayData.item("timestamp", Instant.now())) + .add(DisplayData.item("duration", Duration.standardHours(1))); } }); - Collection items = data.items(); + Collection> items = data.items(); assertThat( items, hasItem(allOf(hasKey("string"), hasType(DisplayData.Type.STRING)))); assertThat( @@ -605,9 +667,6 @@ public void populateDisplayData(DisplayData.Builder builder) { assertThat( items, hasItem(allOf(hasKey("java_class"), hasType(DisplayData.Type.JAVA_CLASS)))); - assertThat( - items, - hasItem(allOf(hasKey("java_class2"), hasType(DisplayData.Type.JAVA_CLASS)))); assertThat( items, hasItem(allOf(hasKey("timestamp"), hasType(DisplayData.Type.TIMESTAMP)))); @@ -621,8 +680,8 @@ public void testExplicitItemType() { @Override public void populateDisplayData(Builder builder) { builder - .add("integer", DisplayData.Type.INTEGER, 1234L) - .add("string", DisplayData.Type.STRING, "foobar"); + .add(DisplayData.item("integer", DisplayData.Type.INTEGER, 1234L)) + .add(DisplayData.item("string", DisplayData.Type.STRING, "foobar")); } }); @@ -686,7 +745,7 @@ public void testInvalidExplicitItemType() { HasDisplayData component = new HasDisplayData() { @Override public void populateDisplayData(Builder builder) { - builder.add("integer", DisplayData.Type.INTEGER, "foobar"); + builder.add(DisplayData.item("integer", DisplayData.Type.INTEGER, "foobar")); } }; @@ -703,8 +762,6 @@ public void testKnownTypeInference() { assertEquals(DisplayData.Type.BOOLEAN, DisplayData.inferType(true)); assertEquals(DisplayData.Type.TIMESTAMP, DisplayData.inferType(Instant.now())); assertEquals(DisplayData.Type.DURATION, DisplayData.inferType(Duration.millis(1234))); - assertEquals(DisplayData.Type.JAVA_CLASS, - DisplayData.inferType(ClassForDisplay.of(DisplayDataTest.class))); assertEquals(DisplayData.Type.JAVA_CLASS, DisplayData.inferType(DisplayDataTest.class)); assertEquals(DisplayData.Type.STRING, DisplayData.inferType("hello world")); @@ -721,13 +778,13 @@ public void testStringFormatting() throws IOException { @Override public void populateDisplayData(DisplayData.Builder builder) { builder - .add("string", "foobar") - .add("integer", 123) - .add("float", 3.14) - .add("boolean", true) - .add("java_class", DisplayDataTest.class) - .add("timestamp", now) - .add("duration", oneHour); + .add(DisplayData.item("string", "foobar")) + .add(DisplayData.item("integer", 123)) + .add(DisplayData.item("float", 3.14)) + .add(DisplayData.item("boolean", true)) + .add(DisplayData.item("java_class", DisplayDataTest.class)) + .add(DisplayData.item("timestamp", now)) + .add(DisplayData.item("duration", oneHour)); } }; DisplayData data = DisplayData.from(component); @@ -747,7 +804,7 @@ public void testContextProperlyReset() { new HasDisplayData() { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("foo", "bar"); + builder.add(DisplayData.item("foo", "bar")); } }; @@ -757,7 +814,7 @@ public void populateDisplayData(DisplayData.Builder builder) { public void populateDisplayData(DisplayData.Builder builder) { builder .include(subComponent) - .add("alpha", "bravo"); + .add(DisplayData.item("alpha", "bravo")); } }; @@ -800,7 +857,7 @@ public void populateDisplayData(Builder builder) { DisplayData.from(new HasDisplayData() { @Override public void populateDisplayData(Builder builder) { - builder.include(subComponent, (ClassForDisplay) null); + builder.include(subComponent, (Class) null); } }); } @@ -812,7 +869,7 @@ public void testNullKey() { new HasDisplayData() { @Override public void populateDisplayData(Builder builder) { - builder.add(null, "foo"); + builder.add(DisplayData.item(null, "foo")); } }); } @@ -824,28 +881,28 @@ public void testRejectsNullValues() { @Override public void populateDisplayData(Builder builder) { try { - builder.add("key", (String) null); + builder.add(DisplayData.item("key", (String) null)); throw new RuntimeException("Should throw on null string value"); } catch (NullPointerException ex) { // Expected } try { - builder.add("key", (Class) null); + builder.add(DisplayData.item("key", (Class) null)); throw new RuntimeException("Should throw on null class value"); } catch (NullPointerException ex) { // Expected } try { - builder.add("key", (Duration) null); + builder.add(DisplayData.item("key", (Duration) null)); throw new RuntimeException("Should throw on null duration value"); } catch (NullPointerException ex) { // Expected } try { - builder.add("key", (Instant) null); + builder.add(DisplayData.item("key", (Instant) null)); throw new RuntimeException("Should throw on null instant value"); } catch (NullPointerException ex) { // Expected @@ -860,9 +917,9 @@ public void testAcceptsNullOptionalValues() { new HasDisplayData() { @Override public void populateDisplayData(Builder builder) { - builder.add("key", "value") + builder.add(DisplayData.item("key", "value") .withLabel(null) - .withLinkUrl(null); + .withLinkUrl(null)); } }); @@ -881,16 +938,15 @@ public void testJsonSerialization() throws IOException { @Override public void populateDisplayData(Builder builder) { builder - .add("string", stringValue) - .add("long", intValue) - .add("double", floatValue) - .add("boolean", boolValue) - .add("instant", new Instant(0)) - .add("duration", Duration.millis(durationMillis)) - .add("class", DisplayDataTest.class) + .add(DisplayData.item("string", stringValue)) + .add(DisplayData.item("long", intValue)) + .add(DisplayData.item("double", floatValue)) + .add(DisplayData.item("boolean", boolValue)) + .add(DisplayData.item("instant", new Instant(0))) + .add(DisplayData.item("duration", Duration.millis(durationMillis))) + .add(DisplayData.item("class", DisplayDataTest.class) .withLinkUrl("http://abc") - .withLabel("baz") - ; + .withLabel("baz")); } }; DisplayData data = DisplayData.from(component); @@ -908,6 +964,38 @@ public void populateDisplayData(Builder builder) { quoted("DisplayDataTest"), "baz", "http://abc")); } + /** + * Validate that all runners are resilient to exceptions thrown while retrieving display data. + */ + @Test + @Category(RunnableOnService.class) + public void testRunnersResilientToDisplayDataExceptions() { + Pipeline p = TestPipeline.create(); + PCollection pCol = p + .apply(Create.of(1, 2, 3)) + .apply(new IdentityTransform() { + @Override + public void populateDisplayData(Builder builder) { + throw new RuntimeException("bug!"); + } + }); + + DataflowAssert.that(pCol).containsInAnyOrder(1, 2, 3); + p.run(); + } + + private static class IdentityTransform extends PTransform, PCollection> { + @Override + public PCollection apply(PCollection input) { + return input.apply(ParDo.of(new DoFn() { + @Override + public void processElement(DoFn.ProcessContext c) throws Exception { + c.output(c.element()); + } + })); + } + } + private String quoted(Object obj) { return String.format("\"%s\"", obj); } @@ -951,31 +1039,31 @@ private Matcher> hasExpectedJson( return hasItem(jsonNode); } - private static Matcher hasLabel(Matcher labelMatcher) { - return new FeatureMatcher( + private static Matcher> hasLabel(Matcher labelMatcher) { + return new FeatureMatcher, String>( labelMatcher, "display item with label", "label") { @Override - protected String featureValueOf(DisplayData.Item actual) { + protected String featureValueOf(DisplayData.Item actual) { return actual.getLabel(); } }; } - private static Matcher hasUrl(Matcher urlMatcher) { - return new FeatureMatcher( + private static Matcher> hasUrl(Matcher urlMatcher) { + return new FeatureMatcher, String>( urlMatcher, "display item with url", "URL") { @Override - protected String featureValueOf(DisplayData.Item actual) { + protected String featureValueOf(DisplayData.Item actual) { return actual.getLinkUrl(); } }; } - private static Matcher hasShortValue(Matcher valueStringMatcher) { - return new FeatureMatcher( + private static Matcher> hasShortValue(Matcher valueStringMatcher) { + return new FeatureMatcher, T>( valueStringMatcher, "display item with short value", "short value") { @Override - protected T featureValueOf(DisplayData.Item actual) { + protected T featureValueOf(DisplayData.Item actual) { @SuppressWarnings("unchecked") T shortValue = (T) actual.getShortValue(); return shortValue; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowTest.java index 970b815ccc..060202a742 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowTest.java @@ -18,7 +18,7 @@ import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasKey; -import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includes; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.isOneOf; @@ -249,7 +249,7 @@ public void testDisplayData() { DisplayData displayData = DisplayData.from(window); assertThat(displayData, hasDisplayItem("windowFn", windowFn.getClass())); - assertThat(displayData, includes(windowFn)); + assertThat(displayData, includesDisplayDataFrom(windowFn)); assertThat(displayData, hasDisplayItem("trigger", triggerBuilder.toString())); assertThat(displayData, @@ -262,24 +262,27 @@ public void testDisplayData() { @Test public void testDisplayDataExcludesUnspecifiedProperties() { - Window.Bound window = Window.into(new GlobalWindows()); - - DisplayData displayData = DisplayData.from(window); - assertThat(displayData, not(hasDisplayItem(hasKey(isOneOf( + Window.Bound onlyHasAccumulationMode = Window.named("foobar").discardingFiredPanes(); + assertThat(DisplayData.from(onlyHasAccumulationMode), not(hasDisplayItem(hasKey(isOneOf( + "windowFn", "trigger", "outputTimeFn", - "accumulationMode", "allowedLateness", "closingBehavior"))))); + Window.Bound noAccumulationMode = Window.into(new GlobalWindows()); + assertThat(DisplayData.from(noAccumulationMode), + not(hasDisplayItem(hasKey("accumulationMode")))); } @Test - public void testDisplayDataExcludesDefaultTrigger() { + public void testDisplayDataExcludesDefaults() { Window.Bound window = Window.into(new GlobalWindows()) - .triggering(DefaultTrigger.of()); + .triggering(DefaultTrigger.of()) + .withAllowedLateness(Duration.millis(BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis())); DisplayData data = DisplayData.from(window); - assertThat(data, not(hasDisplayItem(hasKey("trigger")))); + assertThat(data, not(hasDisplayItem("trigger"))); + assertThat(data, not(hasDisplayItem("allowedLateness"))); } } diff --git a/sdk/src/test/java8/com/google/cloud/dataflow/sdk/transforms/display/ClassForDisplayJava8Test.java b/sdk/src/test/java8/com/google/cloud/dataflow/sdk/transforms/display/ClassForDisplayJava8Test.java deleted file mode 100644 index de6bd7df87..0000000000 --- a/sdk/src/test/java8/com/google/cloud/dataflow/sdk/transforms/display/ClassForDisplayJava8Test.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.google.cloud.dataflow.sdk.transforms.display; - -import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; -import com.google.cloud.dataflow.sdk.util.SerializableUtils; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -import java.io.Serializable; - -/** - * Java 8 tests for {@link ClassForDisplay}. - */ -@RunWith(JUnit4.class) -public class ClassForDisplayJava8Test implements Serializable { - @Test - public void testLambdaClassSerialization() { - final SerializableFunction f = x -> x; - Serializable myClass = new Serializable() { - // Class references for lambdas do not serialize, which is why we support ClassForDisplay - // Specifically, the following would not work: - // Class clazz = f.getClass(); - ClassForDisplay javaClass = ClassForDisplay.fromInstance(f); - }; - - SerializableUtils.ensureSerializable(myClass); - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java new file mode 100644 index 0000000000..8ad57d2bb3 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupByKey.java @@ -0,0 +1,287 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.transforms; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.Coder.NonDeterministicException; +import org.apache.beam.sdk.coders.IterableCoder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; +import org.apache.beam.sdk.transforms.windowing.GlobalWindows; +import org.apache.beam.sdk.transforms.windowing.InvalidWindows; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollection.IsBounded; + +/** + * {@code GroupByKey} takes a {@code PCollection>}, + * groups the values by key and windows, and returns a + * {@code PCollection>>} representing a map from + * each distinct key and window of the input {@code PCollection} to an + * {@code Iterable} over all the values associated with that key in + * the input per window. Absent repeatedly-firing + * {@link Window#triggering triggering}, each key in the output + * {@code PCollection} is unique within each window. + * + *

{@code GroupByKey} is analogous to converting a multi-map into + * a uni-map, and related to {@code GROUP BY} in SQL. It corresponds + * to the "shuffle" step between the Mapper and the Reducer in the + * MapReduce framework. + * + *

Two keys of type {@code K} are compared for equality + * not by regular Java {@link Object#equals}, but instead by + * first encoding each of the keys using the {@code Coder} of the + * keys of the input {@code PCollection}, and then comparing the + * encoded bytes. This admits efficient parallel evaluation. Note that + * this requires that the {@code Coder} of the keys be deterministic (see + * {@link Coder#verifyDeterministic()}). If the key {@code Coder} is not + * deterministic, an exception is thrown at pipeline construction time. + * + *

By default, the {@code Coder} of the keys of the output + * {@code PCollection} is the same as that of the keys of the input, + * and the {@code Coder} of the elements of the {@code Iterable} + * values of the output {@code PCollection} is the same as the + * {@code Coder} of the values of the input. + * + *

Example of use: + *

 {@code
+ * PCollection> urlDocPairs = ...;
+ * PCollection>> urlToDocs =
+ *     urlDocPairs.apply(GroupByKey.create());
+ * PCollection results =
+ *     urlToDocs.apply(ParDo.of(new DoFn>, R>() {
+ *       public void processElement(ProcessContext c) {
+ *         String url = c.element().getKey();
+ *         Iterable docsWithThatUrl = c.element().getValue();
+ *         ... process all docs having that url ...
+ *       }}));
+ * } 
+ * + *

{@code GroupByKey} is a key primitive in data-parallel + * processing, since it is the main way to efficiently bring + * associated data together into one location. It is also a key + * determiner of the performance of a data-parallel pipeline. + * + *

See {@link org.apache.beam.sdk.transforms.join.CoGroupByKey} + * for a way to group multiple input PCollections by a common key at once. + * + *

See {@link Combine.PerKey} for a common pattern of + * {@code GroupByKey} followed by {@link Combine.GroupedValues}. + * + *

When grouping, windows that can be merged according to the {@link WindowFn} + * of the input {@code PCollection} will be merged together, and a window pane + * corresponding to the new, merged window will be created. The items in this pane + * will be emitted when a trigger fires. By default this will be when the input + * sources estimate there will be no more data for the window. See + * {@link org.apache.beam.sdk.transforms.windowing.AfterWatermark} + * for details on the estimation. + * + *

The timestamp for each emitted pane is determined by the + * {@link Window.Bound#withOutputTimeFn windowing operation}. + * The output {@code PCollection} will have the same {@link WindowFn} + * as the input. + * + *

If the input {@code PCollection} contains late data (see + * {@link org.apache.beam.sdk.io.PubsubIO.Read.Bound#timestampLabel} + * for an example of how this can occur) or the + * {@link Window#triggering requested TriggerFn} can fire before + * the watermark, then there may be multiple elements + * output by a {@code GroupByKey} that correspond to the same key and window. + * + *

If the {@link WindowFn} of the input requires merging, it is not + * valid to apply another {@code GroupByKey} without first applying a new + * {@link WindowFn} or applying {@link Window#remerge()}. + * + * @param the type of the keys of the input and output + * {@code PCollection}s + * @param the type of the values of the input {@code PCollection} + * and the elements of the {@code Iterable}s in the output + * {@code PCollection} + */ +public class GroupByKey + extends PTransform>, + PCollection>>> { + + private final boolean fewKeys; + + private GroupByKey(boolean fewKeys) { + this.fewKeys = fewKeys; + } + + /** + * Returns a {@code GroupByKey} {@code PTransform}. + * + * @param the type of the keys of the input and output + * {@code PCollection}s + * @param the type of the values of the input {@code PCollection} + * and the elements of the {@code Iterable}s in the output + * {@code PCollection} + */ + public static GroupByKey create() { + return new GroupByKey<>(false); + } + + /** + * Returns a {@code GroupByKey} {@code PTransform}. + * + * @param the type of the keys of the input and output + * {@code PCollection}s + * @param the type of the values of the input {@code PCollection} + * and the elements of the {@code Iterable}s in the output + * {@code PCollection} + * @param fewKeys whether it groups just few keys. + */ + static GroupByKey create(boolean fewKeys) { + return new GroupByKey<>(fewKeys); + } + + /** + * Returns whether it groups just few keys. + */ + public boolean fewKeys() { + return fewKeys; + } + + ///////////////////////////////////////////////////////////////////////////// + + public static void applicableTo(PCollection input) { + WindowingStrategy windowingStrategy = input.getWindowingStrategy(); + // Verify that the input PCollection is bounded, or that there is windowing/triggering being + // used. Without this, the watermark (at end of global window) will never be reached. + if (windowingStrategy.getWindowFn() instanceof GlobalWindows + && windowingStrategy.getTrigger().getSpec() instanceof DefaultTrigger + && input.isBounded() != IsBounded.BOUNDED) { + throw new IllegalStateException("GroupByKey cannot be applied to non-bounded PCollection in " + + "the GlobalWindow without a trigger. Use a Window.into or Window.triggering transform " + + "prior to GroupByKey."); + } + + // Validate the window merge function. + if (windowingStrategy.getWindowFn() instanceof InvalidWindows) { + String cause = ((InvalidWindows) windowingStrategy.getWindowFn()).getCause(); + throw new IllegalStateException( + "GroupByKey must have a valid Window merge function. " + + "Invalid because: " + cause); + } + } + + @Override + public void validate(PCollection> input) { + applicableTo(input); + + // Verify that the input Coder> is a KvCoder, and that + // the key coder is deterministic. + Coder keyCoder = getKeyCoder(input.getCoder()); + try { + keyCoder.verifyDeterministic(); + } catch (NonDeterministicException e) { + throw new IllegalStateException( + "the keyCoder of a GroupByKey must be deterministic", e); + } + } + + public WindowingStrategy updateWindowingStrategy(WindowingStrategy inputStrategy) { + WindowFn inputWindowFn = inputStrategy.getWindowFn(); + if (!inputWindowFn.isNonMerging()) { + // Prevent merging windows again, without explicit user + // involvement, e.g., by Window.into() or Window.remerge(). + inputWindowFn = new InvalidWindows<>( + "WindowFn has already been consumed by previous GroupByKey", inputWindowFn); + } + + // We also switch to the continuation trigger associated with the current trigger. + return inputStrategy + .withWindowFn(inputWindowFn) + .withTrigger(inputStrategy.getTrigger().getSpec().getContinuationTrigger()); + } + + @Override + public PCollection>> apply(PCollection> input) { + // This primitive operation groups by the combination of key and window, + // merging windows as needed, using the windows assigned to the + // key/value input elements and the window merge operation of the + // window function associated with the input PCollection. + return PCollection.createPrimitiveOutputInternal(input.getPipeline(), + updateWindowingStrategy(input.getWindowingStrategy()), input.isBounded()); + } + + @Override + protected Coder>> getDefaultOutputCoder(PCollection> input) { + return getOutputKvCoder(input.getCoder()); + } + + /** + * Returns the {@code Coder} of the input to this transform, which + * should be a {@code KvCoder}. + */ + @SuppressWarnings("unchecked") + static KvCoder getInputKvCoder(Coder> inputCoder) { + if (!(inputCoder instanceof KvCoder)) { + throw new IllegalStateException( + "GroupByKey requires its input to use KvCoder"); + } + return (KvCoder) inputCoder; + } + + ///////////////////////////////////////////////////////////////////////////// + + /** + * Returns the {@code Coder} of the keys of the input to this + * transform, which is also used as the {@code Coder} of the keys of + * the output of this transform. + */ + public static Coder getKeyCoder(Coder> inputCoder) { + return getInputKvCoder(inputCoder).getKeyCoder(); + } + + /** + * Returns the {@code Coder} of the values of the input to this transform. + */ + public static Coder getInputValueCoder(Coder> inputCoder) { + return getInputKvCoder(inputCoder).getValueCoder(); + } + + /** + * Returns the {@code Coder} of the {@code Iterable} values of the + * output of this transform. + */ + static Coder> getOutputValueCoder(Coder> inputCoder) { + return IterableCoder.of(getInputValueCoder(inputCoder)); + } + + /** + * Returns the {@code Coder} of the output of this transform. + */ + public static KvCoder> getOutputKvCoder(Coder> inputCoder) { + return KvCoder.of(getKeyCoder(inputCoder), getOutputValueCoder(inputCoder)); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + if (fewKeys) { + builder.add(DisplayData.item("fewKeys", true) + .withLabel("Has Few Keys")); + } + } + +}