diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java index 3a39e41e2b7b..58b38d2f56c3 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java @@ -874,10 +874,10 @@ public void processElement(ProcessContext c) throws Exception { @Override public void populateDisplayData(DisplayData.Builder builder) { builder - .add("foo", "bar") - .add("foo2", DataflowPipelineTranslatorTest.class) + .add(DisplayData.item("foo", "bar")) + .add(DisplayData.item("foo2", DataflowPipelineTranslatorTest.class) .withLabel("Test Class") - .withLinkUrl("http://www.google.com"); + .withLinkUrl("http://www.google.com")); } }; @@ -889,7 +889,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)); } }; @@ -915,8 +915,10 @@ public void populateDisplayData(DisplayData.Builder builder) { Map parDo2Properties = steps.get(2).getProperties(); assertThat(parDo1Properties, hasKey("display_data")); + @SuppressWarnings("unchecked") Collection> fn1displayData = (Collection>) parDo1Properties.get("display_data"); + @SuppressWarnings("unchecked") Collection> fn2displayData = (Collection>) parDo2Properties.get("display_data"); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java index c706fc1624e2..6b9f0100f034 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java @@ -331,8 +331,8 @@ 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)) + .addIfNotDefault(DisplayData.item("validation", validate), true); } @Override @@ -694,12 +694,14 @@ 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)) + .addIfNotNull(DisplayData.item("filePrefix", filenamePrefix)) + .addIfNotDefault( + DisplayData.item("shardNameTemplate", shardTemplate), + DEFAULT_SHARD_TEMPLATE) + .addIfNotDefault(DisplayData.item("fileSuffix", filenameSuffix), "") + .addIfNotDefault(DisplayData.item("numShards", numShards), 0) + .addIfNotDefault(DisplayData.item("validation", validate), true); } /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java index 1d5b993174fd..4a0bdacb6970 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java @@ -522,13 +522,13 @@ public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); if (table != null) { - builder.add("table", toTableSpec(table)); + builder.add(DisplayData.item("table", toTableSpec(table))); } builder - .addIfNotNull("query", query) - .addIfNotNull("flattenResults", flattenResults) - .addIfNotDefault("validation", validate, true); + .addIfNotNull(DisplayData.item("query", query)) + .addIfNotNull(DisplayData.item("flattenResults", flattenResults)) + .addIfNotDefault(DisplayData.item("validation", validate), true); } static { @@ -1067,17 +1067,17 @@ public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .addIfNotNull("table", jsonTableRef) - .addIfNotNull("schema", jsonSchema); + .addIfNotNull(DisplayData.item("table", jsonTableRef)) + .addIfNotNull(DisplayData.item("schema", jsonSchema)); if (tableRefFunction != null) { - builder.add("tableFn", tableRefFunction.getClass()); + builder.add(DisplayData.item("tableFn", tableRefFunction.getClass())); } builder - .add("createDisposition", createDisposition.toString()) - .add("writeDisposition", writeDisposition.toString()) - .addIfNotDefault("validation", validate, true); + .add(DisplayData.item("createDisposition", createDisposition.toString())) + .add(DisplayData.item("writeDisposition", writeDisposition.toString())) + .addIfNotDefault(DisplayData.item("validation", validate), true); } /** Returns the create disposition. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java index dc57b911bfc3..cf4f02ddd60c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java @@ -113,9 +113,9 @@ 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())) + .addIfNotDefault(DisplayData.item("maxRecords", maxNumRecords), Long.MAX_VALUE) + .addIfNotNull(DisplayData.item("maxReadTime", maxReadTime)) .include(source); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java index ce21595c39f4..370c1e22c2ec 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java @@ -326,14 +326,14 @@ 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())); 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())); } else { - builder.add("compressionMode", channelFactory.getClass()); + builder.add(DisplayData.item("compressionMode", channelFactory.getClass())); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingInput.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingInput.java index f6b1256ca549..5a53e4d990dd 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingInput.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingInput.java @@ -119,7 +119,7 @@ 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)); } } @@ -233,14 +233,14 @@ 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())); if (maxReadTime.isPresent()) { - builder.add("maxReadTime", maxReadTime.get()); + builder.add(DisplayData.item("maxReadTime", maxReadTime.get())); } if (maxNumRecords.isPresent()) { - builder.add("maxRecords", maxNumRecords.get()); + builder.add(DisplayData.item("maxRecords", maxNumRecords.get())); } } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DatastoreIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DatastoreIO.java index c26565983046..81c3a395c156 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DatastoreIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DatastoreIO.java @@ -391,12 +391,12 @@ 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), DEFAULT_HOST) + .addIfNotNull(DisplayData.item("dataset", datasetId)) + .addIfNotNull(DisplayData.item("namespace", namespace)); if (query != null) { - builder.add("query", query.toString()); + builder.add(DisplayData.item("query", query.toString())); } } @@ -606,8 +606,8 @@ public DatastoreWriteOperation createWriteOperation(PipelineOptions options) { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .addIfNotDefault("host", host, DEFAULT_HOST) - .addIfNotNull("dataset", datasetId); + .addIfNotDefault(DisplayData.item("host", host), DEFAULT_HOST) + .addIfNotNull(DisplayData.item("dataset", datasetId)); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java index ba3d4ea11dda..10e93f5aeb2e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java @@ -142,7 +142,7 @@ 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)); } /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java index bbb2fef1f950..954877f6ec73 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java @@ -277,7 +277,7 @@ private static long getEstimatedSizeOfFilesBySampling( @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - builder.add("filePattern", getFileOrPatternSpec()); + builder.add(DisplayData.item("filePattern", getFileOrPatternSpec())); } private ListenableFuture>> createFutureForFileSplit( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java index 69585a5447f5..126535a5b52d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java @@ -207,9 +207,9 @@ 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); + .add(DisplayData.item("minBundleSize", minBundleSize)) + .addIfNotDefault(DisplayData.item("startOffset", startOffset), 0L) + .addIfNotDefault(DisplayData.item("endOffset", endOffset), Long.MAX_VALUE); } /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java index 1b0ec4a02488..66d1d4335505 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java @@ -694,17 +694,17 @@ public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .addIfNotNull("timestampLabel", timestampLabel) - .addIfNotNull("idLabel", idLabel) - .addIfNotNull("maxReadTime", maxReadTime) - .addIfNotDefault("maxNumRecords", maxNumRecords, 0); + .addIfNotNull(DisplayData.item("timestampLabel", timestampLabel)) + .addIfNotNull(DisplayData.item("idLabel", idLabel)) + .addIfNotNull(DisplayData.item("maxReadTime", maxReadTime)) + .addIfNotDefault(DisplayData.item("maxNumRecords", maxNumRecords), 0); if (topic != null) { - builder.add("topic", topic.asPath()); + builder.add(DisplayData.item("topic", topic.asPath())); } if (subscription != null) { - builder.add("subscription", subscription.asPath()); + builder.add(DisplayData.item("subscription", subscription.asPath())); } } @@ -998,11 +998,11 @@ public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .addIfNotNull("timestampLabel", timestampLabel) - .addIfNotNull("idLabel", idLabel); + .addIfNotNull(DisplayData.item("timestampLabel", timestampLabel)) + .addIfNotNull(DisplayData.item("idLabel", idLabel)); if (topic != null) { - builder.add("topic", topic.asPath()); + builder.add(DisplayData.item("topic", topic.asPath())); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java index 05b70ac8eb44..965b14f07baf 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java @@ -149,7 +149,7 @@ public String getKindString() { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .add("source", source.getClass()) + .add(DisplayData.item("source", source.getClass())) .include(source); } @@ -264,7 +264,7 @@ public String getKindString() { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .add("source", source.getClass()) + .add(DisplayData.item("source", source.getClass())) .include(source); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java index 3882ee1a3dda..7f69c0a2131a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java @@ -344,9 +344,9 @@ 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())) + .addIfNotDefault(DisplayData.item("validation", validate), true) + .addIfNotNull(DisplayData.item("filePattern", filepattern)); } @Override @@ -649,11 +649,13 @@ 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)) + .addIfNotDefault(DisplayData.item("fileSuffix", filenameSuffix), "") + .addIfNotDefault( + DisplayData.item("shardNameTemplate", shardTemplate), + DEFAULT_SHARD_TEMPLATE) + .addIfNotDefault(DisplayData.item("validation", validate), true) + .addIfNotDefault(DisplayData.item("numShards", numShards), 0); } /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java index a7d182da1e89..b6743fa93824 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java @@ -81,7 +81,7 @@ public PDone apply(PCollection input) { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .add("sink", sink.getClass()) + .add(DisplayData.item("sink", sink.getClass())) .include(sink); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSink.java index 00819c944396..eb7ba38dee03 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSink.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSink.java @@ -227,8 +227,8 @@ 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)) + .addIfNotNull(DisplayData.item("recordClass", classToBind)); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSource.java index 89b10327f769..0a418c883ac9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSource.java @@ -222,9 +222,9 @@ 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)) + .addIfNotNull(DisplayData.item("recordElement", recordElement)) + .addIfNotNull(DisplayData.item("recordClass", recordClass)); } @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java index a269f4e6f8b7..745549f4cd60 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java @@ -283,11 +283,11 @@ private void populateDisplayData(DisplayData.Builder builder) { for (PipelineOptionSpec optionSpec : specs) { 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)); } } } @@ -300,18 +300,18 @@ 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) { 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())); } } } @@ -542,7 +542,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/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java index dd99b9acbc50..66f7a6a327a8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java @@ -365,8 +365,8 @@ public Coder> getAccumulatorCoder( public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .add("numQuantiles", numQuantiles) - .add("comparer", compareFn.getClass()); + .add(DisplayData.item("numQuantiles", numQuantiles)) + .add(DisplayData.item("comparer", compareFn.getClass())); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateUnique.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateUnique.java index 46bad0cf9368..e4559d472a76 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateUnique.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateUnique.java @@ -461,7 +461,7 @@ 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)) + .addIfNotNull(DisplayData.item("maximumEstimationError", maxEstimationError)); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java index b13b0fc62079..68fc1cf5b14e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java @@ -1416,14 +1416,14 @@ private static void populateDisplayData( DisplayData.Builder builder, HasDisplayData fn, ClassForDisplay fnClass) { builder .include(fn, fnClass) - .add("combineFn", fnClass); + .add(DisplayData.item("combineFn", fnClass)); } private static void populateGlobalDisplayData( DisplayData.Builder builder, int fanout, boolean insertDefault) { builder - .addIfNotDefault("fanout", fanout, 0) - .add("emitDefaultOnEmptyInput", insertDefault); + .addIfNotDefault(DisplayData.item("fanout", fanout), 0) + .add(DisplayData.item("emitDefaultOnEmptyInput", insertDefault)); } /** @@ -1598,7 +1598,7 @@ public List compact(List accumulator) { @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - builder.add("combineFn", combiner.getClass()); + builder.add(DisplayData.item("combineFn", combiner.getClass())); } private List mergeToSingleton(Iterable values) { @@ -2088,7 +2088,7 @@ public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); Combine.populateDisplayData(builder, fn, fnClass); - builder.add("fanoutFn", hotKeyFanout.getClass()); + builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass())); } /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java index a9500d84b152..1bf305fb9e87 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java @@ -1053,7 +1053,7 @@ 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())); combineFnMap.put(combineFn.getClass(), combineFn); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Filter.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Filter.java index 0108958ed3dc..da692b2ce19d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Filter.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Filter.java @@ -261,6 +261,6 @@ protected Coder getDefaultOutputCoder(PCollection input) { private static void populateDisplayData( DisplayData.Builder builder, String predicateDescription) { - builder.add("predicate", predicateDescription); + builder.add(DisplayData.item("predicate", predicateDescription)); } } 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 index 0e46a2ad4edd..21e6ecfb816f 100644 --- 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 @@ -279,7 +279,7 @@ public static KvCoder> getOutputKvCoder(Coder> in public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); if (fewKeys) { - builder.add("fewKeys", true); + builder.add(DisplayData.item("fewKeys", true)); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/IntraBundleParallelization.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/IntraBundleParallelization.java index 36149caf51c8..4b3afb426ea0 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/IntraBundleParallelization.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/IntraBundleParallelization.java @@ -178,8 +178,8 @@ public PCollection apply(PCollection input) { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .add("maxParallelism", maxParallelism) - .add("fn", doFn.getClass()) + .add(DisplayData.item("maxParallelism", maxParallelism)) + .add(DisplayData.item("fn", doFn.getClass())) .include(doFn); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java index 96900b4e0108..01a190f234ac 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java @@ -209,7 +209,7 @@ public T apply(T left, T right) { @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - builder.add("comparer", comparator.getClass()); + builder.add(DisplayData.item("comparer", comparator.getClass())); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java index 7d69c5aad571..f20d48bbe4ee 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java @@ -209,7 +209,7 @@ public T apply(T left, T right) { @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - builder.add("comparer", comparator.getClass()); + builder.add(DisplayData.item("comparer", comparator.getClass())); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java index 547486dfd7fa..000a777f8f82 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java @@ -1266,7 +1266,7 @@ private static void populateDisplayData( DisplayData.Builder builder, DoFn fn, Class fnClass) { builder .include(fn, fnClass) - .add("fn", fnClass); + .add(DisplayData.item("fn", fnClass)); } /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Partition.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Partition.java index fbdb9bec189f..44ba2f30b111 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Partition.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Partition.java @@ -182,8 +182,8 @@ public void processElement(ProcessContext c) { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .add("numPartitions", numPartitions) - .add("partitionFn", partitionFn.getClass()); + .add(DisplayData.item("numPartitions", numPartitions)) + .add(DisplayData.item("partitionFn", partitionFn.getClass())); } } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sample.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sample.java index 2fba5e37642a..58188def4226 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sample.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sample.java @@ -158,7 +158,7 @@ public PCollection apply(PCollection in) { @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - builder.add("sampleSize", limit); + builder.add(DisplayData.item("sampleSize", limit)); } } @@ -258,7 +258,7 @@ public Coder> getDefaultOutputCoder( @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - builder.add("sampleSize", sampleSize); + builder.add(DisplayData.item("sampleSize", sampleSize)); } } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java index 2d38e04ec1ef..71af081278a8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java @@ -397,8 +397,8 @@ public Coder> getAccumulatorCoder( public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .add("count", count) - .add("comparer", compareFn.getClass()); + .add(DisplayData.item("count", count)) + .add(DisplayData.item("comparer", compareFn.getClass())); } @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/ClassForDisplay.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/ClassForDisplay.java index 3e31c1e09e81..b5142e803c7f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/ClassForDisplay.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/ClassForDisplay.java @@ -26,7 +26,7 @@ * Display data representing a Java class. * *

Java classes can be registered as display data via - * {@link DisplayData.Builder#add(String, ClassForDisplay)}. {@link ClassForDisplay} is + * {@link DisplayData.Builder#item(String, ClassForDisplay)}. {@link ClassForDisplay} is * serializable, unlike {@link Class} which can fail to serialize for Java 8 lambda functions. */ public class ClassForDisplay implements Serializable { @@ -42,14 +42,15 @@ private ClassForDisplay(Class clazz) { * Create a {@link ClassForDisplay} instance representing the specified class. */ public static ClassForDisplay of(Class clazz) { - return new ClassForDisplay(checkNotNull(clazz)); + checkNotNull(clazz, "clazz argument cannot be null"); + return new ClassForDisplay(clazz); } /** * Create a {@link ClassForDisplay} from the class of the specified object instance. */ public static ClassForDisplay fromInstance(Object obj) { - checkNotNull(obj); + checkNotNull(obj, "obj argument instance cannot be null"); return new ClassForDisplay(obj.getClass()); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java index c119057f5eb6..76661722bb7d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java @@ -17,13 +17,14 @@ */ package org.apache.beam.sdk.transforms.display; -import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; +import com.google.auto.value.AutoValue; +import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.common.collect.Sets; @@ -52,12 +53,12 @@ * interface. */ public class DisplayData { - private static final DisplayData EMPTY = new DisplayData(Maps.newHashMap()); + 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); } @@ -74,7 +75,7 @@ public static DisplayData none() { * a namespace derived from the component. */ public static DisplayData from(HasDisplayData component) { - checkNotNull(component); + checkNotNull(component, "component argument cannot be null"); return InternalBuilder.forRoot(component).build(); } @@ -89,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)); * } * } * } @@ -103,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; } @@ -130,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 { @@ -182,309 +183,79 @@ public interface Builder { Builder include(HasDisplayData subComponent, String namespace); /** - * Register the given string display data. 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. + * Register the given display item. */ - ItemBuilder add(String key, String value); + Builder add(Item item); /** - * 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 data. 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 data. 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 data. 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) + * Register the given display item if the value is not null. */ - ItemBuilder addIfNotDefault(String key, boolean value, boolean defaultValue); + Builder addIfNotNull(Item item); /** - * Register the given timestamp display data. 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 data. 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. - * - * @see DisplayData.Builder#add(String, Duration) - */ - ItemBuilder addIfNotDefault( - String key, @Nullable Duration value, @Nullable Duration defaultValue); - - /** - * Register the given class display data. 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 data. 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. - * - * @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. - * - * @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. - * - * @see DisplayData.Builder#add(String, Class) + * Register the given display item if the value is different than the specified default. */ - ItemBuilder addIfNotDefault( - String key, @Nullable Class value, @Nullable Class defaultValue); + Builder addIfNotDefault(Item item, @Nullable T defaultValue); + } - /** - * Register the given class display data if the value is different than the specified default. - * - * @see DisplayData.Builder#add(String, ClassForDisplay) - */ - ItemBuilder addIfNotDefault( - String key, @Nullable ClassForDisplay value, @Nullable ClassForDisplay defaultValue); /** - * Register the given display data with the specified type. - * - *

The added display data is identified by the specified key and namespace from the current - * transform or component. + * {@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. * - * @throws ClassCastException if the value cannot be safely cast to the specified type. - * - * @see DisplayData#inferType(Object) + *

{@link Item Items} are registered via {@link DisplayData.Builder#add} + * within {@link HasDisplayData#populateDisplayData} implementations. */ - ItemBuilder add(String key, Type type, Object value); - } + @AutoValue + public abstract static class Item { - /** - * Utility to append optional fields to display data, or register additional display data - * 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. - * - *

Specifying a null value will clear the label if it was previously defined. - */ - ItemBuilder withLabel(@Nullable String label); - - /** - * Add a link URL to the most-recently added display data. 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. - */ - ItemBuilder withLinkUrl(@Nullable String url); - - /** - * Adds an explicit namespace to the most-recently added display data. The namespace - * and key uniquely identify the display data. - * - *

Leaving the namespace unspecified will default to the registering instance's class. + * The namespace for the display item. The namespace defaults to the component which + * the display item belongs to. */ - ItemBuilder withNamespace(Class namespace); + @Nullable + @JsonGetter("namespace") + public abstract String getNamespace(); /** - * Adds an explicit namespace to the most-recently added display data. The namespace - * and key uniquely identify the display data. - * - *

Leaving the namespace unspecified will default to the registering instance's class. + * The key for the display item. Each display item is created with a key and value + * via {@link DisplayData#item). */ - ItemBuilder withNamespace(ClassForDisplay namespace); - } - - /** - * A display data 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. - */ - public static class Item { - 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; - } - - @JsonGetter("namespace") - public String getNamespace() { - return ns; - } - @JsonGetter("key") - public String getKey() { - return key; - } + public abstract String getKey(); /** * 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; - } + public abstract Type getType(); /** - * 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") - public Object getValue() { - return value; - } + @Nullable + public abstract Object getValue(); /** - * 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. * - *

Some display data types will not provide a short value, in which case the return value - * will be null. + *

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. + * + * 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; - } + public abstract Object getShortValue(); /** * Retrieve the optional label for an item. The label is a human-readable description of what @@ -495,9 +266,7 @@ public Object getShortValue() { @JsonGetter("label") @JsonInclude(JsonInclude.Include.NON_NULL) @Nullable - public String getLabel() { - return label; - } + public abstract String getLabel(); /** * Retrieve the optional link URL for an item. The URL points to an address where the reader @@ -508,55 +277,89 @@ public String getLabel() { @JsonGetter("linkUrl") @JsonInclude(JsonInclude.Include.NON_NULL) @Nullable - public String getLinkUrl() { - return url; + public abstract String getLinkUrl(); + + private static Item create(String key, Type type, @Nullable T value) { + FormattedItemValue formatted = type.safeFormat(value); + return of(null, key, type, formatted.getLongValue(), formatted.getShortValue(), null, null); } - @Override - public String toString() { - return String.format("%s:%s=%s", ns, key, value); + /** + * 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(ClassForDisplay.of(namespace)); } - @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); - } + /** @see #withNamespace(Class) */ + private Item withNamespace(ClassForDisplay namespace) { + checkNotNull(namespace, "namesapce 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 of( + namespace, getKey(), getType(), getValue(), getShortValue(), getLabel(), getLinkUrl()); } - @Override - public int hashCode() { - return Objects.hash( - this.ns, - this.key, - this.type, - this.value, - this.shortValue, - this.label, - this.url); + /** + * 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 of( + 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 of(getNamespace(), getKey(), getType(), getValue(), getShortValue(), getLabel(), url); } - private Item withLabel(String label) { - return new Item(this.ns, this.key, this.type, this.value, this.shortValue, this.url, label); + /** + * 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 of(getNamespace(), getKey(), getType(), formatted.getLongValue(), + formatted.getShortValue(), getLabel(), getLinkUrl()); } - private Item withUrl(String url) { - return new Item(this.ns, this.key, this.type, this.value, this.shortValue, url, this.label); + private static Item of( + @Nullable String namespace, + String key, + Type type, + @Nullable Object value, + @Nullable Object shortValue, + @Nullable String label, + @Nullable String linkUrl) { + return new AutoValue_DisplayData_Item<>( + namespace, key, type, value, shortValue, label, 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()); } } @@ -696,6 +499,20 @@ private static T checkType(Object value, Class clazz, DisplayData.Type ex */ 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) { @@ -719,6 +536,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; @@ -740,15 +562,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(); @@ -762,28 +581,27 @@ 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); + checkNotNull(namespace, "Input namespace override cannot be null"); 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; @@ -796,202 +614,133 @@ 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); - } - - @Override - public ItemBuilder add(String key, Duration value) { - return addItemIf(true, key, Type.DURATION, value); - } - - @Override - public ItemBuilder addIfNotNull(String key, @Nullable Duration value) { - return addItemIf(value != null, key, Type.DURATION, value); - } - - @Override - public ItemBuilder addIfNotDefault( - String key, @Nullable Duration value, @Nullable Duration defaultValue) { - return addItemIf(!Objects.equals(value, defaultValue), key, Type.DURATION, value); + public Builder add(Item item) { + checkNotNull(item, "Input display item cannot be null"); + return addItemIf(true, item); } @Override - public ItemBuilder add(String key, Class value) { - return addItemIf(true, key, Type.JAVA_CLASS, value); + public Builder addIfNotNull(Item item) { + checkNotNull(item, "Input display item cannot be null"); + return addItemIf(item.getValue() != null, item); } @Override - public ItemBuilder add(String key, ClassForDisplay value) { - checkNotNull(value); - return addItemIf(true, key, Type.JAVA_CLASS, 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 addIfNotNull(String key, @Nullable Class value) { - return addItemIf(value != null, key, Type.JAVA_CLASS, value); - } + private Builder addItemIf(boolean condition, Item item) { + if (!condition) { + return this; + } - @Override - public ItemBuilder addIfNotNull(String key, @Nullable ClassForDisplay value) { - return addItemIf(value != null, 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 Class value, @Nullable Class 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 addIfNotDefault( - String key, @Nullable ClassForDisplay value, @Nullable ClassForDisplay defaultValue) { - return addItemIf(!Objects.equals(value, defaultValue), key, Type.JAVA_CLASS, value); + entries.put(id, item); + return this; } - @Override - public ItemBuilder add(String key, Type type, Object value) { - checkNotNull(type); - return addItemIf(true, key, type, value); + private DisplayData build() { + return new DisplayData(this.entries); } + } - 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; - } + /** + * 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); + } - private void commitLatest() { - if (latestItem == null) { - return; - } + /** + * 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); + } - 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 Long value) { + return item(key, Type.INTEGER, value); + } - entries.put(id, latestItem); - latestItem = null; - } + /** + * 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); + } - @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 Double value) { + return item(key, Type.FLOAT, value); + } - return this; - } + /** + * 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); + } - @Override - public ItemBuilder withLinkUrl(@Nullable String url) { - if (latestItem != null) { - latestItem = latestItem.withUrl(url); - } + /** + * 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); + } - return this; - } + /** + * 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(Class namespace) { - checkNotNull(namespace); - return withNamespace(ClassForDisplay.of(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); + } - @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 ClassForDisplay 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/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/HasDisplayData.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/HasDisplayData.java index b264a1468aad..d74944e55cab 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/HasDisplayData.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/HasDisplayData.java @@ -39,12 +39,12 @@ public interface HasDisplayData { * 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"); + * .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")); * } * } * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/CalendarWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/CalendarWindows.java index 3084059f81d2..490fbd1a41ec 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/CalendarWindows.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/CalendarWindows.java @@ -150,8 +150,9 @@ 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)) + .addIfNotDefault( + DisplayData.item("startDate", new DateTime(startDate, timeZone).toInstant()), new DateTime(DEFAULT_START_DATE, DateTimeZone.UTC).toInstant()); } @@ -247,8 +248,9 @@ 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)) + .addIfNotDefault( + DisplayData.item("startDate", new DateTime(startDate, timeZone).toInstant()), new DateTime(DEFAULT_START_DATE, DateTimeZone.UTC).toInstant()); } @@ -353,8 +355,9 @@ 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)) + .addIfNotDefault( + DisplayData.item("startDate", new DateTime(startDate, timeZone).toInstant()), new DateTime(DEFAULT_START_DATE, DateTimeZone.UTC).toInstant()); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/FixedWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/FixedWindows.java index fdffebe0fe4e..322259c990d0 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/FixedWindows.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/FixedWindows.java @@ -87,8 +87,8 @@ public IntervalWindow assignWindow(Instant timestamp) { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .add("size", size) - .addIfNotDefault("offset", offset, Duration.ZERO); + .add(DisplayData.item("size", size)) + .addIfNotDefault(DisplayData.item("offset", offset), Duration.ZERO); } @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Sessions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Sessions.java index 7010256a939f..8e8a00595912 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Sessions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Sessions.java @@ -101,7 +101,7 @@ public Duration getGapDuration() { @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - builder.add("gapDuration", gapDuration); + builder.add(DisplayData.item("gapDuration", gapDuration)); } @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java index bbbedad95989..4153e2176350 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java @@ -144,9 +144,9 @@ public boolean isCompatible(WindowFn other) { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .add("size", size) - .add("period", period) - .add("offset", offset); + .add(DisplayData.item("size", size)) + .add(DisplayData.item("period", period)) + .add(DisplayData.item("offset", offset)); } /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java index b751bece66b0..e9b34374740e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java @@ -600,24 +600,24 @@ public PCollection apply(PCollection input) { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .add("windowFn", windowFn.getClass()) + .add(DisplayData.item("windowFn", windowFn.getClass())) .include(windowFn) - .addIfNotNull("allowedLateness", allowedLateness); + .addIfNotNull(DisplayData.item("allowedLateness", allowedLateness)); if (trigger != null && !(trigger instanceof DefaultTrigger)) { - builder.add("trigger", trigger.toString()); + builder.add(DisplayData.item("trigger", trigger.toString())); } if (mode != null) { - builder.add("accumulationMode", mode.toString()); + builder.add(DisplayData.item("accumulationMode", mode.toString())); } if (closingBehavior != null) { - builder.add("closingBehavior", closingBehavior.toString()); + builder.add(DisplayData.item("closingBehavior", closingBehavior.toString())); } if (outputTimeFn != null) { - builder.add("outputTimeFn", outputTimeFn.getClass()); + builder.add(DisplayData.item("outputTimeFn", outputTimeFn.getClass())); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java index f7949e7bc0c2..774968f85ef9 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java @@ -75,7 +75,7 @@ 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")); } }; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java index 36ef4b33a795..76e547f84752 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java @@ -343,7 +343,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")); } }; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java index dd12bf43869a..237c025405d7 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java @@ -74,13 +74,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); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java index a95ef9535fa2..66cad6015ea2 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java @@ -164,7 +164,7 @@ 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); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java index df5a828c2443..486c738ceabe 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java @@ -340,8 +340,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/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java index e3308e586aa9..803daec08171 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java @@ -701,7 +701,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) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/IntraBundleParallelizationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/IntraBundleParallelizationTest.java index 4810513e4d7b..66b189a233ac 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/IntraBundleParallelizationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/IntraBundleParallelizationTest.java @@ -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")); } }; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java index fdf3ddfcfc93..9193267c2800 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java @@ -1537,7 +1537,7 @@ public void processElement(ProcessContext c) { @Override public void populateDisplayData(Builder builder) { - builder.add("doFnMetadata", "bar"); + builder.add(DisplayData.item("doFnMetadata", "bar")); } }; @@ -1560,7 +1560,7 @@ public void proccessElement(ProcessContext c) {} @Override public void populateDisplayData(Builder builder) { - builder.add("fnMetadata", "foobar"); + builder.add(DisplayData.item("fnMetadata", "foobar")); } }; @@ -1579,7 +1579,7 @@ public void proccessElement(ProcessContext c) {} @Override public void populateDisplayData(Builder builder) { - builder.add("fnMetadata", "foobar"); + builder.add(DisplayData.item("fnMetadata", "foobar")); } }; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchers.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchers.java index 90e4441f1dd9..c47b4c7182a1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchers.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchers.java @@ -47,7 +47,13 @@ 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(); + } + }; } /** @@ -119,14 +125,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; } @@ -138,7 +144,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"); @@ -205,8 +211,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) { @@ -220,19 +226,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); } } @@ -243,7 +249,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)); } @@ -251,10 +257,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(); } }; @@ -264,7 +270,7 @@ 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)); } @@ -272,11 +278,11 @@ public static Matcher hasNamespace(Class namespace) { * 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) { @@ -290,7 +296,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)); } @@ -298,11 +304,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(); } }; @@ -313,7 +319,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)); } @@ -321,11 +327,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/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchersTest.java index 8bb65cc7a9c0..f848c5ed3082 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchersTest.java @@ -24,8 +24,7 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasValue; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.core.StringStartsWith.startsWith; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; @@ -62,8 +61,8 @@ 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 @@ -82,7 +81,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)); } }); @@ -112,7 +111,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() { @@ -120,13 +119,13 @@ 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 = includesDisplayDataFrom(subComponent); @@ -152,7 +151,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/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java index 98604c7111e3..108409fbeecf 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java @@ -86,7 +86,7 @@ public void testTypicalUsage() { new HasDisplayData() { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("ExpectedAnswer", 42); + builder.add(DisplayData.item("ExpectedAnswer", 42)); } }; @@ -94,7 +94,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")); } }; @@ -109,14 +111,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("Mimimum 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")); } }; @@ -142,12 +144,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")); } }); @@ -162,11 +164,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()); @@ -178,15 +180,15 @@ 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]; + DisplayData.Item item = (DisplayData.Item) data.items().toArray()[0]; assertThat( item, Matchers.allOf( @@ -206,7 +208,7 @@ public void testUnspecifiedOptionalProperties() { new HasDisplayData() { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("foo", "bar"); + builder.add(DisplayData.item("foo", "bar")); } }); @@ -221,20 +223,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); } }); @@ -242,26 +250,59 @@ 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)) + .addIfNotDefault( + DisplayData.item("Class", DisplayDataTest.class), + DisplayDataTest.class) + .addIfNotDefault( + DisplayData.item("ClassForDisplay", ClassForDisplay.of(DisplayDataTest.class)), + ClassForDisplay.of(DisplayDataTest.class)); + } + }); + + 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)); } }); @@ -274,10 +315,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 shoudl be safe")); } }; @@ -291,7 +332,7 @@ public void testIncludes() { new HasDisplayData() { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("foo", "bar"); + builder.add(DisplayData.item("foo", "bar")); } }; @@ -312,7 +353,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")); } }; @@ -339,8 +380,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)); } }); } @@ -358,28 +399,9 @@ public void testIdentifierEquality() { @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(); } @@ -388,13 +410,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")); } }; @@ -415,11 +437,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( @@ -440,12 +462,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")); } }); } @@ -462,8 +484,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")); } }); } @@ -475,9 +497,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)); } }); @@ -489,7 +511,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")); } }; @@ -554,7 +576,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 @@ -574,7 +596,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(); @@ -588,18 +612,18 @@ 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("java_class2", ClassForDisplay.of(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( @@ -625,8 +649,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")); } }); @@ -690,7 +714,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")); } }; @@ -725,13 +749,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); @@ -751,7 +775,7 @@ public void testContextProperlyReset() { new HasDisplayData() { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("foo", "bar"); + builder.add(DisplayData.item("foo", "bar")); } }; @@ -761,7 +785,7 @@ public void populateDisplayData(DisplayData.Builder builder) { public void populateDisplayData(DisplayData.Builder builder) { builder .include(subComponent) - .add("alpha", "bravo"); + .add(DisplayData.item("alpha", "bravo")); } }; @@ -816,7 +840,7 @@ public void testNullKey() { new HasDisplayData() { @Override public void populateDisplayData(Builder builder) { - builder.add(null, "foo"); + builder.add(DisplayData.item(null, "foo")); } }); } @@ -828,28 +852,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 @@ -864,9 +888,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)); } }); @@ -885,16 +909,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); @@ -955,31 +978,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/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index f99eb1db93af..bf85dea6d330 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -266,14 +266,14 @@ public void validate(PBegin input) { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - builder.add("tableId", tableId); + builder.add(DisplayData.item("tableId", tableId)); if (options != null) { - builder.add("bigtableOptions", options.toString()); + builder.add(DisplayData.item("bigtableOptions", options.toString())); } if (filter != null) { - builder.add("rowFilter", filter.toString()); + builder.add(DisplayData.item("rowFilter", filter.toString())); } } @@ -448,10 +448,10 @@ Write withBigtableService(BigtableService bigtableService) { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - builder.add("tableId", tableId); + builder.add(DisplayData.item("tableId", tableId)); if (options != null) { - builder.add("bigtableOptions", options.toString()); + builder.add(DisplayData.item("bigtableOptions", options.toString())); } }