Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -88,7 +88,7 @@ public Map<Identifier, Item> asMap() {
public String toString() {
StringBuilder builder = new StringBuilder();
boolean isFirstLine = true;
for (Map.Entry<Identifier, Item> entry : entries.entrySet()) {
for (Item entry : entries.values()) {
if (isFirstLine) {
isFirstLine = false;
} else {
Expand All @@ -107,13 +107,18 @@ public String toString() {
*/
public interface Builder {
/**
* Include display metadata from the specified subcomponent. For example, a {@link ParDo}
* Register display metadata from the specified subcomponent. For example, a {@link ParDo}
* transform includes display metadata from the encapsulated {@link DoFn}.
*
* @return A builder instance to continue to build in a fluent-style.
*/
Builder include(HasDisplayData subComponent);

/**
* Register display metadata from the specified subcomponent, using the specified namespace.
* For example, a {@link ParDo} transform includes display metadata from the encapsulated
* {@link DoFn}.
*/
Builder include(HasDisplayData subComponent, Class<?> namespace);

/**
* Register the given string display metadata. The metadata item will be registered with type
* {@link DisplayData.Type#STRING}, and is identified by the specified key and namespace from
Expand All @@ -135,6 +140,13 @@ public interface Builder {
*/
ItemBuilder add(String key, double value);

/**
* Register the given floating point display metadata. The metadata item will be registered with
* type {@link DisplayData.Type#BOOLEAN}, and is identified by the specified key and namespace
* from the current transform or component.
*/
ItemBuilder add(String key, boolean value);

/**
* Register the given timestamp display metadata. The metadata item will be registered with type
* {@link DisplayData.Type#TIMESTAMP}, and is identified by the specified key and namespace from
Expand Down Expand Up @@ -287,7 +299,35 @@ public String getLinkUrl() {

@Override
public String toString() {
return getValue();
return String.format("%s:%s=%s", ns, key, value);
}

@Override
public boolean equals(Object obj) {
if (obj instanceof Item) {
Item that = (Item) obj;
return Objects.equals(this.ns, that.ns)
&& Objects.equals(this.key, that.key)
&& Objects.equals(this.type, that.type)
&& Objects.equals(this.value, that.value)
&& Objects.equals(this.shortValue, that.shortValue)
&& Objects.equals(this.label, that.label)
&& Objects.equals(this.url, that.url);
}

return false;
}

@Override
public int hashCode() {
return Objects.hash(
this.ns,
this.key,
this.type,
this.value,
this.shortValue,
this.label,
this.url);
}

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

static Identifier of(Class<?> namespace, String key) {
return new Identifier(namespace.getName(), key);
public static Identifier of(Class<?> namespace, String key) {
return of(namespace.getName(), key);
}

public static Identifier of(String namespace, String key) {
return new Identifier(namespace, key);
}

private Identifier(String ns, String key) {
Expand Down Expand Up @@ -355,7 +399,7 @@ public String toString() {
/**
* Display metadata type.
*/
enum Type {
public enum Type {
STRING {
@Override
FormattedItemValue format(Object value) {
Expand All @@ -374,6 +418,12 @@ FormattedItemValue format(Object value) {
return new FormattedItemValue(Double.toString((Double) value));
}
},
BOOLEAN() {
@Override
FormattedItemValue format(Object value) {
return new FormattedItemValue(Boolean.toString((boolean) value));
}
},
TIMESTAMP() {
@Override
FormattedItemValue format(Object value) {
Expand Down Expand Up @@ -403,7 +453,7 @@ FormattedItemValue format(Object value) {
abstract FormattedItemValue format(Object value);
}

private static class FormattedItemValue {
static class FormattedItemValue {
private final String shortValue;
private final String longValue;

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

private String getLongValue () {
String getLongValue() {
return this.longValue;
}

private String getShortValue() {
String getShortValue() {
return this.shortValue;
}
}
Expand All @@ -446,11 +496,17 @@ private static InternalBuilder forRoot(HasDisplayData instance) {

@Override
public Builder include(HasDisplayData subComponent) {
checkNotNull(subComponent);
return include(subComponent, subComponent.getClass());
}

@Override
public Builder include(HasDisplayData subComponent, Class<?> namespace) {
checkNotNull(subComponent);
boolean newComponent = visited.add(subComponent);
if (newComponent) {
Class prevNs = this.latestNs;
this.latestNs = subComponent.getClass();
this.latestNs = namespace;
subComponent.populateDisplayData(this);
this.latestNs = prevNs;
}
Expand All @@ -474,6 +530,11 @@ public ItemBuilder add(String key, double value) {
return addItem(key, Type.FLOAT, value);
}

@Override
public ItemBuilder add(String key, boolean value) {
return addItem(key, Type.BOOLEAN, value);
}

@Override
public ItemBuilder add(String key, Instant value) {
checkNotNull(value);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,13 +17,19 @@
*/
package com.google.cloud.dataflow.sdk.transforms.display;

import static org.hamcrest.Matchers.allOf;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Today we have no user-facing APIs that link with Hamcrest AFAIK. I think we are trying to sort out a good way to do that without causing dependency hell. (We redacted SerializableMatchers to be package-private because of this issue; we have a lot of value to add by solving it).

It looks like you might be able to get away with submitting it package-private for now.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see a few other matchers classes exposed:

  • DataflowMatchers
  • TestUtils.KVMatcher
  • WindowMatchers

Should these all be moved internal? I can follow-up with another PR.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I chatted with @lukecwik on this; since DisplayDataMatchers will be packaged in the test jar, it should be ok. This is the case with the others listed above as well.


import com.google.cloud.dataflow.sdk.transforms.display.DisplayData.Item;

import com.google.common.collect.Sets;
import org.hamcrest.CustomTypeSafeMatcher;
import org.hamcrest.Description;
import org.hamcrest.FeatureMatcher;
import org.hamcrest.Matcher;
import org.hamcrest.Matchers;
import org.hamcrest.TypeSafeDiagnosingMatcher;
import org.joda.time.Duration;
import org.joda.time.Instant;

import java.util.Collection;

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

/**
* Create a matcher that matches if the examined {@link DisplayData} contains an item with the
* specified key and String value.
*/
public static Matcher<DisplayData> hasDisplayItem(String key, String value) {
return hasDisplayItem(key, DisplayData.Type.STRING, value);
}

/**
* Create a matcher that matches if the examined {@link DisplayData} contains an item with the
* specified key and Boolean value.
*/
public static Matcher<DisplayData> hasDisplayItem(String key, Boolean value) {
return hasDisplayItem(key, DisplayData.Type.BOOLEAN, value);
}

/**
* Create a matcher that matches if the examined {@link DisplayData} contains an item with the
* specified key and Duration value.
*/
public static Matcher<DisplayData> hasDisplayItem(String key, Duration value) {
return hasDisplayItem(key, DisplayData.Type.DURATION, value);
}

/**
* Create a matcher that matches if the examined {@link DisplayData} contains an item with the
* specified key and Float value.
*/
public static Matcher<DisplayData> hasDisplayItem(String key, double value) {
return hasDisplayItem(key, DisplayData.Type.FLOAT, value);
}

/**
* Create a matcher that matches if the examined {@link DisplayData} contains an item with the
* specified key and Integer value.
*/
public static Matcher<DisplayData> hasDisplayItem(String key, long value) {
return hasDisplayItem(key, DisplayData.Type.INTEGER, value);
}

/**
* Create a matcher that matches if the examined {@link DisplayData} contains an item with the
* specified key and Class value.
*/
public static Matcher<DisplayData> hasDisplayItem(String key, Class<?> value) {
return hasDisplayItem(key, DisplayData.Type.JAVA_CLASS, value);
}

/**
* Create a matcher that matches if the examined {@link DisplayData} contains an item with the
* specified key and Timestamp value.
*/
public static Matcher<DisplayData> hasDisplayItem(String key, Instant value) {
return hasDisplayItem(key, DisplayData.Type.TIMESTAMP, value);
}

private static Matcher<DisplayData> hasDisplayItem(
String key, DisplayData.Type type, Object value) {
DisplayData.FormattedItemValue formattedValue = type.format(value);
return hasDisplayItem(allOf(
hasKey(key),
hasType(type),
hasValue(formattedValue.getLongValue())));
}

/**
* Creates a matcher that matches if the examined {@link DisplayData} contains any item
* matching the specified {@code itemMatcher}.
Expand All @@ -69,13 +140,93 @@ protected boolean matchesSafely(DisplayData data, Description mismatchDescriptio
Collection<Item> items = data.items();
boolean isMatch = Matchers.hasItem(itemMatcher).matches(items);
if (!isMatch) {
mismatchDescription.appendText("found " + items.size() + " non-matching items");
mismatchDescription.appendText("found " + items.size() + " non-matching item(s):\n");
mismatchDescription.appendValue(data);
}

return isMatch;
}
}

/**
* Create a matcher that matches if the examined {@link DisplayData} contains all display data
* registered from the specified subcomponent.
*/
public static Matcher<DisplayData> includes(final HasDisplayData subComponent) {
return includes(subComponent, subComponent.getClass());
}

/**
* Create a matcher that matches if the examined {@link DisplayData} contains all display data
* registered from the specified subcomponent and namespace.
*/
public static Matcher<DisplayData> includes(
final HasDisplayData subComponent, final Class<? extends HasDisplayData> namespace) {
return new CustomTypeSafeMatcher<DisplayData>("includes subcomponent") {
@Override
protected boolean matchesSafely(DisplayData displayData) {
DisplayData subComponentData = DisplayData.from(subComponent);
if (subComponentData.items().size() == 0) {
throw new UnsupportedOperationException("subComponent contains no display data; " +
"cannot verify whether it is included");
}

DisplayDataComparision comparison = checkSubset(displayData, subComponentData, namespace);
return comparison.missingItems.isEmpty();
}


@Override
protected void describeMismatchSafely(
DisplayData displayData, Description mismatchDescription) {
DisplayData subComponentDisplayData = DisplayData.from(subComponent);
DisplayDataComparision comparison = checkSubset(
displayData, subComponentDisplayData, subComponent.getClass());

mismatchDescription
.appendText("did not include:\n")
.appendValue(comparison.missingItems)
.appendText("\nNon-matching items:\n")
.appendValue(comparison.unmatchedItems);
}

private DisplayDataComparision checkSubset(
DisplayData displayData, DisplayData included, Class<?> namespace) {
DisplayDataComparision comparison = new DisplayDataComparision(displayData.items());
for (Item item : included.items()) {
Item matchedItem = displayData.asMap().get(
DisplayData.Identifier.of(namespace, item.getKey()));

if (matchedItem != null) {
comparison.matched(matchedItem);
} else {
comparison.missing(item);
}
}

return comparison;
}

class DisplayDataComparision {
Collection<DisplayData.Item> missingItems;
Collection<DisplayData.Item> unmatchedItems;

DisplayDataComparision(Collection<Item> superset) {
missingItems = Sets.newHashSet();
unmatchedItems = Sets.newHashSet(superset);
}

void matched(Item supersetItem) {
unmatchedItems.remove(supersetItem);
}

void missing(Item subsetItem) {
missingItems.add(subsetItem);
}
}
};
}

/**
* Creates a matcher that matches if the examined {@link DisplayData.Item} contains a key
* with the specified value.
Expand All @@ -96,4 +247,32 @@ protected String featureValueOf(DisplayData.Item actual) {
}
};
}

public static Matcher<DisplayData.Item> hasType(DisplayData.Type type) {
return hasType(Matchers.is(type));
}

public static Matcher<DisplayData.Item> hasType(Matcher<DisplayData.Type> typeMatcher) {
return new FeatureMatcher<DisplayData.Item, DisplayData.Type>(
typeMatcher, "with type", "type") {
@Override
protected DisplayData.Type featureValueOf(DisplayData.Item actual) {
return actual.getType();
}
};
}

public static Matcher<DisplayData.Item> hasValue(String value) {
return hasValue(Matchers.is(value));
}

public static Matcher<DisplayData.Item> hasValue(Matcher<String> valueMatcher) {
return new FeatureMatcher<DisplayData.Item, String>(
valueMatcher, "with value", "value") {
@Override
protected String featureValueOf(DisplayData.Item actual) {
return actual.getValue();
}
};
}
}
Loading