From 4e25182f419b8a25505f6d686789389ad08ffe13 Mon Sep 17 00:00:00 2001 From: Scott Wegner Date: Wed, 27 Apr 2016 13:08:49 -0700 Subject: [PATCH 1/7] Refactor Combine display data to not use ClassForDisplay --- .../apache/beam/sdk/transforms/Combine.java | 158 +++++++++--------- .../sdk/transforms/display/DisplayData.java | 5 +- .../beam/sdk/transforms/CombineTest.java | 16 ++ 3 files changed, 102 insertions(+), 77 deletions(-) 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 68fc1cf5b14e..f3a7c0360100 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 @@ -36,7 +36,6 @@ import org.apache.beam.sdk.transforms.CombineWithContext.Context; import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext; import org.apache.beam.sdk.transforms.CombineWithContext.RequiresContextInternal; -import org.apache.beam.sdk.transforms.display.ClassForDisplay; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.HasDisplayData; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; @@ -103,7 +102,7 @@ private Combine() { */ public static Globally globally( SerializableFunction, V> combiner) { - return globally(IterableCombineFn.of(combiner), ClassForDisplay.fromInstance(combiner)); + return globally(IterableCombineFn.of(combiner), displayDataForFn(combiner)); } @@ -124,12 +123,17 @@ public static Globally globally( */ public static Globally globally( GlobalCombineFn fn) { - return globally(fn, ClassForDisplay.fromInstance(fn)); + return globally(fn, displayDataForFn(fn)); + } + + private static DisplayData.Item> displayDataForFn(T fn) { + return DisplayData.item("combineFn", fn.getClass()); } private static Globally globally( - GlobalCombineFn fn, ClassForDisplay fnClass) { - return new Globally<>(fn, fnClass, true, 0); + GlobalCombineFn fn, + DisplayData.Item> fnDisplayData) { + return new Globally<>(fn, fnDisplayData, true, 0); } /** @@ -150,7 +154,7 @@ private static Globally globally( */ public static PerKey perKey( SerializableFunction, V> fn) { - return perKey(IterableCombineFn.of(fn).asKeyedFn(), ClassForDisplay.fromInstance(fn)); + return perKey(IterableCombineFn.of(fn).asKeyedFn(), displayDataForFn(fn)); } /** @@ -171,7 +175,7 @@ public static PerKey perKey( */ public static PerKey perKey( GlobalCombineFn fn) { - return perKey(fn.asKeyedFn(), ClassForDisplay.fromInstance(fn)); + return perKey(fn.asKeyedFn(), displayDataForFn(fn)); } /** @@ -192,12 +196,13 @@ public static PerKey perKey( */ public static PerKey perKey( PerKeyCombineFn fn) { - return perKey(fn, ClassForDisplay.fromInstance(fn)); + return perKey(fn, displayDataForFn(fn)); } private static PerKey perKey( - PerKeyCombineFn fn, ClassForDisplay fnClass) { - return new PerKey<>(fn, fnClass, false /*fewKeys*/); + PerKeyCombineFn fn, + DisplayData.Item> fnDisplayData) { + return new PerKey<>(fn, fnDisplayData, false /*fewKeys*/); } /** @@ -205,8 +210,9 @@ private static PerKey perKey( * in {@link GroupByKey}. */ private static PerKey fewKeys( - PerKeyCombineFn fn, ClassForDisplay fnClass) { - return new PerKey<>(fn, fnClass, true /*fewKeys*/); + PerKeyCombineFn fn, + DisplayData.Item> fnDisplayData) { + return new PerKey<>(fn, fnDisplayData, true /*fewKeys*/); } /** @@ -232,7 +238,7 @@ private static PerKey fewKeys( */ public static GroupedValues groupedValues( SerializableFunction, V> fn) { - return groupedValues(IterableCombineFn.of(fn).asKeyedFn(), ClassForDisplay.fromInstance(fn)); + return groupedValues(IterableCombineFn.of(fn).asKeyedFn(), displayDataForFn(fn)); } /** @@ -258,7 +264,7 @@ public static GroupedValues groupedValues( */ public static GroupedValues groupedValues( GlobalCombineFn fn) { - return groupedValues(fn.asKeyedFn(), ClassForDisplay.fromInstance(fn)); + return groupedValues(fn.asKeyedFn(), displayDataForFn(fn)); } /** @@ -284,12 +290,13 @@ public static GroupedValues groupedValu */ public static GroupedValues groupedValues( PerKeyCombineFn fn) { - return groupedValues(fn, ClassForDisplay.fromInstance(fn)); + return groupedValues(fn, displayDataForFn(fn)); } private static GroupedValues groupedValues( - PerKeyCombineFn fn, ClassForDisplay fnClass) { - return new GroupedValues<>(fn, fnClass); + PerKeyCombineFn fn, + DisplayData.Item> fnDisplayData) { + return new GroupedValues<>(fn, fnDisplayData); } ///////////////////////////////////////////////////////////////////////////// @@ -431,7 +438,7 @@ public AccumT compact(AccumT accumulator) { * *

Useful when using a {@code CombineFn} separately from a * {@code Combine} transform. Does not invoke the - * {@link mergeAccumulators} operation. + * {@link #mergeAccumulators} operation. */ public OutputT apply(Iterable inputs) { AccumT accum = createAccumulator(); @@ -515,8 +522,7 @@ public CombineFn forKey(K key, Coder keyCoder) { @Override public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - CombineFn.this.populateDisplayData(builder); + builder.include(CombineFn.this); } }; } @@ -1194,8 +1200,7 @@ public Coder getDefaultOutputCoder( @Override public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - KeyedCombineFn.this.populateDisplayData(builder); + builder.include(KeyedCombineFn.this); } }; } @@ -1262,36 +1267,36 @@ public static class Globally extends PTransform, PCollection> { private final GlobalCombineFn fn; - private final ClassForDisplay fnClass; + private final DisplayData.Item> fnDisplayData; private final boolean insertDefault; private final int fanout; private final List> sideInputs; - private Globally(GlobalCombineFn fn, ClassForDisplay fnClass, - boolean insertDefault, int fanout) { + private Globally(GlobalCombineFn fn, + DisplayData.Item> fnDisplayData, boolean insertDefault, int fanout) { this.fn = fn; - this.fnClass = fnClass; + this.fnDisplayData = fnDisplayData; this.insertDefault = insertDefault; this.fanout = fanout; - this.sideInputs = ImmutableList.>of(); + this.sideInputs = ImmutableList.of(); } private Globally(String name, GlobalCombineFn fn, - ClassForDisplay fnClass, boolean insertDefault, int fanout) { + DisplayData.Item> fnDisplayData, boolean insertDefault, int fanout) { super(name); this.fn = fn; - this.fnClass = fnClass; + this.fnDisplayData = fnDisplayData; this.insertDefault = insertDefault; this.fanout = fanout; - this.sideInputs = ImmutableList.>of(); + this.sideInputs = ImmutableList.of(); } private Globally(String name, GlobalCombineFn fn, - ClassForDisplay fnClass, boolean insertDefault, int fanout, + DisplayData.Item> fnDisplayData, boolean insertDefault, int fanout, List> sideInputs) { super(name); this.fn = fn; - this.fnClass = fnClass; + this.fnDisplayData = fnDisplayData; this.insertDefault = insertDefault; this.fanout = fanout; this.sideInputs = sideInputs; @@ -1302,7 +1307,7 @@ private Globally(String name, GlobalCombineFn fn, * specified name. Does not modify this transform. */ public Globally named(String name) { - return new Globally<>(name, fn, fnClass, insertDefault, fanout); + return new Globally<>(name, fn, fnDisplayData, insertDefault, fanout); } /** @@ -1313,7 +1318,7 @@ public Globally named(String name) { * to an empty input set will be returned. */ public GloballyAsSingletonView asSingletonView() { - return new GloballyAsSingletonView<>(fn, fnClass, insertDefault, fanout); + return new GloballyAsSingletonView<>(fn, fnDisplayData, insertDefault, fanout); } /** @@ -1322,7 +1327,7 @@ public GloballyAsSingletonView asSingletonView() { * is not globally windowed and the output is not being used as a side input. */ public Globally withoutDefaults() { - return new Globally<>(name, fn, fnClass, false, fanout); + return new Globally<>(name, fn, fnDisplayData, false, fanout); } /** @@ -1333,7 +1338,7 @@ public Globally withoutDefaults() { * that will be used. */ public Globally withFanout(int fanout) { - return new Globally<>(name, fn, fnClass, insertDefault, fanout); + return new Globally<>(name, fn, fnDisplayData, insertDefault, fanout); } /** @@ -1343,8 +1348,8 @@ public Globally withFanout(int fanout) { public Globally withSideInputs( Iterable> sideInputs) { Preconditions.checkState(fn instanceof RequiresContextInternal); - return new Globally(name, fn, fnClass, insertDefault, fanout, - ImmutableList.>copyOf(sideInputs)); + return new Globally<>(name, fn, fnDisplayData, insertDefault, fanout, + ImmutableList.copyOf(sideInputs)); } @Override @@ -1354,7 +1359,7 @@ public PCollection apply(PCollection input) { .setCoder(KvCoder.of(VoidCoder.of(), input.getCoder())); Combine.PerKey combine = - Combine.fewKeys(fn.asKeyedFn(), fnClass); + Combine.fewKeys(fn.asKeyedFn(), fnDisplayData); if (!sideInputs.isEmpty()) { combine = combine.withSideInputs(sideInputs); } @@ -1382,7 +1387,7 @@ public PCollection apply(PCollection input) { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - Combine.populateDisplayData(builder, fn, fnClass); + Combine.populateDisplayData(builder, fn, fnDisplayData); Combine.populateGlobalDisplayData(builder, fanout, insertDefault); } @@ -1413,10 +1418,11 @@ public void processElement(DoFn.ProcessContext c) { } private static void populateDisplayData( - DisplayData.Builder builder, HasDisplayData fn, ClassForDisplay fnClass) { + DisplayData.Builder builder, HasDisplayData fn, + DisplayData.Item> fnDisplayItem) { builder - .include(fn, fnClass) - .add(DisplayData.item("combineFn", fnClass)); + .include(fn) + .add(fnDisplayItem); } private static void populateGlobalDisplayData( @@ -1469,15 +1475,15 @@ public static class GloballyAsSingletonView extends PTransform, PCollectionView> { private final GlobalCombineFn fn; - private final ClassForDisplay fnClass; + private final DisplayData.Item> fnDisplayData; private final boolean insertDefault; private final int fanout; private GloballyAsSingletonView( - GlobalCombineFn fn, ClassForDisplay fnClass, - boolean insertDefault, int fanout) { + GlobalCombineFn fn, + DisplayData.Item> fnDisplayData, boolean insertDefault, int fanout) { this.fn = fn; - this.fnClass = fnClass; + this.fnDisplayData = fnDisplayData; this.insertDefault = insertDefault; this.fanout = fanout; } @@ -1513,7 +1519,7 @@ public boolean getInsertDefault() { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - Combine.populateDisplayData(builder, fn, fnClass); + Combine.populateDisplayData(builder, fn, fnDisplayData); Combine.populateGlobalDisplayData(builder, fanout, insertDefault); } } @@ -1674,35 +1680,36 @@ public static class PerKey extends PTransform>, PCollection>> { private final transient PerKeyCombineFn fn; - private final ClassForDisplay fnClass; + private final DisplayData.Item> fnDisplayData; private final boolean fewKeys; private final List> sideInputs; private PerKey( - PerKeyCombineFn fn, ClassForDisplay fnClass, - boolean fewKeys) { + PerKeyCombineFn fn, + DisplayData.Item> fnDisplayData, boolean fewKeys) { this.fn = fn; - this.fnClass = fnClass; + this.fnDisplayData = fnDisplayData; this.fewKeys = fewKeys; this.sideInputs = ImmutableList.of(); } private PerKey(String name, - PerKeyCombineFn fn, ClassForDisplay fnClass, + PerKeyCombineFn fn, + DisplayData.Item> fnDisplayData, boolean fewKeys, List> sideInputs) { super(name); this.fn = fn; - this.fnClass = fnClass; + this.fnDisplayData = fnDisplayData; this.fewKeys = fewKeys; this.sideInputs = sideInputs; } private PerKey( String name, PerKeyCombineFn fn, - ClassForDisplay fnClass, boolean fewKeys) { + DisplayData.Item> fnDisplayData, boolean fewKeys) { super(name); this.fn = fn; - this.fnClass = fnClass; + this.fnDisplayData = fnDisplayData; this.fewKeys = fewKeys; this.sideInputs = ImmutableList.of(); } @@ -1712,7 +1719,7 @@ private PerKey( * specified name. Does not modify this transform. */ public PerKey named(String name) { - return new PerKey(name, fn, fnClass, fewKeys); + return new PerKey<>(name, fn, fnDisplayData, fewKeys); } /** @@ -1722,8 +1729,8 @@ public PerKey named(String name) { public PerKey withSideInputs( Iterable> sideInputs) { Preconditions.checkState(fn instanceof RequiresContextInternal); - return new PerKey(name, fn, fnClass, fewKeys, - ImmutableList.>copyOf(sideInputs)); + return new PerKey<>(name, fn, fnDisplayData, fewKeys, + ImmutableList.copyOf(sideInputs)); } /** @@ -1739,7 +1746,7 @@ public PerKey withSideInputs( */ public PerKeyWithHotKeyFanout withHotKeyFanout( SerializableFunction hotKeyFanout) { - return new PerKeyWithHotKeyFanout(name, fn, fnClass, hotKeyFanout); + return new PerKeyWithHotKeyFanout<>(name, fn, fnDisplayData, hotKeyFanout); } /** @@ -1747,8 +1754,8 @@ public PerKeyWithHotKeyFanout withHotKeyFanout( * constant value for every key. */ public PerKeyWithHotKeyFanout withHotKeyFanout(final int hotKeyFanout) { - return new PerKeyWithHotKeyFanout(name, fn, fnClass, - new SerializableFunction(){ + return new PerKeyWithHotKeyFanout<>(name, fn, fnDisplayData, + new SerializableFunction() { @Override public Integer apply(K unused) { return hotKeyFanout; @@ -1780,7 +1787,7 @@ public PCollection> apply(PCollection> input) { @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - Combine.populateDisplayData(builder, fn, fnClass); + Combine.populateDisplayData(builder, fn, fnDisplayData); } } @@ -1791,16 +1798,16 @@ public static class PerKeyWithHotKeyFanout extends PTransform>, PCollection>> { private final transient PerKeyCombineFn fn; - private final ClassForDisplay fnClass; + private final DisplayData.Item> fnDisplayData; private final SerializableFunction hotKeyFanout; private PerKeyWithHotKeyFanout(String name, PerKeyCombineFn fn, - ClassForDisplay fnClass, + DisplayData.Item> fnDisplayData, SerializableFunction hotKeyFanout) { super(name); this.fn = fn; - this.fnClass = fnClass; + this.fnDisplayData = fnDisplayData; this.hotKeyFanout = hotKeyFanout; } @@ -2087,7 +2094,7 @@ public void processElement(ProcessContext c) { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - Combine.populateDisplayData(builder, fn, fnClass); + Combine.populateDisplayData(builder, fn, fnDisplayData); builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass())); } @@ -2232,28 +2239,29 @@ public static class GroupedValues PCollection>> { private final PerKeyCombineFn fn; - private final ClassForDisplay fnClass; + private final DisplayData.Item> fnDisplayData; private final List> sideInputs; private GroupedValues( - PerKeyCombineFn fn, ClassForDisplay fnClass) { + PerKeyCombineFn fn, + DisplayData.Item> fnDisplayData) { this.fn = SerializableUtils.clone(fn); - this.fnClass = fnClass; - this.sideInputs = ImmutableList.>of(); + this.fnDisplayData = fnDisplayData; + this.sideInputs = ImmutableList.of(); } private GroupedValues( PerKeyCombineFn fn, - ClassForDisplay fnClass, + DisplayData.Item> fnDisplayData, List> sideInputs) { this.fn = SerializableUtils.clone(fn); - this.fnClass = fnClass; + this.fnDisplayData = fnDisplayData; this.sideInputs = sideInputs; } public GroupedValues withSideInputs( Iterable> sideInputs) { - return new GroupedValues<>(fn, fnClass, ImmutableList.>copyOf(sideInputs)); + return new GroupedValues<>(fn, fnDisplayData, ImmutableList.copyOf(sideInputs)); } /** @@ -2344,7 +2352,7 @@ public Coder> getDefaultOutputCoder( @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - Combine.populateDisplayData(builder, fn, fnClass); + Combine.populateDisplayData(builder, fn, fnDisplayData); } } } 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 76661722bb7d..9d4d55d70bb7 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 @@ -39,6 +39,7 @@ import org.joda.time.format.DateTimeFormatter; import org.joda.time.format.ISODateTimeFormat; +import java.io.Serializable; import java.util.Collection; import java.util.Map; import java.util.Objects; @@ -52,7 +53,7 @@ *

Components specify their display data by implementing the {@link HasDisplayData} * interface. */ -public class DisplayData { +public class DisplayData implements Serializable { private static final DisplayData EMPTY = new DisplayData(Maps.>newHashMap()); private static final DateTimeFormatter TIMESTAMP_FORMATTER = ISODateTimeFormat.dateTime(); @@ -206,7 +207,7 @@ public interface Builder { * within {@link HasDisplayData#populateDisplayData} implementations. */ @AutoValue - public abstract static class Item { + public abstract static class Item implements Serializable { /** * The namespace for the display item. The namespace defaults to the component which 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 803daec08171..21768a864b13 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,6 +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.hasNamespace; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; import static com.google.common.base.Preconditions.checkNotNull; @@ -714,6 +715,21 @@ public void populateDisplayData(DisplayData.Builder builder) { assertThat(displayData, includesDisplayDataFrom(combineFn)); } + @Test + public void testDisplayDataForWrappedFn() { + UniqueInts combineFn = new UniqueInts() { + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.add(DisplayData.item("foo", "bar")); + } + }; + Combine.PerKey combine = Combine.perKey(combineFn); + DisplayData displayData = DisplayData.from(combine); + + assertThat(displayData, hasDisplayItem("combineFn", combineFn.getClass())); + assertThat(displayData, hasDisplayItem(hasNamespace(combineFn.getClass()))); + } + //////////////////////////////////////////////////////////////////////////// // Test classes, for different kinds of combining fns. From c2ef36c11a346eec7ee850bcd990cef61bce8d03 Mon Sep 17 00:00:00 2001 From: Scott Wegner Date: Wed, 27 Apr 2016 13:15:33 -0700 Subject: [PATCH 2/7] Refactor DoFnReflector.SimpleDoFnAdapter to not use display data namespace override --- .../main/java/org/apache/beam/sdk/transforms/DoFnReflector.java | 2 +- .../src/main/java/org/apache/beam/sdk/transforms/ParDo.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java index bbc022026af9..cb9c43bd4f68 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java @@ -656,7 +656,7 @@ protected TypeDescriptor getOutputTypeDescriptor() { @Override public void populateDisplayData(DisplayData.Builder builder) { - fn.populateDisplayData(builder); + builder.include(fn); } private void readObject(java.io.ObjectInputStream in) diff --git a/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 000a777f8f82..88945c43ecb6 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 @@ -1265,7 +1265,7 @@ private static SideInputReader makeSideInputReader( private static void populateDisplayData( DisplayData.Builder builder, DoFn fn, Class fnClass) { builder - .include(fn, fnClass) + .include(fn) .add(DisplayData.item("fn", fnClass)); } From 80ba09020a4067e23eddce02120ddd9fab50c272 Mon Sep 17 00:00:00 2001 From: Scott Wegner Date: Wed, 27 Apr 2016 13:25:12 -0700 Subject: [PATCH 3/7] Remove ClassForDisplay helper type --- .../transforms/display/ClassForDisplay.java | 94 ------------------- .../sdk/transforms/display/DisplayData.java | 39 +------- .../display/ClassForDisplayTest.java | 66 ------------- .../transforms/display/DisplayDataTest.java | 50 ++++------ .../display/ClassForDisplayJava8Test.java | 45 --------- 5 files changed, 24 insertions(+), 270 deletions(-) delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/ClassForDisplay.java delete mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/ClassForDisplayTest.java delete mode 100644 sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/display/ClassForDisplayJava8Test.java 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 deleted file mode 100644 index b5142e803c7f..000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/ClassForDisplay.java +++ /dev/null @@ -1,94 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.transforms.display; - -import static com.google.common.base.Preconditions.checkNotNull; - -import java.io.Serializable; -import java.util.Objects; - -/** - * Display data representing a Java class. - * - *

Java classes can be registered as display data via - * {@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 { - private final String simpleName; - private final String name; - - private ClassForDisplay(Class clazz) { - name = clazz.getName(); - simpleName = clazz.getSimpleName(); - } - - /** - * Create a {@link ClassForDisplay} instance representing the specified class. - */ - public static ClassForDisplay of(Class clazz) { - 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, "obj argument instance cannot be null"); - return new ClassForDisplay(obj.getClass()); - } - - /** - * Retrieve the fully-qualified name of the class. - * - * @see Class#getName() - */ - public String getName() { - return name; - } - - /** - * Retrieve a simple representation of the class name. - * - * @see Class#getSimpleName() - */ - public String getSimpleName() { - return simpleName; - } - - @Override - public String toString() { - return name; - } - - @Override - public int hashCode() { - return name.hashCode(); - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof ClassForDisplay) { - ClassForDisplay that = (ClassForDisplay) obj; - return Objects.equals(this.name, that.name); - } - - return false; - } -} diff --git a/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 9d4d55d70bb7..bb4dd8c608ba 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 @@ -145,7 +145,7 @@ public String toString() { return builder.toString(); } - private static String namespaceOf(ClassForDisplay clazz) { + private static String namespaceOf(Class clazz) { return clazz.getName(); } @@ -168,13 +168,6 @@ public interface Builder { */ Builder include(HasDisplayData subComponent, Class namespace); - /** - * Register display data from the specified subcomponent, using the specified namespace. - * - * @see #include(HasDisplayData, String) - */ - Builder include(HasDisplayData subComponent, ClassForDisplay namespace); - /** * Register display data from the specified subcomponent, using the specified namespace. * @@ -293,12 +286,6 @@ private static Item create(String key, Type type, @Nullable T value) { */ public Item withNamespace(Class namespace) { checkNotNull(namespace, "namespace argument cannot be null"); - return withNamespace(ClassForDisplay.of(namespace)); - } - - /** @see #withNamespace(Class) */ - private Item withNamespace(ClassForDisplay namespace) { - checkNotNull(namespace, "namesapce argument cannot be null"); return withNamespace(namespaceOf(namespace)); } @@ -378,7 +365,7 @@ public static class Identifier { private final String ns; private final String key; - public static Identifier of(ClassForDisplay namespace, String key) { + public static Identifier of(Class namespace, String key) { return of(namespaceOf(namespace), key); } @@ -471,12 +458,7 @@ FormattedItemValue format(Object value) { JAVA_CLASS { @Override FormattedItemValue format(Object value) { - if (value instanceof Class) { - ClassForDisplay classForDisplay = ClassForDisplay.of((Class) value); - return format(classForDisplay); - } - - ClassForDisplay clazz = checkType(value, ClassForDisplay.class, JAVA_CLASS); + Class clazz = checkType(value, Class.class, JAVA_CLASS); return new FormattedItemValue(clazz.getName(), clazz.getSimpleName()); } }; @@ -526,7 +508,7 @@ private static Type tryInferFrom(@Nullable Object value) { return TIMESTAMP; } else if (value instanceof Duration) { return DURATION; - } else if (value instanceof Class || value instanceof ClassForDisplay) { + } else if (value instanceof Class) { return JAVA_CLASS; } else if (value instanceof String) { return STRING; @@ -588,12 +570,6 @@ public Builder include(HasDisplayData subComponent) { @Override public Builder include(HasDisplayData subComponent, Class 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, "Input namespace override cannot be null"); return include(subComponent, namespaceOf(namespace)); } @@ -721,13 +697,6 @@ public static Item> item(String key, @Nullable Class value) { return item(key, Type.JAVA_CLASS, value); } - /** - * 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); - } - /** * 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, diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/ClassForDisplayTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/ClassForDisplayTest.java deleted file mode 100644 index 19f56c60b751..000000000000 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/ClassForDisplayTest.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.transforms.display; - -import static org.junit.Assert.assertEquals; - -import org.apache.beam.sdk.util.SerializableUtils; -import com.google.common.testing.EqualsTester; - -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** - * Unit tests for {@link ClassForDisplay}. - */ -@RunWith(JUnit4.class) -public class ClassForDisplayTest { - @Rule - public final ExpectedException thrown = ExpectedException.none(); - - @Test - public void testProperties() { - ClassForDisplay thisClass = ClassForDisplay.of(ClassForDisplayTest.class); - assertEquals(ClassForDisplayTest.class.getName(), thisClass.getName()); - assertEquals(ClassForDisplayTest.class.getSimpleName(), thisClass.getSimpleName()); - } - - @Test - public void testInputValidation() { - thrown.expect(NullPointerException.class); - ClassForDisplay.of(null); - } - - @Test - public void testEquality() { - new EqualsTester() - .addEqualityGroup( - ClassForDisplay.of(ClassForDisplayTest.class), ClassForDisplay.fromInstance(this)) - .addEqualityGroup(ClassForDisplay.of(ClassForDisplay.class)) - .addEqualityGroup(ClassForDisplay.of(Class.class)) - .testEquals(); - } - - @Test - public void testSerialization() { - SerializableUtils.ensureSerializable(ClassForDisplay.of(ClassForDisplayTest.class)); - } -} diff --git a/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 108409fbeecf..d096e62834d0 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 @@ -112,7 +112,7 @@ public void populateDisplayData(DisplayData.Builder builder) { .include(subComponent1) .include(subComponent2) .add(DisplayData.item("minSproggles", 200) - .withLabel("Mimimum Required Sproggles")) + .withLabel("Minimum Required Sproggles")) .add(DisplayData.item("fireLasers", true)) .addIfNotDefault(DisplayData.item("startTime", startTime), defaultStartTime) .add(DisplayData.item("timeBomb", Instant.now().plus(Duration.standardDays(1)))) @@ -189,16 +189,18 @@ public void populateDisplayData(DisplayData.Builder builder) { @SuppressWarnings("unchecked") DisplayData.Item item = (DisplayData.Item) data.items().toArray()[0]; - assertThat( - item, - Matchers.allOf( - hasNamespace(DisplayDataTest.class), - hasKey("now"), - hasType(DisplayData.Type.TIMESTAMP), - hasValue(ISO_FORMATTER.print(value)), - hasShortValue(nullValue(String.class)), - hasLabel(is("the current instant")), - hasUrl(is("http://time.gov")))); + + @SuppressWarnings("unchecked") + Matcher> matchesAllOf = Matchers.allOf( + hasNamespace(DisplayDataTest.class), + hasKey("now"), + hasType(DisplayData.Type.TIMESTAMP), + hasValue(ISO_FORMATTER.print(value)), + hasShortValue(nullValue(String.class)), + hasLabel(is("the current instant")), + hasUrl(is("http://time.gov"))); + + assertThat(item, matchesAllOf); } @Test @@ -270,13 +272,7 @@ public void populateDisplayData(Builder builder) { .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)); + Boolean.valueOf(true)); } }); @@ -318,7 +314,7 @@ public void populateDisplayData(Builder builder) { builder.addIfNotNull(DisplayData.item("nullItem", (Class) null) .withLinkUrl("http://abc") .withNamespace(DisplayDataTest.class) - .withLabel("Null item shoudl be safe")); + .withLabel("Null item should be safe")); } }; @@ -390,10 +386,10 @@ public void populateDisplayData(Builder builder) { public void testIdentifierEquality() { new EqualsTester() .addEqualityGroup( - DisplayData.Identifier.of(ClassForDisplay.of(DisplayDataTest.class), "1"), - DisplayData.Identifier.of(ClassForDisplay.of(DisplayDataTest.class), "1")) - .addEqualityGroup(DisplayData.Identifier.of(ClassForDisplay.of(Object.class), "1")) - .addEqualityGroup(DisplayData.Identifier.of(ClassForDisplay.of(DisplayDataTest.class), "2")) + DisplayData.Identifier.of(DisplayDataTest.class, "1"), + DisplayData.Identifier.of(DisplayDataTest.class, "1")) + .addEqualityGroup(DisplayData.Identifier.of(Object.class, "1")) + .addEqualityGroup(DisplayData.Identifier.of(DisplayDataTest.class, "2")) .testEquals(); } @@ -617,7 +613,6 @@ public void populateDisplayData(DisplayData.Builder builder) { .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))); } @@ -633,9 +628,6 @@ public void populateDisplayData(DisplayData.Builder builder) { assertThat( items, hasItem(allOf(hasKey("java_class"), hasType(DisplayData.Type.JAVA_CLASS)))); - assertThat( - items, - hasItem(allOf(hasKey("java_class2"), hasType(DisplayData.Type.JAVA_CLASS)))); assertThat( items, hasItem(allOf(hasKey("timestamp"), hasType(DisplayData.Type.TIMESTAMP)))); @@ -731,8 +723,6 @@ public void testKnownTypeInference() { assertEquals(DisplayData.Type.BOOLEAN, DisplayData.inferType(true)); assertEquals(DisplayData.Type.TIMESTAMP, DisplayData.inferType(Instant.now())); assertEquals(DisplayData.Type.DURATION, DisplayData.inferType(Duration.millis(1234))); - assertEquals(DisplayData.Type.JAVA_CLASS, - DisplayData.inferType(ClassForDisplay.of(DisplayDataTest.class))); assertEquals(DisplayData.Type.JAVA_CLASS, DisplayData.inferType(DisplayDataTest.class)); assertEquals(DisplayData.Type.STRING, DisplayData.inferType("hello world")); @@ -828,7 +818,7 @@ public void populateDisplayData(Builder builder) { DisplayData.from(new HasDisplayData() { @Override public void populateDisplayData(Builder builder) { - builder.include(subComponent, (ClassForDisplay) null); + builder.include(subComponent, (Class) null); } }); } diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/display/ClassForDisplayJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/display/ClassForDisplayJava8Test.java deleted file mode 100644 index 8889a788d846..000000000000 --- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/display/ClassForDisplayJava8Test.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.transforms.display; - -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.util.SerializableUtils; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -import java.io.Serializable; - -/** - * Java 8 tests for {@link ClassForDisplay}. - */ -@RunWith(JUnit4.class) -public class ClassForDisplayJava8Test implements Serializable { - @Test - public void testLambdaClassSerialization() { - final SerializableFunction f = x -> x; - Serializable myClass = new Serializable() { - // Class references for lambdas do not serialize, which is why we support ClassForDisplay - // Specifically, the following would not work: - // Class clazz = f.getClass(); - ClassForDisplay javaClass = ClassForDisplay.fromInstance(f); - }; - - SerializableUtils.ensureSerializable(myClass); - } -} From 45a962b0c2bbbfb4f57b70efd84a593ea382ef59 Mon Sep 17 00:00:00 2001 From: Scott Wegner Date: Wed, 27 Apr 2016 15:00:03 -0700 Subject: [PATCH 4/7] Add test case for display data multi-level namespace overrides --- .../display/DisplayDataMatchers.java | 14 ++++++++ .../transforms/display/DisplayDataTest.java | 33 +++++++++++++++++++ 2 files changed, 47 insertions(+) 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 c47b4c7182a1..e3721b8f6ef4 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 @@ -274,6 +274,20 @@ public static Matcher> hasNamespace(Class namespace) { return hasNamespace(Matchers.>is(namespace)); } + /** + * Creates a matcher that matches if the examined {@link DisplayData.Item} contains the + * specified namespace. + */ + public static Matcher> hasNamespace(String namespace) { + return new FeatureMatcher, String>( + Matchers.is(namespace), "display item with namespace", "namespace") { + @Override + protected String featureValueOf(Item actual) { + return actual.getNamespace(); + } + }; + } + /** * Creates a matcher that matches if the examined {@link DisplayData.Item} contains a namespace * matching the specified namespace matcher. 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 d096e62834d0..851769a03e60 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 @@ -369,6 +369,39 @@ public void populateDisplayData(DisplayData.Builder builder) { assertThat(data, includesDisplayDataFrom(subComponent, namespaceOverride.getClass())); } + @Test + public void testNamespaceOverrideMultipleLevels() { + final HasDisplayData componentA = new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder.add(DisplayData.item("foo", "bar")); + } + }; + + final HasDisplayData componentB = new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder + .add(DisplayData.item("foo", "bar")) + .include(componentA); + } + }; + + final HasDisplayData componentC = new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder + .add(DisplayData.item("foo", "bar")) + .include(componentB, "overrideB"); + } + }; + + DisplayData data = DisplayData.from(componentC); + assertThat(data, hasDisplayItem(hasNamespace(componentC.getClass()))); + assertThat(data, hasDisplayItem(hasNamespace("overrideB"))); + assertThat(data, hasDisplayItem(hasNamespace(componentA.getClass()))); + } + @Test public void testNullNamespaceOverride() { thrown.expect(NullPointerException.class); From b951d9985ab3b9a2eb00563ed476f8385ef250da Mon Sep 17 00:00:00 2001 From: Scott Wegner Date: Thu, 28 Apr 2016 14:50:20 -0700 Subject: [PATCH 5/7] Add javadoc comments on display data usage --- .../sdk/transforms/display/DisplayData.java | 37 ++++++++++--- .../transforms/display/HasDisplayData.java | 55 ++++++++++++------- 2 files changed, 62 insertions(+), 30 deletions(-) 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 bb4dd8c608ba..7e8ac73c3433 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 @@ -19,9 +19,7 @@ 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; @@ -155,24 +153,45 @@ private static String namespaceOf(Class clazz) { */ public interface Builder { /** - * Register display data from the specified subcomponent. + * Register display data from the specified subcomponent. For example, a {@link PTransform} + * which delegates to a user-provided function can implement {@link HasDisplayData} on the + * function and include it from the {@link PTransform}: * - * @see #include(HasDisplayData, String) + *

{@literal @Override}
+     * public void populateDisplayData(DisplayData.Builder builder) {
+     *   super.populateDisplayData(builder);
+     *
+     *   builder
+     *     .add(DisplayData.item("userFn", userFn)) // To register the class name of the userFn
+     *     .include(userFn); // To allow the userFn to register additional display data
+     * }
+     * 
+ * + *

Including subcomponent display data via + * {@code DisplayData.Builder.include(HasDisplayData)} ensures that the + * {@link Item#getNamespace() namespace} of registered {@link Item display items} + * is properly set to the namespace of the subcomponent. To register display data from a base + * class, use {@code super.populateDisplayData(builder)}, which will maintain the namespace of + * the current component. In all other cases, {@code builder.include(HasDisplayData)} is the + * preferred approach for including display data from other components. + * + * @see HasDisplayData#populateDisplayData(DisplayData.Builder) */ Builder include(HasDisplayData subComponent); /** - * Register display data from the specified subcomponent, using the specified namespace. + * Register display data from the specified subcomponent, overriding the namespace of + * subcomponent display items with the specified namespace. * - * @see #include(HasDisplayData, String) + * @see #include(HasDisplayData) */ Builder include(HasDisplayData subComponent, Class namespace); /** - * Register display data from the specified subcomponent, using the specified namespace. + * Register display data from the specified subcomponent, overriding the namespace of + * subcomponent display items with the specified namespace. * - *

For example, a {@link ParDo} transform includes display data from the encapsulated - * {@link DoFn}. + * @see #include(HasDisplayData) */ Builder include(HasDisplayData subComponent, String namespace); 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 d74944e55cab..c78ac1b0c946 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 @@ -17,39 +17,52 @@ */ package org.apache.beam.sdk.transforms.display; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.PTransform; /** - * Marker interface for {@link PTransform PTransforms} and components used within - * {@link PTransform PTransforms} to specify display data to be used within UIs and diagnostic - * tools. + * Marker interface for {@link PTransform PTransforms} and components to specify display data used + * within UIs and diagnostic tools. + * + *

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

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

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

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

{@code populateDisplayData(DisplayData.Builder)} is invoked by Pipeline runners to collect + * display data via {@link DisplayData#from(HasDisplayData)}. Implementations may call + * {@code super.populateDisplayData(builder)} in order to register display data from a base class, + * but should otherwise use {@link DisplayData.Builder#include(HasDisplayData)} for including + * display data from a subcomponent. * * @param builder The builder to populate with display data. + * + * @see HasDisplayData */ void populateDisplayData(DisplayData.Builder builder); } From 9bbc14320f937b635d8020e040be62917161761c Mon Sep 17 00:00:00 2001 From: Scott Wegner Date: Fri, 29 Apr 2016 14:02:10 -0700 Subject: [PATCH 6/7] fixup! Add javadoc comments on display data usage --- .../beam/sdk/transforms/display/DisplayData.java | 11 ++++------- .../beam/sdk/transforms/display/HasDisplayData.java | 6 +++--- 2 files changed, 7 insertions(+), 10 deletions(-) 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 7e8ac73c3433..82a759206b83 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 @@ -167,13 +167,10 @@ public interface Builder { * } * * - *

Including subcomponent display data via - * {@code DisplayData.Builder.include(HasDisplayData)} ensures that the - * {@link Item#getNamespace() namespace} of registered {@link Item display items} - * is properly set to the namespace of the subcomponent. To register display data from a base - * class, use {@code super.populateDisplayData(builder)}, which will maintain the namespace of - * the current component. In all other cases, {@code builder.include(HasDisplayData)} is the - * preferred approach for including display data from other components. + * Using {@code include(subcomponent)} will associate each of the registered items with the + * namespace of the {@code subcomponent} being registered. To register display data in the + * current namespace, such as from a base class implementation, use + * {@code super.populateDisplayData(builder)} instead. * * @see HasDisplayData#populateDisplayData(DisplayData.Builder) */ 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 c78ac1b0c946..7fcd8fcd7152 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 @@ -56,9 +56,9 @@ public interface HasDisplayData { * *

{@code populateDisplayData(DisplayData.Builder)} is invoked by Pipeline runners to collect * display data via {@link DisplayData#from(HasDisplayData)}. Implementations may call - * {@code super.populateDisplayData(builder)} in order to register display data from a base class, - * but should otherwise use {@link DisplayData.Builder#include(HasDisplayData)} for including - * display data from a subcomponent. + * {@code super.populateDisplayData(builder)} in order to register display data in the current + * namespace, but should otherwise use {@link DisplayData.Builder#include} to use the namespace + * of the subcomponent. * * @param builder The builder to populate with display data. * From 33b810b8254275b7cdfc72ad825eecfda642c6b6 Mon Sep 17 00:00:00 2001 From: Scott Wegner Date: Mon, 2 May 2016 11:35:15 -0700 Subject: [PATCH 7/7] fixup! Add javadoc comments on display data usage --- .../org/apache/beam/sdk/transforms/display/DisplayData.java | 2 +- .../apache/beam/sdk/transforms/display/HasDisplayData.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) 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 82a759206b83..fa8c0e9f4a84 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 @@ -170,7 +170,7 @@ public interface Builder { * Using {@code include(subcomponent)} will associate each of the registered items with the * namespace of the {@code subcomponent} being registered. To register display data in the * current namespace, such as from a base class implementation, use - * {@code super.populateDisplayData(builder)} instead. + * {@code subcomponent.populateDisplayData(builder)} instead. * * @see HasDisplayData#populateDisplayData(DisplayData.Builder) */ 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 7fcd8fcd7152..0ca1abff28ba 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 @@ -57,8 +57,8 @@ public interface HasDisplayData { *

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