Skip to content

Commit 014a9a5

Browse files
swegnerbchambers
authored andcommitted
DisplayData tweaks based on transform usage.
* Add boolean-valued display data. * Implement equality for DislpayData.Item * Add ability to override namespace for included subcomponents. * Additional Matchers for testing display data * Update DisplayData inner class privacy
1 parent 3e71ed4 commit 014a9a5

4 files changed

Lines changed: 391 additions & 66 deletions

File tree

sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayData.java

Lines changed: 73 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -88,7 +88,7 @@ public Map<Identifier, Item> asMap() {
8888
public String toString() {
8989
StringBuilder builder = new StringBuilder();
9090
boolean isFirstLine = true;
91-
for (Map.Entry<Identifier, Item> entry : entries.entrySet()) {
91+
for (Item entry : entries.values()) {
9292
if (isFirstLine) {
9393
isFirstLine = false;
9494
} else {
@@ -107,13 +107,18 @@ public String toString() {
107107
*/
108108
public interface Builder {
109109
/**
110-
* Include display metadata from the specified subcomponent. For example, a {@link ParDo}
110+
* Register display metadata from the specified subcomponent. For example, a {@link ParDo}
111111
* transform includes display metadata from the encapsulated {@link DoFn}.
112-
*
113-
* @return A builder instance to continue to build in a fluent-style.
114112
*/
115113
Builder include(HasDisplayData subComponent);
116114

115+
/**
116+
* Register display metadata from the specified subcomponent, using the specified namespace.
117+
* For example, a {@link ParDo} transform includes display metadata from the encapsulated
118+
* {@link DoFn}.
119+
*/
120+
Builder include(HasDisplayData subComponent, Class<?> namespace);
121+
117122
/**
118123
* Register the given string display metadata. The metadata item will be registered with type
119124
* {@link DisplayData.Type#STRING}, and is identified by the specified key and namespace from
@@ -135,6 +140,13 @@ public interface Builder {
135140
*/
136141
ItemBuilder add(String key, double value);
137142

143+
/**
144+
* Register the given floating point display metadata. The metadata item will be registered with
145+
* type {@link DisplayData.Type#BOOLEAN}, and is identified by the specified key and namespace
146+
* from the current transform or component.
147+
*/
148+
ItemBuilder add(String key, boolean value);
149+
138150
/**
139151
* Register the given timestamp display metadata. The metadata item will be registered with type
140152
* {@link DisplayData.Type#TIMESTAMP}, and is identified by the specified key and namespace from
@@ -287,7 +299,35 @@ public String getLinkUrl() {
287299

288300
@Override
289301
public String toString() {
290-
return getValue();
302+
return String.format("%s:%s=%s", ns, key, value);
303+
}
304+
305+
@Override
306+
public boolean equals(Object obj) {
307+
if (obj instanceof Item) {
308+
Item that = (Item) obj;
309+
return Objects.equals(this.ns, that.ns)
310+
&& Objects.equals(this.key, that.key)
311+
&& Objects.equals(this.type, that.type)
312+
&& Objects.equals(this.value, that.value)
313+
&& Objects.equals(this.shortValue, that.shortValue)
314+
&& Objects.equals(this.label, that.label)
315+
&& Objects.equals(this.url, that.url);
316+
}
317+
318+
return false;
319+
}
320+
321+
@Override
322+
public int hashCode() {
323+
return Objects.hash(
324+
this.ns,
325+
this.key,
326+
this.type,
327+
this.value,
328+
this.shortValue,
329+
this.label,
330+
this.url);
291331
}
292332

293333
private Item withLabel(String label) {
@@ -313,8 +353,12 @@ public static class Identifier {
313353
private final String ns;
314354
private final String key;
315355

316-
static Identifier of(Class<?> namespace, String key) {
317-
return new Identifier(namespace.getName(), key);
356+
public static Identifier of(Class<?> namespace, String key) {
357+
return of(namespace.getName(), key);
358+
}
359+
360+
public static Identifier of(String namespace, String key) {
361+
return new Identifier(namespace, key);
318362
}
319363

320364
private Identifier(String ns, String key) {
@@ -355,7 +399,7 @@ public String toString() {
355399
/**
356400
* Display metadata type.
357401
*/
358-
enum Type {
402+
public enum Type {
359403
STRING {
360404
@Override
361405
FormattedItemValue format(Object value) {
@@ -374,6 +418,12 @@ FormattedItemValue format(Object value) {
374418
return new FormattedItemValue(Double.toString((Double) value));
375419
}
376420
},
421+
BOOLEAN() {
422+
@Override
423+
FormattedItemValue format(Object value) {
424+
return new FormattedItemValue(Boolean.toString((boolean) value));
425+
}
426+
},
377427
TIMESTAMP() {
378428
@Override
379429
FormattedItemValue format(Object value) {
@@ -403,7 +453,7 @@ FormattedItemValue format(Object value) {
403453
abstract FormattedItemValue format(Object value);
404454
}
405455

406-
private static class FormattedItemValue {
456+
static class FormattedItemValue {
407457
private final String shortValue;
408458
private final String longValue;
409459

@@ -416,11 +466,11 @@ private FormattedItemValue(String longValue, String shortValue) {
416466
this.shortValue = shortValue;
417467
}
418468

419-
private String getLongValue () {
469+
String getLongValue() {
420470
return this.longValue;
421471
}
422472

423-
private String getShortValue() {
473+
String getShortValue() {
424474
return this.shortValue;
425475
}
426476
}
@@ -446,11 +496,17 @@ private static InternalBuilder forRoot(HasDisplayData instance) {
446496

447497
@Override
448498
public Builder include(HasDisplayData subComponent) {
499+
checkNotNull(subComponent);
500+
return include(subComponent, subComponent.getClass());
501+
}
502+
503+
@Override
504+
public Builder include(HasDisplayData subComponent, Class<?> namespace) {
449505
checkNotNull(subComponent);
450506
boolean newComponent = visited.add(subComponent);
451507
if (newComponent) {
452508
Class prevNs = this.latestNs;
453-
this.latestNs = subComponent.getClass();
509+
this.latestNs = namespace;
454510
subComponent.populateDisplayData(this);
455511
this.latestNs = prevNs;
456512
}
@@ -474,6 +530,11 @@ public ItemBuilder add(String key, double value) {
474530
return addItem(key, Type.FLOAT, value);
475531
}
476532

533+
@Override
534+
public ItemBuilder add(String key, boolean value) {
535+
return addItem(key, Type.BOOLEAN, value);
536+
}
537+
477538
@Override
478539
public ItemBuilder add(String key, Instant value) {
479540
checkNotNull(value);

sdks/java/core/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchers.java

Lines changed: 180 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,13 +17,19 @@
1717
*/
1818
package com.google.cloud.dataflow.sdk.transforms.display;
1919

20+
import static org.hamcrest.Matchers.allOf;
21+
2022
import com.google.cloud.dataflow.sdk.transforms.display.DisplayData.Item;
2123

24+
import com.google.common.collect.Sets;
25+
import org.hamcrest.CustomTypeSafeMatcher;
2226
import org.hamcrest.Description;
2327
import org.hamcrest.FeatureMatcher;
2428
import org.hamcrest.Matcher;
2529
import org.hamcrest.Matchers;
2630
import org.hamcrest.TypeSafeDiagnosingMatcher;
31+
import org.joda.time.Duration;
32+
import org.joda.time.Instant;
2733

2834
import java.util.Collection;
2935

@@ -43,6 +49,71 @@ public static Matcher<DisplayData> hasDisplayItem() {
4349
return hasDisplayItem(Matchers.any(DisplayData.Item.class));
4450
}
4551

52+
/**
53+
* Create a matcher that matches if the examined {@link DisplayData} contains an item with the
54+
* specified key and String value.
55+
*/
56+
public static Matcher<DisplayData> hasDisplayItem(String key, String value) {
57+
return hasDisplayItem(key, DisplayData.Type.STRING, value);
58+
}
59+
60+
/**
61+
* Create a matcher that matches if the examined {@link DisplayData} contains an item with the
62+
* specified key and Boolean value.
63+
*/
64+
public static Matcher<DisplayData> hasDisplayItem(String key, Boolean value) {
65+
return hasDisplayItem(key, DisplayData.Type.BOOLEAN, value);
66+
}
67+
68+
/**
69+
* Create a matcher that matches if the examined {@link DisplayData} contains an item with the
70+
* specified key and Duration value.
71+
*/
72+
public static Matcher<DisplayData> hasDisplayItem(String key, Duration value) {
73+
return hasDisplayItem(key, DisplayData.Type.DURATION, value);
74+
}
75+
76+
/**
77+
* Create a matcher that matches if the examined {@link DisplayData} contains an item with the
78+
* specified key and Float value.
79+
*/
80+
public static Matcher<DisplayData> hasDisplayItem(String key, double value) {
81+
return hasDisplayItem(key, DisplayData.Type.FLOAT, value);
82+
}
83+
84+
/**
85+
* Create a matcher that matches if the examined {@link DisplayData} contains an item with the
86+
* specified key and Integer value.
87+
*/
88+
public static Matcher<DisplayData> hasDisplayItem(String key, long value) {
89+
return hasDisplayItem(key, DisplayData.Type.INTEGER, value);
90+
}
91+
92+
/**
93+
* Create a matcher that matches if the examined {@link DisplayData} contains an item with the
94+
* specified key and Class value.
95+
*/
96+
public static Matcher<DisplayData> hasDisplayItem(String key, Class<?> value) {
97+
return hasDisplayItem(key, DisplayData.Type.JAVA_CLASS, value);
98+
}
99+
100+
/**
101+
* Create a matcher that matches if the examined {@link DisplayData} contains an item with the
102+
* specified key and Timestamp value.
103+
*/
104+
public static Matcher<DisplayData> hasDisplayItem(String key, Instant value) {
105+
return hasDisplayItem(key, DisplayData.Type.TIMESTAMP, value);
106+
}
107+
108+
private static Matcher<DisplayData> hasDisplayItem(
109+
String key, DisplayData.Type type, Object value) {
110+
DisplayData.FormattedItemValue formattedValue = type.format(value);
111+
return hasDisplayItem(allOf(
112+
hasKey(key),
113+
hasType(type),
114+
hasValue(formattedValue.getLongValue())));
115+
}
116+
46117
/**
47118
* Creates a matcher that matches if the examined {@link DisplayData} contains any item
48119
* matching the specified {@code itemMatcher}.
@@ -69,13 +140,93 @@ protected boolean matchesSafely(DisplayData data, Description mismatchDescriptio
69140
Collection<Item> items = data.items();
70141
boolean isMatch = Matchers.hasItem(itemMatcher).matches(items);
71142
if (!isMatch) {
72-
mismatchDescription.appendText("found " + items.size() + " non-matching items");
143+
mismatchDescription.appendText("found " + items.size() + " non-matching item(s):\n");
144+
mismatchDescription.appendValue(data);
73145
}
74146

75147
return isMatch;
76148
}
77149
}
78150

151+
/**
152+
* Create a matcher that matches if the examined {@link DisplayData} contains all display data
153+
* registered from the specified subcomponent.
154+
*/
155+
public static Matcher<DisplayData> includes(final HasDisplayData subComponent) {
156+
return includes(subComponent, subComponent.getClass());
157+
}
158+
159+
/**
160+
* Create a matcher that matches if the examined {@link DisplayData} contains all display data
161+
* registered from the specified subcomponent and namespace.
162+
*/
163+
public static Matcher<DisplayData> includes(
164+
final HasDisplayData subComponent, final Class<? extends HasDisplayData> namespace) {
165+
return new CustomTypeSafeMatcher<DisplayData>("includes subcomponent") {
166+
@Override
167+
protected boolean matchesSafely(DisplayData displayData) {
168+
DisplayData subComponentData = DisplayData.from(subComponent);
169+
if (subComponentData.items().size() == 0) {
170+
throw new UnsupportedOperationException("subComponent contains no display data; " +
171+
"cannot verify whether it is included");
172+
}
173+
174+
DisplayDataComparision comparison = checkSubset(displayData, subComponentData, namespace);
175+
return comparison.missingItems.isEmpty();
176+
}
177+
178+
179+
@Override
180+
protected void describeMismatchSafely(
181+
DisplayData displayData, Description mismatchDescription) {
182+
DisplayData subComponentDisplayData = DisplayData.from(subComponent);
183+
DisplayDataComparision comparison = checkSubset(
184+
displayData, subComponentDisplayData, subComponent.getClass());
185+
186+
mismatchDescription
187+
.appendText("did not include:\n")
188+
.appendValue(comparison.missingItems)
189+
.appendText("\nNon-matching items:\n")
190+
.appendValue(comparison.unmatchedItems);
191+
}
192+
193+
private DisplayDataComparision checkSubset(
194+
DisplayData displayData, DisplayData included, Class<?> namespace) {
195+
DisplayDataComparision comparison = new DisplayDataComparision(displayData.items());
196+
for (Item item : included.items()) {
197+
Item matchedItem = displayData.asMap().get(
198+
DisplayData.Identifier.of(namespace, item.getKey()));
199+
200+
if (matchedItem != null) {
201+
comparison.matched(matchedItem);
202+
} else {
203+
comparison.missing(item);
204+
}
205+
}
206+
207+
return comparison;
208+
}
209+
210+
class DisplayDataComparision {
211+
Collection<DisplayData.Item> missingItems;
212+
Collection<DisplayData.Item> unmatchedItems;
213+
214+
DisplayDataComparision(Collection<Item> superset) {
215+
missingItems = Sets.newHashSet();
216+
unmatchedItems = Sets.newHashSet(superset);
217+
}
218+
219+
void matched(Item supersetItem) {
220+
unmatchedItems.remove(supersetItem);
221+
}
222+
223+
void missing(Item subsetItem) {
224+
missingItems.add(subsetItem);
225+
}
226+
}
227+
};
228+
}
229+
79230
/**
80231
* Creates a matcher that matches if the examined {@link DisplayData.Item} contains a key
81232
* with the specified value.
@@ -96,4 +247,32 @@ protected String featureValueOf(DisplayData.Item actual) {
96247
}
97248
};
98249
}
250+
251+
public static Matcher<DisplayData.Item> hasType(DisplayData.Type type) {
252+
return hasType(Matchers.is(type));
253+
}
254+
255+
public static Matcher<DisplayData.Item> hasType(Matcher<DisplayData.Type> typeMatcher) {
256+
return new FeatureMatcher<DisplayData.Item, DisplayData.Type>(
257+
typeMatcher, "with type", "type") {
258+
@Override
259+
protected DisplayData.Type featureValueOf(DisplayData.Item actual) {
260+
return actual.getType();
261+
}
262+
};
263+
}
264+
265+
public static Matcher<DisplayData.Item> hasValue(String value) {
266+
return hasValue(Matchers.is(value));
267+
}
268+
269+
public static Matcher<DisplayData.Item> hasValue(Matcher<String> valueMatcher) {
270+
return new FeatureMatcher<DisplayData.Item, String>(
271+
valueMatcher, "with value", "value") {
272+
@Override
273+
protected String featureValueOf(DisplayData.Item actual) {
274+
return actual.getValue();
275+
}
276+
};
277+
}
99278
}

0 commit comments

Comments
 (0)