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 43489501e1ae..c26565983046 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 @@ -604,6 +604,7 @@ public DatastoreWriteOperation createWriteOperation(PipelineOptions options) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder .addIfNotDefault("host", host, DEFAULT_HOST) .addIfNotNull("dataset", datasetId); 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 1f41e5c35898..05b70ac8eb44 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 @@ -147,6 +147,7 @@ public String getKindString() { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder .add("source", source.getClass()) .include(source); @@ -261,6 +262,7 @@ public String getKindString() { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder .add("source", source.getClass()) .include(source); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Sink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Sink.java index 8b6b6379a992..20b1631306d5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Sink.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Sink.java @@ -139,7 +139,7 @@ public abstract class Sink implements Serializable, HasDisplayData { * {@inheritDoc} * *

By default, does not register any display data. Implementors may override this method - * to provide their own display metadata. + * to provide their own display data. */ @Override public void populateDisplayData(DisplayData.Builder builder) {} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java index 2ab0d4e42063..b8902f92aeda 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java @@ -72,7 +72,7 @@ public abstract class Source implements Serializable, HasDisplayData { * {@inheritDoc} * *

By default, does not register any display data. Implementors may override this method - * to provide their own display metadata. + * to provide their own display data. */ @Override public void populateDisplayData(DisplayData.Builder builder) {} diff --git a/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 b8fa2594d258..a7d182da1e89 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 @@ -79,6 +79,7 @@ public PDone apply(PCollection input) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder .add("sink", sink.getClass()) .include(sink); 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 c58c7368d36e..dd99b9acbc50 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 @@ -363,6 +363,7 @@ public Coder> getAccumulatorCoder( @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder .add("numQuantiles", numQuantiles) .add("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 175897b08a5e..46bad0cf9368 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 @@ -216,6 +216,7 @@ public PCollection apply(PCollection input) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); ApproximateUnique.populateDisplayData(builder, sampleSize, maximumEstimationError); } } @@ -289,6 +290,7 @@ public PCollection> apply(PCollection> input) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); ApproximateUnique.populateDisplayData(builder, sampleSize, maximumEstimationError); } } 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 3566fa52533e..b13b0fc62079 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 @@ -515,6 +515,7 @@ public CombineFn forKey(K key, Coder keyCoder) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); CombineFn.this.populateDisplayData(builder); } }; @@ -1193,6 +1194,7 @@ public Coder getDefaultOutputCoder( @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); KeyedCombineFn.this.populateDisplayData(builder); } }; @@ -1378,6 +1380,8 @@ public PCollection apply(PCollection input) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + Combine.populateDisplayData(builder, fn, fnClass); Combine.populateGlobalDisplayData(builder, fanout, insertDefault); } @@ -1507,6 +1511,8 @@ public boolean getInsertDefault() { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + Combine.populateDisplayData(builder, fn, fnClass); Combine.populateGlobalDisplayData(builder, fanout, insertDefault); } @@ -1591,6 +1597,7 @@ public List compact(List accumulator) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder.add("combineFn", combiner.getClass()); } @@ -1772,6 +1779,7 @@ public PCollection> apply(PCollection> input) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); Combine.populateDisplayData(builder, fn, fnClass); } } @@ -2077,6 +2085,8 @@ public void processElement(ProcessContext c) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + Combine.populateDisplayData(builder, fn, fnClass); builder.add("fanoutFn", hotKeyFanout.getClass()); } @@ -2333,6 +2343,7 @@ public Coder> getDefaultOutputCoder( @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); Combine.populateDisplayData(builder, fn, fnClass); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFnBase.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFnBase.java index 1b64bb24434b..c73ba542c359 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFnBase.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFnBase.java @@ -225,7 +225,7 @@ public TypeVariable getOutputTVariable() { * {@inheritDoc} * *

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

By default, does not register any display data. Implementors may override this method - * to provide their own display metadata. + * to provide their own display data. */ @Override public void populateDisplayData(DisplayData.Builder builder) { diff --git a/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 ed454987e8ae..a9500d84b152 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 @@ -464,6 +464,7 @@ public Coder getAccumulatorCoder(CoderRegistry registry, Coder @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); CombineFns.populateDisplayData(builder, combineFns); } } @@ -602,6 +603,7 @@ public Coder getAccumulatorCoder(CoderRegistry registry, Coder @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); CombineFns.populateDisplayData(builder, combineFnWithContexts); } } @@ -788,6 +790,7 @@ public Coder getAccumulatorCoder( @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); CombineFns.populateDisplayData(builder, keyedCombineFns); } } @@ -939,6 +942,7 @@ public Coder getAccumulatorCoder( @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); CombineFns.populateDisplayData(builder, keyedCombineFns); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineWithContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineWithContext.java index 9bb4a01f1b2b..9722360b2e71 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineWithContext.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineWithContext.java @@ -171,6 +171,7 @@ public CombineFnWithContext forKey(K key, Coder keyC @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); CombineFnWithContext.this.populateDisplayData(builder); } }; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java index 04f272d525dc..6d5d1edb0c1e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java @@ -374,7 +374,7 @@ public void finishBundle(Context c) throws Exception { * {@inheritDoc} * *

By default, does not register any display data. Implementors may override this method - * to provide their own display metadata. + * to provide their own display data. */ @Override public void populateDisplayData(DisplayData.Builder builder) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnWithContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnWithContext.java index 7143626ba4db..8a83e441a6fd 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnWithContext.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnWithContext.java @@ -421,7 +421,7 @@ void prepareForProcessing() { * {@inheritDoc} * *

By default, does not register any display data. Implementors may override this method - * to provide their own display metadata. + * to provide their own display data. */ @Override public void populateDisplayData(DisplayData.Builder builder) { diff --git a/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 0e5e4a62cb2e..0108958ed3dc 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 @@ -103,6 +103,7 @@ public void processElement(ProcessContext c) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); Filter.populateDisplayData(builder, String.format("x < %s", value)); } }); @@ -141,6 +142,7 @@ public void processElement(ProcessContext c) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); Filter.populateDisplayData(builder, String.format("x > %s", value)); } }); @@ -178,6 +180,7 @@ public void processElement(ProcessContext c) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); Filter.populateDisplayData(builder, String.format("x ≤ %s", value)); } }); @@ -215,6 +218,7 @@ public void processElement(ProcessContext c) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); Filter.populateDisplayData(builder, String.format("x ≥ %s", value)); } }); 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 1b3c4542d6d8..0e46a2ad4edd 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 @@ -277,6 +277,7 @@ public static KvCoder> getOutputKvCoder(Coder> in @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); if (fewKeys) { builder.add("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 1b915629ab86..36149caf51c8 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 @@ -176,6 +176,7 @@ public PCollection apply(PCollection input) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder .add("maxParallelism", maxParallelism) .add("fn", doFn.getClass()) 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 28749d731e2a..96900b4e0108 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 @@ -208,6 +208,7 @@ public T apply(T left, T right) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder.add("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 8f3082ee4dbd..7d69c5aad571 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 @@ -208,6 +208,7 @@ public T apply(T left, T right) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder.add("comparer", comparator.getClass()); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java index d502b93a1d02..4bcfb2986208 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java @@ -317,7 +317,7 @@ public Coder getDefaultOutputCoder( * {@inheritDoc} * *

By default, does not register any display data. Implementors may override this method - * to provide their own display metadata. + * to provide their own display data. */ @Override public void populateDisplayData(Builder builder) { diff --git a/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 02464ac2bc5e..547486dfd7fa 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 @@ -808,12 +808,13 @@ protected String getKindString() { /** * {@inheritDoc} * - *

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

{@link ParDo} registers its internal {@link DoFn} as a subcomponent for display data. * {@link DoFn} implementations can register display data by overriding * {@link DoFn#populateDisplayData}. */ @Override public void populateDisplayData(Builder builder) { + super.populateDisplayData(builder); ParDo.populateDisplayData(builder, fn, fnClass); } @@ -1051,6 +1052,7 @@ protected String getKindString() { @Override public void populateDisplayData(Builder builder) { + super.populateDisplayData(builder); ParDo.populateDisplayData(builder, 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 5366fd0c6f83..fbdb9bec189f 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 @@ -124,6 +124,7 @@ public PCollectionList apply(PCollection in) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder.include(partitionDoFn); } @@ -179,6 +180,7 @@ public void processElement(ProcessContext c) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder .add("numPartitions", numPartitions) .add("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 6362bd43a465..2fba5e37642a 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 @@ -157,6 +157,7 @@ public PCollection apply(PCollection in) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder.add("sampleSize", limit); } } @@ -256,6 +257,7 @@ public Coder> getDefaultOutputCoder( @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder.add("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 4b366bc22d63..2d38e04ec1ef 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 @@ -395,6 +395,7 @@ public Coder> getAccumulatorCoder( @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder .add("count", count) .add("comparer", compareFn.getClass()); 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 455d6e19ad98..3e31c1e09e81 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 @@ -23,9 +23,9 @@ import java.util.Objects; /** - * Display metadata representing a Java class. + * Display data representing a Java class. * - *

Java classes can be registered as display metadata via + *

Java classes can be registered as display data via * {@link DisplayData.Builder#add(String, ClassForDisplay)}. {@link ClassForDisplay} is * serializable, unlike {@link Class} which can fail to serialize for Java 8 lambda functions. */ 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 a1037a8db0d1..c119057f5eb6 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 @@ -44,7 +44,7 @@ import java.util.Set; /** - * Static display metadata associated with a pipeline component. Display data is useful for + * Static display data associated with a pipeline component. Display data is useful for * pipeline runner UIs and diagnostic dashboards to display details about * {@link PTransform PTransforms} that make up a pipeline. * @@ -148,41 +148,41 @@ private static String namespaceOf(ClassForDisplay clazz) { } /** - * Utility to build up display metadata from a component and its included + * Utility to build up display data from a component and its included * subcomponents. */ public interface Builder { /** - * Register display metadata from the specified subcomponent. + * Register display data from the specified subcomponent. * * @see #include(HasDisplayData, String) */ Builder include(HasDisplayData subComponent); /** - * Register display metadata from the specified subcomponent, using the specified namespace. + * Register display data from the specified subcomponent, using the specified namespace. * * @see #include(HasDisplayData, String) */ Builder include(HasDisplayData subComponent, Class namespace); /** - * Register display metadata from the specified subcomponent, using the specified namespace. + * Register display data from the specified subcomponent, using the specified namespace. * * @see #include(HasDisplayData, String) */ Builder include(HasDisplayData subComponent, ClassForDisplay namespace); /** - * Register display metadata from the specified subcomponent, using the specified namespace. + * Register display data from the specified subcomponent, using the specified namespace. * - *

For example, a {@link ParDo} transform includes display metadata from the encapsulated + *

For example, a {@link ParDo} transform includes display data from the encapsulated * {@link DoFn}. */ Builder include(HasDisplayData subComponent, String namespace); /** - * Register the given string display metadata. The metadata item will be registered with type + * 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. */ @@ -203,7 +203,7 @@ public interface Builder { ItemBuilder addIfNotDefault(String key, @Nullable String value, @Nullable String defaultValue); /** - * Register the given numeric display metadata. The metadata item will be registered with type + * 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. */ @@ -224,7 +224,7 @@ public interface Builder { ItemBuilder addIfNotDefault(String key, long value, long defaultValue); /** - * Register the given floating point display metadata. The metadata item will be registered with + * 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. */ @@ -246,7 +246,7 @@ public interface Builder { ItemBuilder addIfNotDefault(String key, double value, double defaultValue); /** - * Register the given boolean display metadata. The metadata item will be registered with + * 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. */ @@ -267,7 +267,7 @@ public interface Builder { ItemBuilder addIfNotDefault(String key, boolean value, boolean defaultValue); /** - * Register the given timestamp display metadata. The metadata item will be registered with type + * 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. */ @@ -290,7 +290,7 @@ ItemBuilder addIfNotDefault( String key, @Nullable Instant value, @Nullable Instant defaultValue); /** - * Register the given duration display metadata. The metadata item will be registered with type + * 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. */ @@ -313,14 +313,14 @@ ItemBuilder addIfNotDefault( String key, @Nullable Duration value, @Nullable Duration defaultValue); /** - * Register the given class display metadata. The metadata item will be registered with type + * 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 metadata. The metadata item will be registered with type + * 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. */ @@ -356,7 +356,7 @@ ItemBuilder addIfNotDefault( ItemBuilder addIfNotDefault( String key, @Nullable ClassForDisplay value, @Nullable ClassForDisplay defaultValue); /** - * Register the given display metadata with the specified type. + * 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. @@ -369,7 +369,7 @@ ItemBuilder addIfNotDefault( } /** - * Utility to append optional fields to display metadata, or register additional display metadata + * Utility to append optional fields to display data, or register additional display data * items. */ public interface ItemBuilder extends Builder { @@ -383,7 +383,7 @@ public interface ItemBuilder extends Builder { ItemBuilder withLabel(@Nullable String label); /** - * Add a link URL to the most-recently added display metadata. A link URL is optional and + * 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. @@ -391,16 +391,16 @@ public interface ItemBuilder extends Builder { ItemBuilder withLinkUrl(@Nullable String url); /** - * Adds an explicit namespace to the most-recently added display metadata. The namespace - * and key uniquely identify the display metadata. + * 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. */ ItemBuilder withNamespace(Class namespace); /** - * Adds an explicit namespace to the most-recently added display metadata. The namespace - * and key uniquely identify the display metadata. + * 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. */ @@ -408,7 +408,7 @@ public interface ItemBuilder extends Builder { } /** - * A display metadata item. DisplayData items are registered via {@link Builder#add} within + * 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. @@ -456,7 +456,7 @@ public String getKey() { } /** - * Retrieve the {@link DisplayData.Type} of display metadata. All metadata conforms to a + * Retrieve the {@link DisplayData.Type} of display data. All metadata conforms to a * predefined set of allowed types. */ @JsonGetter("type") @@ -501,7 +501,7 @@ public String getLabel() { /** * Retrieve the optional link URL for an item. The URL points to an address where the reader - * can find additional context for the display metadata. + * can find additional context for the display data. * *

If no URL was specified, this will return {@code null}. */ @@ -561,11 +561,11 @@ private Item withNamespace(ClassForDisplay nsClass) { } /** - * Unique identifier for a display metadata item within a component. + * Unique identifier for a display data item within a component. * Identifiers are composed of the key they are registered with and a namespace generated from * the class of the component which registered the item. * - *

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

Display data registered with the same key from different components will have different * namespaces and thus will both be represented in the composed {@link DisplayData}. If a * single component registers multiple metadata items with the same key, only the most recent * item will be retained; previous versions are discarded. @@ -618,7 +618,7 @@ public String toString() { } /** - * Display metadata type. + * Display data type. */ public enum Type { STRING { @@ -689,7 +689,7 @@ private static T checkType(Object value, Class clazz, DisplayData.Type ex } /** - * Format the display metadata value into a long string representation, and optionally + * Format the display data value into a long string representation, and optionally * a shorter representation for display. * *

Internal-only. Value objects can be safely cast to the expected Java type. 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 b295d978f083..b264a1468aad 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 @@ -21,16 +21,16 @@ /** * Marker interface for {@link PTransform PTransforms} and components used within - * {@link PTransform PTransforms} to specify display metadata to be used within UIs and diagnostic + * {@link PTransform PTransforms} to specify display data to be used within UIs and diagnostic * tools. * - *

Display metadata is optional and may be collected during pipeline construction. It should + *

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

@@ -49,7 +49,7 @@ public interface HasDisplayData {
    * }
    * 
* - * @param builder The builder to populate with display metadata. + * @param builder The builder to populate with display data. */ void populateDisplayData(DisplayData.Builder builder); } 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 86565370eef6..3084059f81d2 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 @@ -147,6 +147,8 @@ public boolean isCompatible(WindowFn other) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder .add("numDays", number) .addIfNotDefault("startDate", new DateTime(startDate, timeZone).toInstant(), @@ -242,6 +244,8 @@ public boolean isCompatible(WindowFn other) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder .add("numMonths", number) .addIfNotDefault("startDate", new DateTime(startDate, timeZone).toInstant(), @@ -346,6 +350,8 @@ public boolean isCompatible(WindowFn other) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder .add("numYears", number) .addIfNotDefault("startDate", new DateTime(startDate, timeZone).toInstant(), 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 bba1f3beb68c..fdffebe0fe4e 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 @@ -85,6 +85,7 @@ public IntervalWindow assignWindow(Instant timestamp) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder .add("size", size) .addIfNotDefault("offset", offset, Duration.ZERO); 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 74ca26800066..7010256a939f 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 @@ -100,6 +100,7 @@ public Duration getGapDuration() { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder.add("gapDuration", gapDuration); } 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 abb407849089..bbbedad95989 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 @@ -142,6 +142,7 @@ public boolean isCompatible(WindowFn other) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder .add("size", size) .add("period", period) 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 da512b815907..b751bece66b0 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 @@ -598,6 +598,7 @@ public PCollection apply(PCollection input) { @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); builder .add("windowFn", windowFn.getClass()) .include(windowFn) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowFn.java index 2eac93612124..e291beef76a2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowFn.java @@ -184,7 +184,7 @@ public boolean assignsToSingleWindow() { * {@inheritDoc} * *

By default, does not register any display data. Implementors may override this method - * to provide their own display metadata. + * to provide their own display data. */ @Override public void populateDisplayData(DisplayData.Builder builder) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java index 34197f78f8fb..351a158c1edf 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java @@ -104,6 +104,7 @@ public Coder getDefaultOutputCoder( } @Override public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); combineFn.populateDisplayData(builder); } }; 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 30f3eca29738..f7949e7bc0c2 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 @@ -17,7 +17,8 @@ */ package org.apache.beam.sdk.io; -import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includes; +import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; + import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; @@ -79,7 +80,7 @@ public void populateDisplayData(DisplayData.Builder builder) { }; BoundedReadFromUnboundedSource> read = Read.from(src).withMaxNumRecords(5); - assertThat(DisplayData.from(read), includes(src)); + assertThat(DisplayData.from(read), includesDisplayDataFrom(src)); } private static class Checker diff --git a/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 4ad89040939c..36ef4b33a795 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 @@ -19,7 +19,8 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasKey; -import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includes; +import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; + import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.instanceOf; import static org.junit.Assert.assertFalse; @@ -355,7 +356,7 @@ public void populateDisplayData(DisplayData.Builder builder) { assertThat(compressedSourceDisplayData, hasDisplayItem(hasKey("compressionMode"))); assertThat(gzipDisplayData, hasDisplayItem("compressionMode", CompressionMode.GZIP.toString())); assertThat(compressedSourceDisplayData, hasDisplayItem("source", inputSource.getClass())); - assertThat(compressedSourceDisplayData, includes(inputSource)); + assertThat(compressedSourceDisplayData, includesDisplayDataFrom(inputSource)); } /** diff --git a/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 bf18a3376f78..dd12bf43869a 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 @@ -18,7 +18,7 @@ package org.apache.beam.sdk.io; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; -import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includes; +import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; import static org.hamcrest.MatcherAssert.assertThat; @@ -92,11 +92,11 @@ public void populateDisplayData(DisplayData.Builder builder) { DisplayData boundedDisplayData = DisplayData.from(bounded); assertThat(boundedDisplayData, hasDisplayItem("source", boundedSource.getClass())); - assertThat(boundedDisplayData, includes(boundedSource)); + assertThat(boundedDisplayData, includesDisplayDataFrom(boundedSource)); DisplayData unboundedDisplayData = DisplayData.from(unbounded); assertThat(unboundedDisplayData, hasDisplayItem("source", unboundedSource.getClass())); - assertThat(unboundedDisplayData, includes(unboundedSource)); + assertThat(unboundedDisplayData, includesDisplayDataFrom(unboundedSource)); assertThat(unboundedDisplayData, hasDisplayItem("maxRecords", 1234)); assertThat(unboundedDisplayData, hasDisplayItem("maxReadTime", maxReadTime)); } diff --git a/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 6e66faf932f4..a95ef9535fa2 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 @@ -18,7 +18,8 @@ package org.apache.beam.sdk.io; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; -import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includes; +import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; + import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; @@ -170,7 +171,7 @@ public void populateDisplayData(DisplayData.Builder builder) { DisplayData displayData = DisplayData.from(write); assertThat(displayData, hasDisplayItem("sink", sink.getClass())); - assertThat(displayData, includes(sink)); + assertThat(displayData, includesDisplayDataFrom(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 e66f13a5611d..df5a828c2443 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 @@ -18,7 +18,7 @@ package org.apache.beam.sdk.transforms; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; -import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includes; +import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; import static org.junit.Assert.assertThat; @@ -303,8 +303,8 @@ public String apply(String input) { assertThat(displayData, hasDisplayItem("combineFn2", combineFn2.getClass())); String nsBase = DisplayDataCombineFn.class.getName(); - assertThat(displayData, includes(combineFn1, nsBase + "#1")); - assertThat(displayData, includes(combineFn2, nsBase + "#2")); + assertThat(displayData, includesDisplayDataFrom(combineFn1, nsBase + "#1")); + assertThat(displayData, includesDisplayDataFrom(combineFn2, nsBase + "#2")); } private static class DisplayDataCombineFn extends Combine.CombineFn { 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 9fa148ea6501..e3308e586aa9 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 @@ -19,7 +19,7 @@ import static org.apache.beam.sdk.TestUtils.checkCombineFn; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; -import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includes; +import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; import static com.google.common.base.Preconditions.checkNotNull; @@ -711,7 +711,7 @@ public void populateDisplayData(DisplayData.Builder builder) { assertThat(displayData, hasDisplayItem("combineFn", combineFn.getClass())); assertThat(displayData, hasDisplayItem("emitDefaultOnEmptyInput", true)); assertThat(displayData, hasDisplayItem("fanout", 1234)); - assertThat(displayData, includes(combineFn)); + assertThat(displayData, includesDisplayDataFrom(combineFn)); } //////////////////////////////////////////////////////////////////////////// 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 80f6188c1cb6..4810513e4d7b 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 @@ -19,7 +19,7 @@ import static org.apache.beam.sdk.testing.SystemNanoTimeSleeper.sleepMillis; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; -import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includes; +import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; import static org.hamcrest.Matchers.both; import static org.hamcrest.Matchers.containsString; @@ -236,7 +236,7 @@ public void populateDisplayData(DisplayData.Builder builder) { .of(fn); DisplayData displayData = DisplayData.from(transform); - assertThat(displayData, includes(fn)); + assertThat(displayData, includesDisplayDataFrom(fn)); assertThat(displayData, hasDisplayItem("fn", fn.getClass())); assertThat(displayData, hasDisplayItem("maxParallelism", 1234)); } diff --git a/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 44154e62e4f0..fdf3ddfcfc93 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 @@ -20,7 +20,7 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasKey; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasType; -import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includes; +import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; import static org.apache.beam.sdk.util.SerializableUtils.serializeToByteArray; import static org.apache.beam.sdk.util.StringUtils.byteArrayToJsonString; import static org.apache.beam.sdk.util.StringUtils.jsonStringToByteArray; @@ -1549,7 +1549,7 @@ public void populateDisplayData(Builder builder) { hasType(DisplayData.Type.JAVA_CLASS), DisplayDataMatchers.hasValue(fn.getClass().getName())))); - assertThat(displayData, includes(fn)); + assertThat(displayData, includesDisplayDataFrom(fn)); } @Test @@ -1567,7 +1567,7 @@ public void populateDisplayData(Builder builder) { Bound parDo = ParDo.of(fn); DisplayData displayData = DisplayData.from(parDo); - assertThat(displayData, includes(fn)); + assertThat(displayData, includesDisplayDataFrom(fn)); assertThat(displayData, hasDisplayItem("fn", fn.getClass())); } @@ -1588,7 +1588,7 @@ public void populateDisplayData(Builder builder) { .of(fn); DisplayData displayData = DisplayData.from(parDo); - assertThat(displayData, includes(fn)); + assertThat(displayData, includesDisplayDataFrom(fn)); assertThat(displayData, hasDisplayItem("fn", fn.getClass())); } } diff --git a/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 abdc350cddd7..90e4441f1dd9 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 @@ -149,22 +149,22 @@ protected boolean matchesSafely(DisplayData data, Description mismatchDescriptio } } - /** @see #includes(HasDisplayData, String) */ - public static Matcher includes(HasDisplayData subComponent) { - return includes(subComponent, subComponent.getClass()); + /** @see #includesDisplayDataFrom(HasDisplayData, String) */ + public static Matcher includesDisplayDataFrom(HasDisplayData subComponent) { + return includesDisplayDataFrom(subComponent, subComponent.getClass()); } - /** @see #includes(HasDisplayData, String) */ - public static Matcher includes( + /** @see #includesDisplayDataFrom(HasDisplayData, String) */ + public static Matcher includesDisplayDataFrom( HasDisplayData subComponent, Class namespace) { - return includes(subComponent, namespace.getName()); + return includesDisplayDataFrom(subComponent, namespace.getName()); } /** * Create a matcher that matches if the examined {@link DisplayData} contains all display data * registered from the specified subcomponent and namespace. */ - public static Matcher includes( + public static Matcher includesDisplayDataFrom( final HasDisplayData subComponent, final String namespace) { return new CustomTypeSafeMatcher("includes subcomponent") { @Override 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 5165e1ac706c..8bb65cc7a9c0 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 @@ -22,7 +22,7 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasNamespace; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasType; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasValue; -import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includes; +import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.core.StringStartsWith.startsWith; @@ -129,7 +129,7 @@ public void populateDisplayData(Builder builder) { builder.add("foo", "bar"); } }; - Matcher matcher = includes(subComponent); + Matcher matcher = includesDisplayDataFrom(subComponent); assertFalse(matcher.matches(DisplayData.from(sameKeyDifferentNamespace))); assertThat(DisplayData.from(hasSubcomponent), matcher); 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 05d0f6f0a046..98604c7111e3 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 @@ -22,7 +22,7 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasNamespace; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasType; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasValue; -import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includes; +import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.empty; @@ -304,7 +304,7 @@ public void populateDisplayData(DisplayData.Builder builder) { } }); - assertThat(data, includes(subComponent)); + assertThat(data, includesDisplayDataFrom(subComponent)); } @Test @@ -329,7 +329,7 @@ public void populateDisplayData(DisplayData.Builder builder) { } }); - assertThat(data, includes(subComponent, namespaceOverride.getClass())); + assertThat(data, includesDisplayDataFrom(subComponent, namespaceOverride.getClass())); } @Test diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java index 6be6df819a91..91bd84656ddf 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java @@ -19,7 +19,7 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasKey; -import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includes; +import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.isOneOf; @@ -250,7 +250,7 @@ public void testDisplayData() { DisplayData displayData = DisplayData.from(window); assertThat(displayData, hasDisplayItem("windowFn", windowFn.getClass())); - assertThat(displayData, includes(windowFn)); + assertThat(displayData, includesDisplayDataFrom(windowFn)); assertThat(displayData, hasDisplayItem("trigger", triggerBuilder.toString())); assertThat(displayData,