You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by bc...@apache.org on 2016/05/02 22:24:51 UTC
[1/2] incubator-beam git commit: Remove ClassForDisplay helper type
Repository: incubator-beam
Updated Branches:
refs/heads/master 659cf2ee0 -> 1262819ac
Remove ClassForDisplay helper type
- Refactor Combine display data to not use ClassForDisplay
- Refactor DoFnReflector.SimpleDoFnAdapter to not use display data namespace override
- Add test case for display data multi-level namespace overrides
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/5eafdee5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/5eafdee5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/5eafdee5
Branch: refs/heads/master
Commit: 5eafdee53cc49938921c5fbdf2bb81483f8773af
Parents: 659cf2e
Author: Scott Wegner <sw...@google.com>
Authored: Wed Apr 27 13:08:49 2016 -0700
Committer: bchambers <bc...@google.com>
Committed: Mon May 2 13:10:10 2016 -0700
----------------------------------------------------------------------
.../org/apache/beam/sdk/transforms/Combine.java | 158 ++++++++++---------
.../beam/sdk/transforms/DoFnReflector.java | 2 +-
.../org/apache/beam/sdk/transforms/ParDo.java | 2 +-
.../sdk/transforms/display/ClassForDisplay.java | 94 -----------
.../sdk/transforms/display/DisplayData.java | 78 ++++-----
.../sdk/transforms/display/HasDisplayData.java | 55 ++++---
.../apache/beam/sdk/transforms/CombineTest.java | 16 ++
.../transforms/display/ClassForDisplayTest.java | 66 --------
.../transforms/display/DisplayDataMatchers.java | 14 ++
.../sdk/transforms/display/DisplayDataTest.java | 83 ++++++----
.../display/ClassForDisplayJava8Test.java | 45 ------
11 files changed, 234 insertions(+), 379 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eafdee5/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java
----------------------------------------------------------------------
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 68fc1cf..f3a7c03 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.CombineFnWithContext;
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 @@ public class Combine {
*/
public static <V> Globally<V, V> globally(
SerializableFunction<Iterable<V>, V> combiner) {
- return globally(IterableCombineFn.of(combiner), ClassForDisplay.fromInstance(combiner));
+ return globally(IterableCombineFn.of(combiner), displayDataForFn(combiner));
}
@@ -124,12 +123,17 @@ public class Combine {
*/
public static <InputT, OutputT> Globally<InputT, OutputT> globally(
GlobalCombineFn<? super InputT, ?, OutputT> fn) {
- return globally(fn, ClassForDisplay.fromInstance(fn));
+ return globally(fn, displayDataForFn(fn));
+ }
+
+ private static <T> DisplayData.Item<? extends Class<?>> displayDataForFn(T fn) {
+ return DisplayData.item("combineFn", fn.getClass());
}
private static <InputT, OutputT> Globally<InputT, OutputT> globally(
- GlobalCombineFn<? super InputT, ?, OutputT> fn, ClassForDisplay fnClass) {
- return new Globally<>(fn, fnClass, true, 0);
+ GlobalCombineFn<? super InputT, ?, OutputT> fn,
+ DisplayData.Item<? extends Class<?>> fnDisplayData) {
+ return new Globally<>(fn, fnDisplayData, true, 0);
}
/**
@@ -150,7 +154,7 @@ public class Combine {
*/
public static <K, V> PerKey<K, V, V> perKey(
SerializableFunction<Iterable<V>, V> fn) {
- return perKey(IterableCombineFn.of(fn).<K>asKeyedFn(), ClassForDisplay.fromInstance(fn));
+ return perKey(IterableCombineFn.of(fn).<K>asKeyedFn(), displayDataForFn(fn));
}
/**
@@ -171,7 +175,7 @@ public class Combine {
*/
public static <K, InputT, OutputT> PerKey<K, InputT, OutputT> perKey(
GlobalCombineFn<? super InputT, ?, OutputT> fn) {
- return perKey(fn.<K>asKeyedFn(), ClassForDisplay.fromInstance(fn));
+ return perKey(fn.<K>asKeyedFn(), displayDataForFn(fn));
}
/**
@@ -192,12 +196,13 @@ public class Combine {
*/
public static <K, InputT, OutputT> PerKey<K, InputT, OutputT> perKey(
PerKeyCombineFn<? super K, ? super InputT, ?, OutputT> fn) {
- return perKey(fn, ClassForDisplay.fromInstance(fn));
+ return perKey(fn, displayDataForFn(fn));
}
private static <K, InputT, OutputT> PerKey<K, InputT, OutputT> perKey(
- PerKeyCombineFn<? super K, ? super InputT, ?, OutputT> fn, ClassForDisplay fnClass) {
- return new PerKey<>(fn, fnClass, false /*fewKeys*/);
+ PerKeyCombineFn<? super K, ? super InputT, ?, OutputT> fn,
+ DisplayData.Item<? extends Class<?>> fnDisplayData) {
+ return new PerKey<>(fn, fnDisplayData, false /*fewKeys*/);
}
/**
@@ -205,8 +210,9 @@ public class Combine {
* in {@link GroupByKey}.
*/
private static <K, InputT, OutputT> PerKey<K, InputT, OutputT> fewKeys(
- PerKeyCombineFn<? super K, ? super InputT, ?, OutputT> fn, ClassForDisplay fnClass) {
- return new PerKey<>(fn, fnClass, true /*fewKeys*/);
+ PerKeyCombineFn<? super K, ? super InputT, ?, OutputT> fn,
+ DisplayData.Item<? extends Class<?>> fnDisplayData) {
+ return new PerKey<>(fn, fnDisplayData, true /*fewKeys*/);
}
/**
@@ -232,7 +238,7 @@ public class Combine {
*/
public static <K, V> GroupedValues<K, V, V> groupedValues(
SerializableFunction<Iterable<V>, V> fn) {
- return groupedValues(IterableCombineFn.of(fn).<K>asKeyedFn(), ClassForDisplay.fromInstance(fn));
+ return groupedValues(IterableCombineFn.of(fn).<K>asKeyedFn(), displayDataForFn(fn));
}
/**
@@ -258,7 +264,7 @@ public class Combine {
*/
public static <K, InputT, OutputT> GroupedValues<K, InputT, OutputT> groupedValues(
GlobalCombineFn<? super InputT, ?, OutputT> fn) {
- return groupedValues(fn.<K>asKeyedFn(), ClassForDisplay.fromInstance(fn));
+ return groupedValues(fn.<K>asKeyedFn(), displayDataForFn(fn));
}
/**
@@ -284,12 +290,13 @@ public class Combine {
*/
public static <K, InputT, OutputT> GroupedValues<K, InputT, OutputT> groupedValues(
PerKeyCombineFn<? super K, ? super InputT, ?, OutputT> fn) {
- return groupedValues(fn, ClassForDisplay.fromInstance(fn));
+ return groupedValues(fn, displayDataForFn(fn));
}
private static <K, InputT, OutputT> GroupedValues<K, InputT, OutputT> groupedValues(
- PerKeyCombineFn<? super K, ? super InputT, ?, OutputT> fn, ClassForDisplay fnClass) {
- return new GroupedValues<>(fn, fnClass);
+ PerKeyCombineFn<? super K, ? super InputT, ?, OutputT> fn,
+ DisplayData.Item<? extends Class<?>> fnDisplayData) {
+ return new GroupedValues<>(fn, fnDisplayData);
}
/////////////////////////////////////////////////////////////////////////////
@@ -431,7 +438,7 @@ public class Combine {
*
* <p>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<? extends InputT> inputs) {
AccumT accum = createAccumulator();
@@ -515,8 +522,7 @@ public class Combine {
@Override
public void populateDisplayData(DisplayData.Builder builder) {
- super.populateDisplayData(builder);
- CombineFn.this.populateDisplayData(builder);
+ builder.include(CombineFn.this);
}
};
}
@@ -1194,8 +1200,7 @@ public class Combine {
@Override
public void populateDisplayData(DisplayData.Builder builder) {
- super.populateDisplayData(builder);
- KeyedCombineFn.this.populateDisplayData(builder);
+ builder.include(KeyedCombineFn.this);
}
};
}
@@ -1262,36 +1267,36 @@ public class Combine {
extends PTransform<PCollection<InputT>, PCollection<OutputT>> {
private final GlobalCombineFn<? super InputT, ?, OutputT> fn;
- private final ClassForDisplay fnClass;
+ private final DisplayData.Item<? extends Class<?>> fnDisplayData;
private final boolean insertDefault;
private final int fanout;
private final List<PCollectionView<?>> sideInputs;
- private Globally(GlobalCombineFn<? super InputT, ?, OutputT> fn, ClassForDisplay fnClass,
- boolean insertDefault, int fanout) {
+ private Globally(GlobalCombineFn<? super InputT, ?, OutputT> fn,
+ DisplayData.Item<? extends Class<?>> fnDisplayData, boolean insertDefault, int fanout) {
this.fn = fn;
- this.fnClass = fnClass;
+ this.fnDisplayData = fnDisplayData;
this.insertDefault = insertDefault;
this.fanout = fanout;
- this.sideInputs = ImmutableList.<PCollectionView<?>>of();
+ this.sideInputs = ImmutableList.of();
}
private Globally(String name, GlobalCombineFn<? super InputT, ?, OutputT> fn,
- ClassForDisplay fnClass, boolean insertDefault, int fanout) {
+ DisplayData.Item<? extends Class<?>> 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.<PCollectionView<?>>of();
+ this.sideInputs = ImmutableList.of();
}
private Globally(String name, GlobalCombineFn<? super InputT, ?, OutputT> fn,
- ClassForDisplay fnClass, boolean insertDefault, int fanout,
+ DisplayData.Item<? extends Class<?>> fnDisplayData, boolean insertDefault, int fanout,
List<PCollectionView<?>> 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 @@ public class Combine {
* specified name. Does not modify this transform.
*/
public Globally<InputT, OutputT> named(String name) {
- return new Globally<>(name, fn, fnClass, insertDefault, fanout);
+ return new Globally<>(name, fn, fnDisplayData, insertDefault, fanout);
}
/**
@@ -1313,7 +1318,7 @@ public class Combine {
* to an empty input set will be returned.
*/
public GloballyAsSingletonView<InputT, OutputT> asSingletonView() {
- return new GloballyAsSingletonView<>(fn, fnClass, insertDefault, fanout);
+ return new GloballyAsSingletonView<>(fn, fnDisplayData, insertDefault, fanout);
}
/**
@@ -1322,7 +1327,7 @@ public class Combine {
* is not globally windowed and the output is not being used as a side input.
*/
public Globally<InputT, OutputT> withoutDefaults() {
- return new Globally<>(name, fn, fnClass, false, fanout);
+ return new Globally<>(name, fn, fnDisplayData, false, fanout);
}
/**
@@ -1333,7 +1338,7 @@ public class Combine {
* that will be used.
*/
public Globally<InputT, OutputT> withFanout(int fanout) {
- return new Globally<>(name, fn, fnClass, insertDefault, fanout);
+ return new Globally<>(name, fn, fnDisplayData, insertDefault, fanout);
}
/**
@@ -1343,8 +1348,8 @@ public class Combine {
public Globally<InputT, OutputT> withSideInputs(
Iterable<? extends PCollectionView<?>> sideInputs) {
Preconditions.checkState(fn instanceof RequiresContextInternal);
- return new Globally<InputT, OutputT>(name, fn, fnClass, insertDefault, fanout,
- ImmutableList.<PCollectionView<?>>copyOf(sideInputs));
+ return new Globally<>(name, fn, fnDisplayData, insertDefault, fanout,
+ ImmutableList.copyOf(sideInputs));
}
@Override
@@ -1354,7 +1359,7 @@ public class Combine {
.setCoder(KvCoder.of(VoidCoder.of(), input.getCoder()));
Combine.PerKey<Void, InputT, OutputT> combine =
- Combine.<Void, InputT, OutputT>fewKeys(fn.asKeyedFn(), fnClass);
+ Combine.fewKeys(fn.asKeyedFn(), fnDisplayData);
if (!sideInputs.isEmpty()) {
combine = combine.withSideInputs(sideInputs);
}
@@ -1382,7 +1387,7 @@ public class Combine {
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 class Combine {
}
private static void populateDisplayData(
- DisplayData.Builder builder, HasDisplayData fn, ClassForDisplay fnClass) {
+ DisplayData.Builder builder, HasDisplayData fn,
+ DisplayData.Item<? extends Class<?>> fnDisplayItem) {
builder
- .include(fn, fnClass)
- .add(DisplayData.item("combineFn", fnClass));
+ .include(fn)
+ .add(fnDisplayItem);
}
private static void populateGlobalDisplayData(
@@ -1469,15 +1475,15 @@ public class Combine {
extends PTransform<PCollection<InputT>, PCollectionView<OutputT>> {
private final GlobalCombineFn<? super InputT, ?, OutputT> fn;
- private final ClassForDisplay fnClass;
+ private final DisplayData.Item<? extends Class<?>> fnDisplayData;
private final boolean insertDefault;
private final int fanout;
private GloballyAsSingletonView(
- GlobalCombineFn<? super InputT, ?, OutputT> fn, ClassForDisplay fnClass,
- boolean insertDefault, int fanout) {
+ GlobalCombineFn<? super InputT, ?, OutputT> fn,
+ DisplayData.Item<? extends Class<?>> 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 class Combine {
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 class Combine {
extends PTransform<PCollection<KV<K, InputT>>, PCollection<KV<K, OutputT>>> {
private final transient PerKeyCombineFn<? super K, ? super InputT, ?, OutputT> fn;
- private final ClassForDisplay fnClass;
+ private final DisplayData.Item<? extends Class<?>> fnDisplayData;
private final boolean fewKeys;
private final List<PCollectionView<?>> sideInputs;
private PerKey(
- PerKeyCombineFn<? super K, ? super InputT, ?, OutputT> fn, ClassForDisplay fnClass,
- boolean fewKeys) {
+ PerKeyCombineFn<? super K, ? super InputT, ?, OutputT> fn,
+ DisplayData.Item<? extends Class<?>> fnDisplayData, boolean fewKeys) {
this.fn = fn;
- this.fnClass = fnClass;
+ this.fnDisplayData = fnDisplayData;
this.fewKeys = fewKeys;
this.sideInputs = ImmutableList.of();
}
private PerKey(String name,
- PerKeyCombineFn<? super K, ? super InputT, ?, OutputT> fn, ClassForDisplay fnClass,
+ PerKeyCombineFn<? super K, ? super InputT, ?, OutputT> fn,
+ DisplayData.Item<? extends Class<?>> fnDisplayData,
boolean fewKeys, List<PCollectionView<?>> sideInputs) {
super(name);
this.fn = fn;
- this.fnClass = fnClass;
+ this.fnDisplayData = fnDisplayData;
this.fewKeys = fewKeys;
this.sideInputs = sideInputs;
}
private PerKey(
String name, PerKeyCombineFn<? super K, ? super InputT, ?, OutputT> fn,
- ClassForDisplay fnClass, boolean fewKeys) {
+ DisplayData.Item<? extends Class<?>> 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 @@ public class Combine {
* specified name. Does not modify this transform.
*/
public PerKey<K, InputT, OutputT> named(String name) {
- return new PerKey<K, InputT, OutputT>(name, fn, fnClass, fewKeys);
+ return new PerKey<>(name, fn, fnDisplayData, fewKeys);
}
/**
@@ -1722,8 +1729,8 @@ public class Combine {
public PerKey<K, InputT, OutputT> withSideInputs(
Iterable<? extends PCollectionView<?>> sideInputs) {
Preconditions.checkState(fn instanceof RequiresContextInternal);
- return new PerKey<K, InputT, OutputT>(name, fn, fnClass, fewKeys,
- ImmutableList.<PCollectionView<?>>copyOf(sideInputs));
+ return new PerKey<>(name, fn, fnDisplayData, fewKeys,
+ ImmutableList.copyOf(sideInputs));
}
/**
@@ -1739,7 +1746,7 @@ public class Combine {
*/
public PerKeyWithHotKeyFanout<K, InputT, OutputT> withHotKeyFanout(
SerializableFunction<? super K, Integer> hotKeyFanout) {
- return new PerKeyWithHotKeyFanout<K, InputT, OutputT>(name, fn, fnClass, hotKeyFanout);
+ return new PerKeyWithHotKeyFanout<>(name, fn, fnDisplayData, hotKeyFanout);
}
/**
@@ -1747,8 +1754,8 @@ public class Combine {
* constant value for every key.
*/
public PerKeyWithHotKeyFanout<K, InputT, OutputT> withHotKeyFanout(final int hotKeyFanout) {
- return new PerKeyWithHotKeyFanout<K, InputT, OutputT>(name, fn, fnClass,
- new SerializableFunction<K, Integer>(){
+ return new PerKeyWithHotKeyFanout<>(name, fn, fnDisplayData,
+ new SerializableFunction<K, Integer>() {
@Override
public Integer apply(K unused) {
return hotKeyFanout;
@@ -1780,7 +1787,7 @@ public class Combine {
@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 class Combine {
extends PTransform<PCollection<KV<K, InputT>>, PCollection<KV<K, OutputT>>> {
private final transient PerKeyCombineFn<? super K, ? super InputT, ?, OutputT> fn;
- private final ClassForDisplay fnClass;
+ private final DisplayData.Item<? extends Class<?>> fnDisplayData;
private final SerializableFunction<? super K, Integer> hotKeyFanout;
private PerKeyWithHotKeyFanout(String name,
PerKeyCombineFn<? super K, ? super InputT, ?, OutputT> fn,
- ClassForDisplay fnClass,
+ DisplayData.Item<? extends Class<?>> fnDisplayData,
SerializableFunction<? super K, Integer> hotKeyFanout) {
super(name);
this.fn = fn;
- this.fnClass = fnClass;
+ this.fnDisplayData = fnDisplayData;
this.hotKeyFanout = hotKeyFanout;
}
@@ -2087,7 +2094,7 @@ public class Combine {
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 class Combine {
PCollection<KV<K, OutputT>>> {
private final PerKeyCombineFn<? super K, ? super InputT, ?, OutputT> fn;
- private final ClassForDisplay fnClass;
+ private final DisplayData.Item<? extends Class<?>> fnDisplayData;
private final List<PCollectionView<?>> sideInputs;
private GroupedValues(
- PerKeyCombineFn<? super K, ? super InputT, ?, OutputT> fn, ClassForDisplay fnClass) {
+ PerKeyCombineFn<? super K, ? super InputT, ?, OutputT> fn,
+ DisplayData.Item<? extends Class<?>> fnDisplayData) {
this.fn = SerializableUtils.clone(fn);
- this.fnClass = fnClass;
- this.sideInputs = ImmutableList.<PCollectionView<?>>of();
+ this.fnDisplayData = fnDisplayData;
+ this.sideInputs = ImmutableList.of();
}
private GroupedValues(
PerKeyCombineFn<? super K, ? super InputT, ?, OutputT> fn,
- ClassForDisplay fnClass,
+ DisplayData.Item<? extends Class<?>> fnDisplayData,
List<PCollectionView<?>> sideInputs) {
this.fn = SerializableUtils.clone(fn);
- this.fnClass = fnClass;
+ this.fnDisplayData = fnDisplayData;
this.sideInputs = sideInputs;
}
public GroupedValues<K, InputT, OutputT> withSideInputs(
Iterable<? extends PCollectionView<?>> sideInputs) {
- return new GroupedValues<>(fn, fnClass, ImmutableList.<PCollectionView<?>>copyOf(sideInputs));
+ return new GroupedValues<>(fn, fnDisplayData, ImmutableList.copyOf(sideInputs));
}
/**
@@ -2344,7 +2352,7 @@ public class Combine {
@Override
public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);
- Combine.populateDisplayData(builder, fn, fnClass);
+ Combine.populateDisplayData(builder, fn, fnDisplayData);
}
}
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eafdee5/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java
----------------------------------------------------------------------
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 bbc0220..cb9c43b 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 @@ public abstract class DoFnReflector {
@Override
public void populateDisplayData(DisplayData.Builder builder) {
- fn.populateDisplayData(builder);
+ builder.include(fn);
}
private void readObject(java.io.ObjectInputStream in)
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eafdee5/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java
----------------------------------------------------------------------
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 000a777..88945c4 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 @@ public class ParDo {
private static void populateDisplayData(
DisplayData.Builder builder, DoFn<?, ?> fn, Class<?> fnClass) {
builder
- .include(fn, fnClass)
+ .include(fn)
.add(DisplayData.item("fn", fnClass));
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eafdee5/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/ClassForDisplay.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 b5142e8..0000000
--- 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.
- *
- * <p>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;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eafdee5/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java
----------------------------------------------------------------------
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 7666172..fa8c0e9 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 @@ package org.apache.beam.sdk.transforms.display;
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;
@@ -39,6 +37,7 @@ import org.joda.time.Instant;
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 +51,7 @@ import java.util.Set;
* <p>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.<Identifier, Item<?>>newHashMap());
private static final DateTimeFormatter TIMESTAMP_FORMATTER = ISODateTimeFormat.dateTime();
@@ -144,7 +143,7 @@ public class DisplayData {
return builder.toString();
}
- private static String namespaceOf(ClassForDisplay clazz) {
+ private static String namespaceOf(Class<?> clazz) {
return clazz.getName();
}
@@ -154,31 +153,42 @@ public class DisplayData {
*/
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)
+ * <pre><code>{@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
+ * }
+ * </code></pre>
+ *
+ * Using {@code include(subcomponent)} will associate each of the registered items with the
+ * namespace of the {@code subcomponent} being registered. To register display data in the
+ * current namespace, such as from a base class implementation, use
+ * {@code subcomponent.populateDisplayData(builder)} instead.
+ *
+ * @see 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.
*
- * @see #include(HasDisplayData, String)
- */
- Builder include(HasDisplayData subComponent, ClassForDisplay namespace);
-
- /**
- * Register display data from the specified subcomponent, using the specified namespace.
- *
- * <p>For example, a {@link ParDo} transform includes display data from the encapsulated
- * {@link DoFn}.
+ * @see #include(HasDisplayData)
*/
Builder include(HasDisplayData subComponent, String namespace);
@@ -206,7 +216,7 @@ public class DisplayData {
* within {@link HasDisplayData#populateDisplayData} implementations.
*/
@AutoValue
- public abstract static class Item<T> {
+ public abstract static class Item<T> implements Serializable {
/**
* The namespace for the display item. The namespace defaults to the component which
@@ -292,12 +302,6 @@ public class DisplayData {
*/
public Item<T> withNamespace(Class<?> namespace) {
checkNotNull(namespace, "namespace argument cannot be null");
- return withNamespace(ClassForDisplay.of(namespace));
- }
-
- /** @see #withNamespace(Class) */
- private Item<T> withNamespace(ClassForDisplay namespace) {
- checkNotNull(namespace, "namesapce argument cannot be null");
return withNamespace(namespaceOf(namespace));
}
@@ -377,7 +381,7 @@ public class DisplayData {
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);
}
@@ -470,12 +474,7 @@ public class DisplayData {
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());
}
};
@@ -525,7 +524,7 @@ public class DisplayData {
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 +587,6 @@ public class DisplayData {
@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 +714,6 @@ public class DisplayData {
}
/**
- * Create a display item for the specified key and class value.
- */
- public static Item<ClassForDisplay> 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,
* {@link HasDisplayData} implementors should call one of the typed factory methods, such as
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eafdee5/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/HasDisplayData.java
----------------------------------------------------------------------
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 d74944e..0ca1abf 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.
+ *
+ * <p>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.
+ *
+ * <pre><code>{@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"));
+ * }
+ * </code></pre>
*
* <p>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.
*
- * <pre>
- * {@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"));
- * }
- * }
- * </pre>
+ * <p>{@code populateDisplayData(DisplayData.Builder)} is invoked by Pipeline runners to collect
+ * display data via {@link DisplayData#from(HasDisplayData)}. Implementations may call
+ * {@code super.populateDisplayData(builder)} in order to register display data in the current
+ * namespace, but should otherwise use {@code subcomponent.populateDisplayData(builder)} to use
+ * the namespace of the subcomponent.
*
* @param builder The builder to populate with display data.
+ *
+ * @see HasDisplayData
*/
void populateDisplayData(DisplayData.Builder builder);
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eafdee5/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java
----------------------------------------------------------------------
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 803daec..21768a8 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 @@ package org.apache.beam.sdk.transforms;
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 class CombineTest implements Serializable {
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.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eafdee5/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/ClassForDisplayTest.java
----------------------------------------------------------------------
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 19f56c6..0000000
--- 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));
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eafdee5/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchers.java
----------------------------------------------------------------------
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 c47b4c7..e3721b8 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
@@ -275,6 +275,20 @@ public class DisplayDataMatchers {
}
/**
+ * Creates a matcher that matches if the examined {@link DisplayData.Item} contains the
+ * specified namespace.
+ */
+ public static Matcher<DisplayData.Item<?>> hasNamespace(String namespace) {
+ return new FeatureMatcher<DisplayData.Item<?>, 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.
*/
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eafdee5/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java
----------------------------------------------------------------------
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 108409f..851769a 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 class DisplayDataTest {
.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 class DisplayDataTest {
@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<Item<?>> 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 class DisplayDataTest {
.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 class DisplayDataTest {
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"));
}
};
@@ -374,6 +370,39 @@ public class DisplayDataTest {
}
@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);
@@ -390,10 +419,10 @@ public class DisplayDataTest {
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 +646,6 @@ public class DisplayDataTest {
.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)));
}
@@ -635,9 +663,6 @@ public class DisplayDataTest {
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))));
assertThat(
items, hasItem(allOf(hasKey("duration"), hasType(DisplayData.Type.DURATION))));
@@ -731,8 +756,6 @@ public class DisplayDataTest {
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 +851,7 @@ public class DisplayDataTest {
DisplayData.from(new HasDisplayData() {
@Override
public void populateDisplayData(Builder builder) {
- builder.include(subComponent, (ClassForDisplay) null);
+ builder.include(subComponent, (Class<?>) null);
}
});
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5eafdee5/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/display/ClassForDisplayJava8Test.java
----------------------------------------------------------------------
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 8889a78..0000000
--- 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<Object, Object> 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);
- }
-}
[2/2] incubator-beam git commit: This closes #259
Posted by bc...@apache.org.
This closes #259
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/1262819a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/1262819a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/1262819a
Branch: refs/heads/master
Commit: 1262819aca92cbc150f6421c78d574e364f3dfb0
Parents: 659cf2e 5eafdee
Author: bchambers <bc...@google.com>
Authored: Mon May 2 13:11:00 2016 -0700
Committer: bchambers <bc...@google.com>
Committed: Mon May 2 13:11:00 2016 -0700
----------------------------------------------------------------------
.../org/apache/beam/sdk/transforms/Combine.java | 158 ++++++++++---------
.../beam/sdk/transforms/DoFnReflector.java | 2 +-
.../org/apache/beam/sdk/transforms/ParDo.java | 2 +-
.../sdk/transforms/display/ClassForDisplay.java | 94 -----------
.../sdk/transforms/display/DisplayData.java | 78 ++++-----
.../sdk/transforms/display/HasDisplayData.java | 55 ++++---
.../apache/beam/sdk/transforms/CombineTest.java | 16 ++
.../transforms/display/ClassForDisplayTest.java | 66 --------
.../transforms/display/DisplayDataMatchers.java | 14 ++
.../sdk/transforms/display/DisplayDataTest.java | 83 ++++++----
.../display/ClassForDisplayJava8Test.java | 45 ------
11 files changed, 234 insertions(+), 379 deletions(-)
----------------------------------------------------------------------