diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BoundedReadFromUnboundedSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BoundedReadFromUnboundedSource.java index 52c730cc39..5a1a29a264 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BoundedReadFromUnboundedSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/BoundedReadFromUnboundedSource.java @@ -25,6 +25,7 @@ import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.util.IntervalBoundedExponentialBackOff; import com.google.cloud.dataflow.sdk.util.ValueWithRecordId; import com.google.cloud.dataflow.sdk.values.PCollection; @@ -104,6 +105,16 @@ public String getKindString() { return "Read(" + approximateSimpleName(source.getClass()) + ")"; } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + // We explicitly do not register base-class data, instead we use the delegate inner source. + builder + .add("source", source.getClass()) + .addIfNotDefault("maxRecords", maxNumRecords, Long.MAX_VALUE) + .addIfNotNull("maxReadTime", maxReadTime) + .include(source); + } + private static class UnboundedToBoundedSourceAdapter extends BoundedSource> { private final UnboundedSource source; diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/CompressedSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/CompressedSource.java index b0636ea35f..05f33d6ba1 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/CompressedSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/CompressedSource.java @@ -19,6 +19,7 @@ import com.google.cloud.dataflow.sdk.annotations.Experimental; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.common.base.Preconditions; import com.google.common.io.ByteStreams; import com.google.common.primitives.Ints; @@ -318,6 +319,22 @@ public final boolean producesSortedKeys(PipelineOptions options) throws Exceptio return sourceDelegate.producesSortedKeys(options); } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + // We explicitly do not register base-class data, instead we use the delegate inner source. + builder + .include(sourceDelegate) + .add("source", sourceDelegate.getClass()); + + if (channelFactory instanceof Enum) { + // GZIP and BZIP are implemented as enums; Enum classes are anonymous, so use the .name() + // value instead + builder.add("compressionMode", ((Enum) channelFactory).name()); + } else { + builder.add("compressionMode", channelFactory.getClass()); + } + } + /** * Returns the delegate source's default output coder. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java index f618bc9d63..482cdb57ba 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/DatastoreIO.java @@ -56,6 +56,7 @@ import com.google.cloud.dataflow.sdk.options.DataflowPipelineWorkerPoolOptions; import com.google.cloud.dataflow.sdk.options.GcpOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.util.AttemptBoundedExponentialBackOff; import com.google.cloud.dataflow.sdk.util.RetryHttpRequestInitializer; import com.google.cloud.dataflow.sdk.values.PCollection; @@ -396,6 +397,19 @@ public long getEstimatedSizeBytes(PipelineOptions options) throws Exception { return getPropertyMap(entity).get("entity_bytes").getIntegerValue(); } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder + .addIfNotDefault("host", host, DEFAULT_HOST) + .addIfNotNull("dataset", datasetId) + .addIfNotNull("namespace", namespace); + + if (query != null) { + builder.add("query", query.toString()); + } + } + @Override public String toString() { return MoreObjects.toStringHelper(getClass()) @@ -597,6 +611,13 @@ public void validate(PipelineOptions options) { public DatastoreWriteOperation createWriteOperation(PipelineOptions options) { return new DatastoreWriteOperation(this); } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder + .addIfNotDefault("host", host, DEFAULT_HOST) + .addIfNotNull("dataset", datasetId); + } } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSink.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSink.java index dda500c369..850ea01e45 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSink.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSink.java @@ -32,6 +32,7 @@ import com.google.cloud.dataflow.sdk.transforms.GroupByKey; import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.transforms.windowing.DefaultTrigger; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.Window; @@ -142,6 +143,30 @@ public void validate(PipelineOptions options) {} @Override public abstract FileBasedWriteOperation createWriteOperation(PipelineOptions options); + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + + String fileNamePattern = String.format("%s%s%s", + baseOutputFilename, fileNamingTemplate, getFileExtension(extension)); + builder.add("fileNamePattern", fileNamePattern); + } + + /** + * Returns the file extension to be used. If the user did not request a file + * extension then this method returns the empty string. Otherwise this method + * adds a {@code "."} to the beginning of the users extension if one is not present. + */ + private static String getFileExtension(String usersExtension) { + if (usersExtension == null || usersExtension.isEmpty()) { + return ""; + } + if (usersExtension.startsWith(".")) { + return usersExtension; + } + return "." + usersExtension; + } + /** * Abstract {@link Sink.WriteOperation} that manages the process of writing to a * {@link FileBasedSink}. @@ -363,21 +388,6 @@ protected final List generateDestinationFilenames(int numFiles) { return destFilenames; } - /** - * Returns the file extension to be used. If the user did not request a file - * extension then this method returns the empty string. Otherwise this method - * adds a {@code "."} to the beginning of the users extension if one is not present. - */ - private String getFileExtension(String usersExtension) { - if (usersExtension == null || usersExtension.isEmpty()) { - return ""; - } - if (usersExtension.startsWith(".")) { - return usersExtension; - } - return "." + usersExtension; - } - /** * Removes temporary output files. Uses the temporary filename to find files to remove. * diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSource.java index 5d32a9d08f..468a276b43 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/FileBasedSource.java @@ -15,6 +15,7 @@ package com.google.cloud.dataflow.sdk.io; import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.util.IOChannelFactory; import com.google.cloud.dataflow.sdk.util.IOChannelUtils; import com.google.common.base.Preconditions; @@ -257,6 +258,12 @@ private static long getEstimatedSizeOfFilesBySampling( / selectedFiles.size(); } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder.add("filePattern", getFileOrPatternSpec()); + } + private ListenableFuture>> createFutureForFileSplit( final String file, final long desiredBundleSizeBytes, diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/OffsetBasedSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/OffsetBasedSource.java index d581b80ca2..04b61b8771 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/OffsetBasedSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/OffsetBasedSource.java @@ -17,6 +17,7 @@ import com.google.cloud.dataflow.sdk.io.range.OffsetRangeTracker; import com.google.cloud.dataflow.sdk.io.range.RangeTracker; import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.common.base.Preconditions; import org.slf4j.Logger; @@ -198,6 +199,15 @@ public boolean allowsDynamicSplitting() { return true; } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder + .add("minBundleSize", minBundleSize) + .addIfNotDefault("startOffset", startOffset, 0) + .addIfNotDefault("endOffset", endOffset, Long.MAX_VALUE); + } + /** * A {@link Source.Reader} that implements code common to readers of all * {@link OffsetBasedSource}s. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Read.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Read.java index cde87696bb..803423b474 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Read.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Read.java @@ -21,6 +21,7 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.util.SerializableUtils; import com.google.cloud.dataflow.sdk.util.WindowedValue; import com.google.cloud.dataflow.sdk.util.WindowingStrategy; @@ -143,6 +144,13 @@ public String getKindString() { return "Read(" + approximateSimpleName(source.getClass()) + ")"; } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder + .add("source", source.getClass()) + .include(source); + } + static { registerDefaultTransformEvaluator(); } @@ -249,5 +257,12 @@ public final PCollection apply(PInput input) { public String getKindString() { return "Read(" + approximateSimpleName(source.getClass()) + ")"; } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder + .add("source", source.getClass()) + .include(source); + } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Sink.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Sink.java index a5649ceb5e..62bfbc1ad2 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Sink.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Sink.java @@ -17,6 +17,8 @@ import com.google.cloud.dataflow.sdk.annotations.Experimental; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; +import com.google.cloud.dataflow.sdk.transforms.display.HasDisplayData; import com.google.cloud.dataflow.sdk.values.PCollection; import java.io.Serializable; @@ -118,7 +120,7 @@ * @param the type that will be written to the Sink. */ @Experimental(Experimental.Kind.SOURCE_SINK) -public abstract class Sink implements Serializable { +public abstract class Sink implements Serializable, HasDisplayData { /** * Ensures that the sink is valid and can be written to before the write operation begins. One * should use {@link com.google.common.base.Preconditions} to implement this method. @@ -130,6 +132,15 @@ public abstract class Sink implements Serializable { */ public abstract WriteOperation createWriteOperation(PipelineOptions options); + /** + * {@inheritDoc} + * + *

By default, does not register any display data. Implementors may override this method + * to provide their own display metadata. + */ + @Override + public void populateDisplayData(DisplayData.Builder builder) {} + /** * A {@link WriteOperation} defines the process of a parallel write of objects to a Sink. * diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Source.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Source.java index 4a020787f5..05a86ca4a5 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Source.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Source.java @@ -18,6 +18,8 @@ import com.google.cloud.dataflow.sdk.annotations.Experimental; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; +import com.google.cloud.dataflow.sdk.transforms.display.HasDisplayData; import org.joda.time.Instant; @@ -51,7 +53,7 @@ * @param Type of elements read by the source. */ @Experimental(Experimental.Kind.SOURCE_SINK) -public abstract class Source implements Serializable { +public abstract class Source implements Serializable, HasDisplayData { /** * Checks that this source is valid, before it can be used in a pipeline. * @@ -65,6 +67,15 @@ public abstract class Source implements Serializable { */ public abstract Coder getDefaultOutputCoder(); + /** + * {@inheritDoc} + * + *

By default, does not register any display data. Implementors may override this method + * to provide their own display metadata. + */ + @Override + public void populateDisplayData(DisplayData.Builder builder) {} + /** * The interface that readers of custom input sources must implement. * diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Write.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Write.java index 65d3c992d8..2bbe878b2d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Write.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/Write.java @@ -26,6 +26,7 @@ import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.transforms.View; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.Window; import com.google.cloud.dataflow.sdk.values.PCollection; @@ -73,6 +74,13 @@ public PDone apply(PCollection input) { return createWrite(input, sink.createWriteOperation(options)); } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder + .add("sink", sink.getClass()) + .include(sink); + } + /** * Returns the {@link Sink} associated with this PTransform. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/XmlSink.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/XmlSink.java index b728c0a792..90d3bfbcdd 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/XmlSink.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/XmlSink.java @@ -20,6 +20,7 @@ import com.google.cloud.dataflow.sdk.io.FileBasedSink.FileBasedWriteOperation; import com.google.cloud.dataflow.sdk.io.FileBasedSink.FileBasedWriter; import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.util.CoderUtils; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.common.base.Preconditions; @@ -219,6 +220,14 @@ public void validate(PipelineOptions options) { public XmlWriteOperation createWriteOperation(PipelineOptions options) { return new XmlWriteOperation<>(this); } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder + .addIfNotNull("rootElement", rootElementName) + .addIfNotNull("recordClass", classToBind); + } } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/XmlSource.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/XmlSource.java index 1ead39187d..c8781f6e1d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/XmlSource.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/XmlSource.java @@ -18,6 +18,7 @@ import com.google.cloud.dataflow.sdk.coders.JAXBCoder; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.runners.PipelineRunner; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.common.base.Preconditions; import org.codehaus.stax2.XMLInputFactory2; @@ -34,7 +35,6 @@ import java.nio.channels.ReadableByteChannel; import java.nio.charset.StandardCharsets; import java.util.NoSuchElementException; - import javax.xml.bind.JAXBContext; import javax.xml.bind.JAXBElement; import javax.xml.bind.JAXBException; @@ -214,6 +214,15 @@ public void validate() { recordClass, "recordClass is null. Use builder method withRecordClass() to set this."); } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder + .addIfNotNull("rootElement", rootElement) + .addIfNotNull("recordElement", recordElement) + .addIfNotNull("recordClass", recordClass); + } + @Override public Coder getDefaultOutputCoder() { return JAXBCoder.of(recordClass); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionSpec.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionSpec.java new file mode 100644 index 0000000000..54e9fc7a1b --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionSpec.java @@ -0,0 +1,87 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import com.google.common.base.MoreObjects; +import com.google.common.base.Objects; + +import java.lang.reflect.Method; + +/** + * For internal use. Specification for an option defined in a {@link PipelineOptions} interface. + */ +class PipelineOptionSpec { + private final Class clazz; + private final String name; + private final Method getter; + + static PipelineOptionSpec of(Class clazz, String name, Method getter) { + return new PipelineOptionSpec(clazz, name, getter); + } + + private PipelineOptionSpec(Class clazz, String name, Method getter) { + this.clazz = clazz; + this.name = name; + this.getter = getter; + } + + /** + * The {@link PipelineOptions} interface which defines this {@link PipelineOptionSpec}. + */ + Class getDefiningInterface() { + return clazz; + } + + /** + * Name of the property. + */ + String getName() { + return name; + } + + /** + * The getter method for this property. + */ + Method getGetterMethod() { + return getter; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("definingInterface", getDefiningInterface()) + .add("name", getName()) + .add("getterMethod", getGetterMethod()) + .toString(); + } + + @Override + public int hashCode() { + return Objects.hashCode(getDefiningInterface(), getName(), getGetterMethod()); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof PipelineOptionSpec)) { + return false; + } + + PipelineOptionSpec that = (PipelineOptionSpec) obj; + return Objects.equal(this.getDefiningInterface(), that.getDefiningInterface()) + && Objects.equal(this.getName(), that.getName()) + && Objects.equal(this.getGetterMethod(), that.getGetterMethod()); + } +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptions.java index 41fa45c9fc..8022aa4e38 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptions.java @@ -25,6 +25,7 @@ import com.google.cloud.dataflow.sdk.runners.PipelineRunner; import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.DoFn.Context; +import com.google.cloud.dataflow.sdk.transforms.display.HasDisplayData; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.databind.ObjectMapper; @@ -192,7 +193,7 @@ @JsonSerialize(using = Serializer.class) @JsonDeserialize(using = Deserializer.class) @ThreadSafe -public interface PipelineOptions { +public interface PipelineOptions extends HasDisplayData { /** * Transforms this object into an object of type {@code } saving each property * that has been manipulated. {@code } must extend {@link PipelineOptions}. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java index 4781d1c829..872ac47099 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsFactory.java @@ -21,8 +21,10 @@ import com.google.cloud.dataflow.sdk.options.Validation.Required; import com.google.cloud.dataflow.sdk.runners.PipelineRunner; import com.google.cloud.dataflow.sdk.runners.PipelineRunnerRegistrar; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.util.StringUtils; import com.google.cloud.dataflow.sdk.util.common.ReflectHelpers; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Optional; @@ -30,7 +32,6 @@ import com.google.common.base.Predicate; import com.google.common.base.Strings; import com.google.common.base.Throwables; -import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Collections2; import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableListMultimap; @@ -41,8 +42,11 @@ import com.google.common.collect.ListMultimap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.collect.Ordering; +import com.google.common.collect.RowSortedTable; import com.google.common.collect.Sets; import com.google.common.collect.SortedSetMultimap; +import com.google.common.collect.TreeBasedTable; import com.google.common.collect.TreeMultimap; import com.fasterxml.jackson.annotation.JsonIgnore; @@ -75,6 +79,7 @@ import java.util.Set; import java.util.SortedMap; import java.util.SortedSet; +import java.util.TreeMap; import java.util.TreeSet; import javax.annotation.Nullable; @@ -442,6 +447,7 @@ Class getProxyClass() { @SuppressWarnings("rawtypes") private static final Class[] EMPTY_CLASS_ARRAY = new Class[0]; private static final ObjectMapper MAPPER = new ObjectMapper(); + private static final ClassLoader CLASS_LOADER; private static final Map>> SUPPORTED_PIPELINE_RUNNERS; /** Classes that are used as the boundary in the stack trace to find the callers class name. */ @@ -508,7 +514,7 @@ static ClassLoader findClassLoader() { throw new ExceptionInInitializerError(e); } - ClassLoader classLoader = findClassLoader(); + CLASS_LOADER = findClassLoader(); // Store the list of all available pipeline runners. ImmutableMap.Builder>> builder = @@ -516,25 +522,14 @@ static ClassLoader findClassLoader() { Set pipelineRunnerRegistrars = Sets.newTreeSet(ObjectsClassComparator.INSTANCE); pipelineRunnerRegistrars.addAll( - Lists.newArrayList(ServiceLoader.load(PipelineRunnerRegistrar.class, classLoader))); + Lists.newArrayList(ServiceLoader.load(PipelineRunnerRegistrar.class, CLASS_LOADER))); for (PipelineRunnerRegistrar registrar : pipelineRunnerRegistrars) { for (Class> klass : registrar.getPipelineRunners()) { builder.put(klass.getSimpleName(), klass); } } SUPPORTED_PIPELINE_RUNNERS = builder.build(); - - // Load and register the list of all classes that extend PipelineOptions. - register(PipelineOptions.class); - Set pipelineOptionsRegistrars = - Sets.newTreeSet(ObjectsClassComparator.INSTANCE); - pipelineOptionsRegistrars.addAll( - Lists.newArrayList(ServiceLoader.load(PipelineOptionsRegistrar.class, classLoader))); - for (PipelineOptionsRegistrar registrar : pipelineOptionsRegistrars) { - for (Class klass : registrar.getPipelineOptions()) { - register(klass); - } - } + initializeRegistry(); } /** @@ -563,6 +558,33 @@ public static synchronized void register(Class iface) REGISTERED_OPTIONS.add(iface); } + /** + * Resets the set of interfaces registered with this factory to the default state. + * + * @see PipelineOptionsFactory#register(Class) + */ + @VisibleForTesting + static synchronized void resetRegistry() { + REGISTERED_OPTIONS.clear(); + initializeRegistry(); + } + + /** + * Load and register the list of all classes that extend PipelineOptions. + */ + private static void initializeRegistry() { + register(PipelineOptions.class); + Set pipelineOptionsRegistrars = + Sets.newTreeSet(ObjectsClassComparator.INSTANCE); + pipelineOptionsRegistrars.addAll( + Lists.newArrayList(ServiceLoader.load(PipelineOptionsRegistrar.class, CLASS_LOADER))); + for (PipelineOptionsRegistrar registrar : pipelineOptionsRegistrars) { + for (Class klass : registrar.getPipelineOptions()) { + register(klass); + } + } + } + /** * Validates that the interface conforms to the following: *

    @@ -672,32 +694,20 @@ public static void printHelp(PrintStream out, Class i Preconditions.checkNotNull(iface); validateWellFormed(iface, REGISTERED_OPTIONS); - Iterable methods = - Iterables.filter( - ReflectHelpers.getClosureOfMethodsOnInterface(iface), NOT_SYNTHETIC_PREDICATE); - ListMultimap, Method> ifaceToMethods = ArrayListMultimap.create(); - for (Method method : methods) { - // Process only methods that are not marked as hidden. - if (method.getAnnotation(Hidden.class) == null) { - ifaceToMethods.put(method.getDeclaringClass(), method); - } + Set properties = + PipelineOptionsReflector.getOptionSpecs(iface); + + RowSortedTable, String, Method> ifacePropGetterTable = TreeBasedTable.create( + ClassNameComparator.INSTANCE, Ordering.natural()); + for (PipelineOptionSpec prop : properties) { + ifacePropGetterTable.put(prop.getDefiningInterface(), prop.getName(), prop.getGetterMethod()); } - SortedSet> ifaces = new TreeSet<>(ClassNameComparator.INSTANCE); - // Keep interfaces that are not marked as hidden. - ifaces.addAll(Collections2.filter(ifaceToMethods.keySet(), new Predicate>() { - @Override - public boolean apply(Class input) { - return input.getAnnotation(Hidden.class) == null; - } - })); - for (Class currentIface : ifaces) { - Map propertyNamesToGetters = - getPropertyNamesToGetters(ifaceToMethods.get(currentIface)); - // Don't output anything if there are no defined options - if (propertyNamesToGetters.isEmpty()) { - continue; - } + for (Map.Entry, Map> ifaceToPropertyMap : + ifacePropGetterTable.rowMap().entrySet()) { + Class currentIface = ifaceToPropertyMap.getKey(); + Map propertyNamesToGetters = ifaceToPropertyMap.getValue(); + SortedSetMultimap requiredGroupNameToProperties = getRequiredGroupNamesToProperties(propertyNamesToGetters); @@ -884,15 +894,21 @@ public static DataflowWorkerHarnessOptions createFromSystemProperties() throws I *

    TODO: Swap back to using Introspector once the proxy class issue with AppEngine is * resolved. */ - private static List getPropertyDescriptors(Class beanClass) + private static List getPropertyDescriptors( + Class beanClass) throws IntrospectionException { // The sorting is important to make this method stable. SortedSet methods = Sets.newTreeSet(MethodComparator.INSTANCE); methods.addAll( Collections2.filter(Arrays.asList(beanClass.getMethods()), NOT_SYNTHETIC_PREDICATE)); - SortedMap propertyNamesToGetters = getPropertyNamesToGetters(methods); - List descriptors = Lists.newArrayList(); + SortedMap propertyNamesToGetters = new TreeMap<>(); + for (Map.Entry entry : + PipelineOptionsReflector.getPropertyNamesToGetters(methods).entries()) { + propertyNamesToGetters.put(entry.getKey(), entry.getValue()); + } + + List descriptors = Lists.newArrayList(); List mismatches = new ArrayList<>(); /* * Add all the getter/setter pairs to the list of descriptors removing the getter once @@ -964,28 +980,6 @@ private static void throwForTypeMismatches(List mismatches) { } } - /** - * Returns a map of the property name to the getter method it represents. - * If there are duplicate methods with the same bean name, then it is indeterminate - * as to which method will be returned. - */ - private static SortedMap getPropertyNamesToGetters(Iterable methods) { - SortedMap propertyNamesToGetters = Maps.newTreeMap(); - for (Method method : methods) { - String methodName = method.getName(); - if ((!methodName.startsWith("get") - && !methodName.startsWith("is")) - || method.getParameterTypes().length != 0 - || method.getReturnType() == void.class) { - continue; - } - String propertyName = Introspector.decapitalize( - methodName.startsWith("is") ? methodName.substring(2) : methodName.substring(3)); - propertyNamesToGetters.put(propertyName, method); - } - return propertyNamesToGetters; - } - /** * Returns a map of required groups of arguments to the properties that satisfy the requirement. */ @@ -1027,21 +1021,22 @@ private static SortedSetMultimap getRequiredGroupNamesToProperti */ private static List validateClass(Class iface, Set> validatedPipelineOptionsInterfaces, - Class klass) throws IntrospectionException { + Class klass) throws IntrospectionException { Set methods = Sets.newHashSet(IGNORED_METHODS); - // Ignore static methods, "equals", "hashCode", "toString" and "as" on the generated class. // Ignore synthetic methods for (Method method : klass.getMethods()) { if (Modifier.isStatic(method.getModifiers()) || method.isSynthetic()) { methods.add(method); } } + // Ignore standard infrastructure methods on the generated class. try { methods.add(klass.getMethod("equals", Object.class)); methods.add(klass.getMethod("hashCode")); methods.add(klass.getMethod("toString")); methods.add(klass.getMethod("as", Class.class)); methods.add(klass.getMethod("cloneAs", Class.class)); + methods.add(klass.getMethod("populateDisplayData", DisplayData.Builder.class)); } catch (NoSuchMethodException | SecurityException e) { throw Throwables.propagate(e); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsReflector.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsReflector.java new file mode 100644 index 0000000000..094590a322 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsReflector.java @@ -0,0 +1,110 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import com.google.cloud.dataflow.sdk.util.common.ReflectHelpers; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Multimap; + +import java.beans.Introspector; +import java.lang.reflect.Method; +import java.util.Map; +import java.util.Set; + +/** + * Utilities to reflect over {@link PipelineOptions}. + */ +class PipelineOptionsReflector { + private PipelineOptionsReflector() {} + + /** + * Retrieve metadata for the full set of pipeline options visible within the type hierarchy + * of a single {@link PipelineOptions} interface. + * + * @see PipelineOptionsReflector#getOptionSpecs(Iterable) + */ + static Set getOptionSpecs(Class optionsInterface) { + Iterable methods = ReflectHelpers.getClosureOfMethodsOnInterface(optionsInterface); + Multimap propsToGetters = getPropertyNamesToGetters(methods); + + ImmutableSet.Builder setBuilder = ImmutableSet.builder(); + for (Map.Entry propAndGetter : propsToGetters.entries()) { + String prop = propAndGetter.getKey(); + Method getter = propAndGetter.getValue(); + + @SuppressWarnings("unchecked") + Class declaringClass = + (Class) getter.getDeclaringClass(); + + if (!PipelineOptions.class.isAssignableFrom(declaringClass)) { + continue; + } + + if (declaringClass.isAnnotationPresent(Hidden.class)) { + continue; + } + + setBuilder.add(PipelineOptionSpec.of(declaringClass, prop, getter)); + } + + return setBuilder.build(); + } + + /** + * Retrieve metadata for the full set of pipeline options visible within the type hierarchy + * closure of the set of input interfaces. An option is "visible" if: + *

    + *

      + *
    • The option is defined within the interface hierarchy closure of the input + * {@link PipelineOptions}.
    • + *
    • The defining interface is not marked {@link Hidden}.
    • + *
    + */ + static Set getOptionSpecs( + Iterable> optionsInterfaces) { + ImmutableSet.Builder setBuilder = ImmutableSet.builder(); + for (Class optionsInterface : optionsInterfaces) { + setBuilder.addAll(getOptionSpecs(optionsInterface)); + } + + return setBuilder.build(); + } + + /** + * Extract pipeline options and their respective getter methods from a series of + * {@link Method methods}. A single pipeline option may appear in many methods. + * + * @return A mapping of option name to the input methods which declare it. + */ + static Multimap getPropertyNamesToGetters(Iterable methods) { + Multimap propertyNamesToGetters = HashMultimap.create(); + for (Method method : methods) { + String methodName = method.getName(); + if ((!methodName.startsWith("get") + && !methodName.startsWith("is")) + || method.getParameterTypes().length != 0 + || method.getReturnType() == void.class) { + continue; + } + String propertyName = Introspector.decapitalize( + methodName.startsWith("is") ? methodName.substring(2) : methodName.substring(3)); + propertyNamesToGetters.put(propertyName, method); + } + return propertyNamesToGetters; + } + +} diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java index 527f712ca4..fce2e18993 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandler.java @@ -16,8 +16,11 @@ package com.google.cloud.dataflow.sdk.options; +import com.google.api.client.repackaged.com.google.common.base.Objects; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory.JsonIgnorePredicate; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory.Registration; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; +import com.google.cloud.dataflow.sdk.transforms.display.HasDisplayData; import com.google.cloud.dataflow.sdk.util.InstanceBuilder; import com.google.cloud.dataflow.sdk.util.common.ReflectHelpers; import com.google.common.base.Defaults; @@ -25,8 +28,11 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ClassToInstanceMap; import com.google.common.collect.FluentIterable; +import com.google.common.collect.HashMultimap; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.collect.Multimap; import com.google.common.collect.MutableClassToInstanceMap; import com.fasterxml.jackson.annotation.JsonIgnore; @@ -50,6 +56,8 @@ import java.lang.reflect.Proxy; import java.lang.reflect.Type; import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; @@ -58,6 +66,7 @@ import java.util.SortedMap; import java.util.TreeMap; +import javax.annotation.Nullable; import javax.annotation.concurrent.ThreadSafe; /** @@ -83,16 +92,26 @@ class ProxyInvocationHandler implements InvocationHandler { private final int hashCode = (int) (Math.random() * Integer.MAX_VALUE); private final Set> knownInterfaces; private final ClassToInstanceMap interfaceToProxyCache; - private final Map options; + private final Map options; private final Map jsonOptions; private final Map gettersToPropertyNames; private final Map settersToPropertyNames; ProxyInvocationHandler(Map options) { - this(options, Maps.newHashMap()); + this(bindOptions(options), Maps.newHashMap()); } - private ProxyInvocationHandler(Map options, Map jsonOptions) { + private static Map bindOptions(Map inputOptions) { + HashMap options = Maps.newHashMap(); + for (Map.Entry entry : inputOptions.entrySet()) { + options.put(entry.getKey(), BoundValue.fromExplicitOption(entry.getValue())); + } + + return options; + } + + private ProxyInvocationHandler( + Map options, Map jsonOptions) { this.options = options; this.jsonOptions = jsonOptions; this.knownInterfaces = new HashSet<>(PipelineOptionsFactory.getRegisteredOptions()); @@ -117,21 +136,27 @@ public Object invoke(Object proxy, Method method, Object[] args) { @SuppressWarnings("unchecked") Class clazz = (Class) args[0]; return cloneAs(proxy, clazz); + } else if (args != null && "populateDisplayData".equals(method.getName()) + && args[0] instanceof DisplayData.Builder) { + @SuppressWarnings("unchecked") + DisplayData.Builder builder = (DisplayData.Builder) args[0]; + populateDisplayData(builder); + return Void.TYPE; } String methodName = method.getName(); synchronized (this) { - if (gettersToPropertyNames.keySet().contains(methodName)) { + if (gettersToPropertyNames.containsKey(methodName)) { String propertyName = gettersToPropertyNames.get(methodName); if (!options.containsKey(propertyName)) { // Lazy bind the default to the method. Object value = jsonOptions.containsKey(propertyName) ? getValueFromJson(propertyName, method) : getDefault((PipelineOptions) proxy, method); - options.put(propertyName, value); + options.put(propertyName, BoundValue.fromDefault(value)); } - return options.get(propertyName); + return options.get(propertyName).getValue(); } else if (settersToPropertyNames.containsKey(methodName)) { - options.put(settersToPropertyNames.get(methodName), args[0]); + options.put(settersToPropertyNames.get(methodName), BoundValue.fromExplicitOption(args[0])); return Void.TYPE; } } @@ -139,6 +164,70 @@ public Object invoke(Object proxy, Method method, Object[] args) { + Arrays.toString(args) + "]."); } + /** + * Track whether options values are explicitly set, or retrieved from defaults. + */ + static final class BoundValue { + @Nullable + private final Object value; + private final boolean isDefault; + + @Nullable + Object getValue() { + return value; + } + + boolean isDefault() { + return isDefault; + } + + private static BoundValue of(@Nullable Object value, boolean isDefault) { + return new BoundValue(value, isDefault); + } + + private BoundValue(@Nullable Object value, boolean isDefault) { + this.value = value; + this.isDefault = isDefault; + } + + /** + * Create a {@link BoundValue} representing an explicitly set option. + */ + static BoundValue fromExplicitOption(@Nullable Object value) { + return BoundValue.of(value, false); + } + + /** + * Create a {@link BoundValue} representing a default option value. + */ + static BoundValue fromDefault(@Nullable Object value) { + return BoundValue.of(value, true); + } + + @Override + public String toString() { + return Objects.toStringHelper(BoundValue.class) + .add("value", value) + .add("isDefault", isDefault) + .toString(); + } + + @Override + public int hashCode() { + return Objects.hashCode(value, isDefault); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof BoundValue)) { + return false; + } + + BoundValue that = (BoundValue) obj; + return Objects.equal(this.value, that.value) && Objects.equal(this.isDefault, that.isDefault); + } + } + /** * Backing implementation for {@link PipelineOptions#as(Class)}. * @@ -207,6 +296,128 @@ public int hashCode() { return hashCode; } + /** + * Populate display data. See {@link HasDisplayData#populateDisplayData}. All explicitly set + * pipeline options will be added as display data. + */ + private void populateDisplayData(DisplayData.Builder builder) { + Set optionSpecs = PipelineOptionsReflector.getOptionSpecs(knownInterfaces); + Multimap optionsMap = buildOptionNameToSpecMap(optionSpecs); + + for (Map.Entry option : options.entrySet()) { + BoundValue boundValue = option.getValue(); + if (boundValue.isDefault()) { + continue; + } + + Object value = boundValue.getValue() == null ? "" : boundValue.getValue(); + DisplayData.Type type = DisplayData.inferType(value); + HashSet specs = new HashSet<>(optionsMap.get(option.getKey())); + + for (PipelineOptionSpec optionSpec : specs) { + Class pipelineInterface = optionSpec.getDefiningInterface(); + if (type != null) { + builder.add(option.getKey(), type, value) + .withNamespace(pipelineInterface); + } else { + builder.add(option.getKey(), value.toString()) + .withNamespace(pipelineInterface); + } + } + } + + for (Map.Entry jsonOption : jsonOptions.entrySet()) { + if (options.containsKey(jsonOption.getKey())) { + // Option overwritten since deserialization; don't re-write + continue; + } + + HashSet specs = new HashSet<>(optionsMap.get(jsonOption.getKey())); + if (specs.isEmpty()) { + builder.add(jsonOption.getKey(), jsonOption.getValue().toString()) + .withNamespace(UnknownPipelineOptions.class); + } else { + for (PipelineOptionSpec spec : specs) { + Object value = getValueFromJson(jsonOption.getKey(), spec.getGetterMethod()); + DisplayData.Type type = DisplayData.inferType(value); + if (type != null) { + builder.add(jsonOption.getKey(), type, value) + .withNamespace(spec.getDefiningInterface()); + } else { + builder.add(jsonOption.getKey(), value.toString()) + .withNamespace(spec.getDefiningInterface()); + } + } + } + } + } + + /** + * Marker interface used when the original {@link PipelineOptions} interface is not known at + * runtime. This can occur if {@link PipelineOptions} are deserialized from JSON. + *

    + *

    Pipeline authors can ensure {@link PipelineOptions} type information is available at + * runtime by registering their {@link PipelineOptions options} interfaces. See the "Registration" + * section of {@link PipelineOptions} documentation. + */ + interface UnknownPipelineOptions extends PipelineOptions {} + + /** + * Construct a mapping from an option name to its {@link PipelineOptions} interface(s) + * declarations. An option may be declared in multiple interfaces. If it is overridden in a + * type hierarchy, only the overriding interface will be included. + */ + private Multimap buildOptionNameToSpecMap( + Set props) { + + Multimap optionsMap = HashMultimap.create(); + for (PipelineOptionSpec prop : props) { + optionsMap.put(prop.getName(), prop); + } + + // Filter out overridden options + for (Map.Entry> entry : optionsMap.asMap().entrySet()) { + + /* Compare all interfaces for an option pairwise (iface1, iface2) to look for type + hierarchies. If one is the base-class of the other, remove it from the output and continue + iterating. + + This is an N^2 operation per-option, but the number of interfaces defining an option + should always be small (usually 1). */ + List specs = Lists.newArrayList(entry.getValue()); + if (specs.size() < 2) { + // Only one known implementing interface, no need to check for inheritance + continue; + } + + for (int i = 0; i < specs.size() - 1; i++) { + Class iface1 = specs.get(i).getDefiningInterface(); + for (int j = i + 1; j < specs.size(); j++) { + Class iface2 = specs.get(j).getDefiningInterface(); + + if (iface1.isAssignableFrom(iface2)) { + optionsMap.remove(entry.getKey(), specs.get(i)); + specs.remove(i); + + // Removed element at current "i" index. Set iterators to re-evaluate + // new "i" element in outer loop. + i--; + j = specs.size(); + } else if (iface2.isAssignableFrom(iface1)) { + optionsMap.remove(entry.getKey(), specs.get(j)); + specs.remove(j); + + // Removed element at current "j" index. Set iterator to re-evaluate + // new "j" element in inner-loop. + j--; + } + } + } + } + + return optionsMap; + } + /** * This will output all the currently set values. This is a relatively costly function * as it will call {@code toString()} on each object that has been set and format @@ -220,7 +431,9 @@ public synchronized String toString() { // Add the options that we received from deserialization sortedOptions.putAll(jsonOptions); // Override with any programmatically set options. - sortedOptions.putAll(options); + for (Map.Entry entry : options.entrySet()) { + sortedOptions.put(entry.getKey(), entry.getValue().getValue()); + } StringBuilder b = new StringBuilder(); b.append("Current Settings:\n"); @@ -345,7 +558,7 @@ public void serialize(PipelineOptions value, JsonGenerator jgen, SerializerProvi // We first filter out any properties that have been modified since // the last serialization of this PipelineOptions and then verify that // they are all serializable. - Map filteredOptions = Maps.newHashMap(handler.options); + Map filteredOptions = Maps.newHashMap(handler.options); removeIgnoredOptions(handler.knownInterfaces, filteredOptions); ensureSerializable(handler.knownInterfaces, filteredOptions); @@ -354,10 +567,23 @@ public void serialize(PipelineOptions value, JsonGenerator jgen, SerializerProvi // instances that have been modified since the previous serialization. Map serializableOptions = Maps.newHashMap(handler.jsonOptions); - serializableOptions.putAll(filteredOptions); + for (Map.Entry entry : filteredOptions.entrySet()) { + serializableOptions.put(entry.getKey(), entry.getValue().getValue()); + } + jgen.writeStartObject(); jgen.writeFieldName("options"); jgen.writeObject(serializableOptions); + + List> serializedDisplayData = Lists.newArrayList(); + for (DisplayData.Item item : DisplayData.from(value).items()) { + @SuppressWarnings("unchecked") + Map serializedItem = MAPPER.convertValue(item, Map.class); + serializedDisplayData.add(serializedItem); + } + + jgen.writeFieldName("display_data"); + jgen.writeObject(serializedDisplayData); jgen.writeEndObject(); } } @@ -367,7 +593,7 @@ public void serialize(PipelineOptions value, JsonGenerator jgen, SerializerProvi * {@link JsonIgnore @JsonIgnore} from the passed in options using the passed in interfaces. */ private void removeIgnoredOptions( - Set> interfaces, Map options) { + Set> interfaces, Map options) { // Find all the method names that are annotated with JSON ignore. Set jsonIgnoreMethodNames = FluentIterable.from( ReflectHelpers.getClosureOfMethodsOnInterfaces(interfaces)) @@ -392,7 +618,7 @@ public String apply(Method input) { * and deserializable. */ private void ensureSerializable(Set> interfaces, - Map options) throws IOException { + Map options) throws IOException { // Construct a map from property name to the return type of the getter. Map propertyToReturnType = Maps.newHashMap(); for (PropertyDescriptor descriptor @@ -404,16 +630,16 @@ private void ensureSerializable(Set> interfaces } // Attempt to serialize and deserialize each property. - for (Map.Entry entry : options.entrySet()) { + for (Map.Entry entry : options.entrySet()) { try { - String serializedValue = MAPPER.writeValueAsString(entry.getValue()); + String serializedValue = MAPPER.writeValueAsString(entry.getValue().getValue()); JavaType type = MAPPER.getTypeFactory() .constructType(propertyToReturnType.get(entry.getKey())); MAPPER.readValue(serializedValue, type); } catch (Exception e) { throw new IOException(String.format( "Failed to serialize and deserialize property '%s' with value '%s'", - entry.getKey(), entry.getValue()), e); + entry.getKey(), entry.getValue().getValue()), e); } } } @@ -433,7 +659,7 @@ public PipelineOptions deserialize(JsonParser jp, DeserializationContext ctxt) fields.put(field.getKey(), field.getValue()); } PipelineOptions options = - new ProxyInvocationHandler(Maps.newHashMap(), fields) + new ProxyInvocationHandler(Maps.newHashMap(), fields) .as(PipelineOptions.class); return options; } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java index 75fd5b3f93..32dfb876ff 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslator.java @@ -80,7 +80,6 @@ import com.google.cloud.dataflow.sdk.values.TypedPValue; import com.google.common.base.Preconditions; import com.google.common.base.Strings; -import com.google.common.collect.Lists; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; @@ -729,12 +728,8 @@ private void addOutput(String name, PValue value, Coder valueCoder) { } private void addDisplayData(String name, DisplayData displayData) { - List> serializedItems = Lists.newArrayList(); - for (DisplayData.Item item : displayData.items()) { - serializedItems.add(MAPPER.convertValue(item, Map.class)); - } - - addList(getProperties(), name, serializedItems); + List> list = MAPPER.convertValue(displayData, List.class); + addList(getProperties(), name, list); } @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ForwardingPTransform.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ForwardingPTransform.java index b736e35d31..cd04336d05 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ForwardingPTransform.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/ForwardingPTransform.java @@ -18,6 +18,7 @@ import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.values.PInput; import com.google.cloud.dataflow.sdk.values.POutput; import com.google.cloud.dataflow.sdk.values.TypedPValue; @@ -51,4 +52,9 @@ public Coder getDefaultOutputCoder(InputT input, @SuppressWarnings("unuse TypedPValue output) throws CannotProvideCoderException { return delegate().getDefaultOutputCoder(input, output); } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + delegate().populateDisplayData(builder); + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java index 57dd51009b..0f4e738993 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantiles.java @@ -24,6 +24,7 @@ import com.google.cloud.dataflow.sdk.coders.ListCoder; import com.google.cloud.dataflow.sdk.transforms.Combine.AccumulatingCombineFn; import com.google.cloud.dataflow.sdk.transforms.Combine.AccumulatingCombineFn.Accumulator; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.util.WeightedValue; import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; import com.google.cloud.dataflow.sdk.values.KV; @@ -357,6 +358,13 @@ public Coder> getAccumulatorCoder( CoderRegistry registry, Coder elementCoder) { return new QuantileStateCoder<>(compareFn, elementCoder); } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder + .add("numQuantiles", numQuantiles) + .add("comparer", compareFn.getClass()); + } } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUnique.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUnique.java index 3c936a2b13..27176f6f33 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUnique.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUnique.java @@ -23,12 +23,15 @@ import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.coders.SerializableCoder; import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.common.hash.Hashing; import com.google.common.hash.HashingOutputStream; import com.google.common.io.ByteStreams; +import org.apache.avro.reflect.Nullable; + import java.io.IOException; import java.io.Serializable; import java.util.Arrays; @@ -165,6 +168,12 @@ static class Globally extends PTransform, PCollection> { */ private final long sampleSize; + /** + * The desired maximum estimation error or null if not specified. + */ + @Nullable + private final Double maximumEstimationError; + /** * @see ApproximateUnique#globally(int) */ @@ -176,7 +185,9 @@ public Globally(int sampleSize) { + "In general, the estimation " + "error is about 2 / sqrt(sampleSize)."); } + this.sampleSize = sampleSize; + this.maximumEstimationError = null; } /** @@ -188,7 +199,9 @@ public Globally(double maximumEstimationError) { "ApproximateUnique needs an " + "estimation error between 1% (0.01) and 50% (0.5)."); } + this.sampleSize = sampleSizeFromEstimationError(maximumEstimationError); + this.maximumEstimationError = maximumEstimationError; } @Override @@ -198,6 +211,11 @@ public PCollection apply(PCollection input) { Combine.globally( new ApproximateUniqueCombineFn<>(sampleSize, coder))); } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + ApproximateUnique.populateDisplayData(builder, sampleSize, maximumEstimationError); + } } /** @@ -211,8 +229,18 @@ public PCollection apply(PCollection input) { static class PerKey extends PTransform>, PCollection>> { + /** + * The number of entries in the statistical sample; the higher this number, + * the more accurate the estimate will be. + */ private final long sampleSize; + /** + * The the desired maximum estimation error or null if not specified. + */ + @Nullable + private final Double maximumEstimationError; + /** * @see ApproximateUnique#perKey(int) */ @@ -223,7 +251,9 @@ public PerKey(int sampleSize) { + "sampleSize >= 16 for an estimation error <= 50%. In general, " + "the estimation error is about 2 / sqrt(sampleSize)."); } + this.sampleSize = sampleSize; + this.maximumEstimationError = null; } /** @@ -235,7 +265,9 @@ public PerKey(double estimationError) { "ApproximateUnique.PerKey needs an " + "estimation error between 1% (0.01) and 50% (0.5)."); } + this.sampleSize = sampleSizeFromEstimationError(estimationError); + this.maximumEstimationError = estimationError; } @Override @@ -252,6 +284,11 @@ public PCollection> apply(PCollection> input) { Combine.perKey(new ApproximateUniqueCombineFn<>( sampleSize, coder).asKeyedFn())); } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + ApproximateUnique.populateDisplayData(builder, sampleSize, maximumEstimationError); + } } @@ -416,4 +453,11 @@ static long hash(T element, Coder coder) throws CoderException, IOExcepti static long sampleSizeFromEstimationError(double estimationError) { return Math.round(Math.ceil(4.0 / Math.pow(estimationError, 2.0))); } + + private static void populateDisplayData( + DisplayData.Builder builder, long sampleSize, Double maxEstimationError) { + builder + .add("sampleSize", sampleSize) + .addIfNotNull("maximumEstimationError", maxEstimationError); + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java index b8d20e303f..22afcce409 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Combine.java @@ -34,6 +34,9 @@ import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.Context; import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext; import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.RequiresContextInternal; +import com.google.cloud.dataflow.sdk.transforms.display.ClassForDisplay; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; +import com.google.cloud.dataflow.sdk.transforms.display.HasDisplayData; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.Window; @@ -97,9 +100,10 @@ private Combine() { */ public static Globally globally( SerializableFunction, V> combiner) { - return globally(IterableCombineFn.of(combiner)); + return globally(IterableCombineFn.of(combiner), ClassForDisplay.fromInstance(combiner)); } + /** * Returns a {@link Globally Combine.Globally} {@code PTransform} * that uses the given {@code GloballyCombineFn} to combine all @@ -117,7 +121,12 @@ public static Globally globally( */ public static Globally globally( GlobalCombineFn fn) { - return new Globally<>(fn, true, 0); + return globally(fn, ClassForDisplay.fromInstance(fn)); + } + + private static Globally globally( + GlobalCombineFn fn, ClassForDisplay fnClass) { + return new Globally<>(fn, fnClass, true, 0); } /** @@ -138,7 +147,7 @@ public static Globally globally( */ public static PerKey perKey( SerializableFunction, V> fn) { - return perKey(Combine.IterableCombineFn.of(fn)); + return perKey(IterableCombineFn.of(fn).asKeyedFn(), ClassForDisplay.fromInstance(fn)); } /** @@ -159,7 +168,7 @@ public static PerKey perKey( */ public static PerKey perKey( GlobalCombineFn fn) { - return perKey(fn.asKeyedFn()); + return perKey(fn.asKeyedFn(), ClassForDisplay.fromInstance(fn)); } /** @@ -180,7 +189,12 @@ public static PerKey perKey( */ public static PerKey perKey( PerKeyCombineFn fn) { - return new PerKey<>(fn, false /*fewKeys*/); + return perKey(fn, ClassForDisplay.fromInstance(fn)); + } + + private static PerKey perKey( + PerKeyCombineFn fn, ClassForDisplay fnClass) { + return new PerKey<>(fn, fnClass, false /*fewKeys*/); } /** @@ -188,8 +202,8 @@ public static PerKey perKey( * in {@link GroupByKey}. */ private static PerKey fewKeys( - PerKeyCombineFn fn) { - return new PerKey<>(fn, true /*fewKeys*/); + PerKeyCombineFn fn, ClassForDisplay fnClass) { + return new PerKey<>(fn, fnClass, true /*fewKeys*/); } /** @@ -215,7 +229,7 @@ private static PerKey fewKeys( */ public static GroupedValues groupedValues( SerializableFunction, V> fn) { - return groupedValues(IterableCombineFn.of(fn)); + return groupedValues(IterableCombineFn.of(fn).asKeyedFn(), ClassForDisplay.fromInstance(fn)); } /** @@ -241,7 +255,7 @@ public static GroupedValues groupedValues( */ public static GroupedValues groupedValues( GlobalCombineFn fn) { - return groupedValues(fn.asKeyedFn()); + return groupedValues(fn.asKeyedFn(), ClassForDisplay.fromInstance(fn)); } /** @@ -267,9 +281,13 @@ public static GroupedValues groupedValu */ public static GroupedValues groupedValues( PerKeyCombineFn fn) { - return new GroupedValues<>(fn); + return groupedValues(fn, ClassForDisplay.fromInstance(fn)); } + private static GroupedValues groupedValues( + PerKeyCombineFn fn, ClassForDisplay fnClass) { + return new GroupedValues<>(fn, fnClass); + } ///////////////////////////////////////////////////////////////////////////// @@ -491,6 +509,11 @@ public Coder getDefaultOutputCoder( public CombineFn forKey(K key, Coder keyCoder) { return CombineFn.this; } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + CombineFn.this.populateDisplayData(builder); + } }; } } @@ -1164,6 +1187,11 @@ public Coder getDefaultOutputCoder( CoderRegistry registry, Coder inputCoder) throws CannotProvideCoderException { return KeyedCombineFn.this.getDefaultOutputCoder(registry, keyCoder, inputCoder); } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + KeyedCombineFn.this.populateDisplayData(builder); + } }; } @@ -1229,31 +1257,36 @@ public static class Globally extends PTransform, PCollection> { private final GlobalCombineFn fn; + private final ClassForDisplay fnClass; private final boolean insertDefault; private final int fanout; private final List> sideInputs; - private Globally(GlobalCombineFn fn, + private Globally(GlobalCombineFn fn, ClassForDisplay fnClass, boolean insertDefault, int fanout) { this.fn = fn; + this.fnClass = fnClass; this.insertDefault = insertDefault; this.fanout = fanout; this.sideInputs = ImmutableList.>of(); } private Globally(String name, GlobalCombineFn fn, - boolean insertDefault, int fanout) { + ClassForDisplay fnClass, boolean insertDefault, int fanout) { super(name); this.fn = fn; + this.fnClass = fnClass; this.insertDefault = insertDefault; this.fanout = fanout; this.sideInputs = ImmutableList.>of(); } private Globally(String name, GlobalCombineFn fn, - boolean insertDefault, int fanout, List> sideInputs) { + ClassForDisplay fnClass, boolean insertDefault, int fanout, + List> sideInputs) { super(name); this.fn = fn; + this.fnClass = fnClass; this.insertDefault = insertDefault; this.fanout = fanout; this.sideInputs = sideInputs; @@ -1264,7 +1297,7 @@ private Globally(String name, GlobalCombineFn fn, * specified name. Does not modify this transform. */ public Globally named(String name) { - return new Globally<>(name, fn, insertDefault, fanout); + return new Globally<>(name, fn, fnClass, insertDefault, fanout); } /** @@ -1275,7 +1308,7 @@ public Globally named(String name) { * to an empty input set will be returned. */ public GloballyAsSingletonView asSingletonView() { - return new GloballyAsSingletonView<>(fn, insertDefault, fanout); + return new GloballyAsSingletonView<>(fn, fnClass, insertDefault, fanout); } /** @@ -1284,7 +1317,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, false, fanout); + return new Globally<>(name, fn, fnClass, false, fanout); } /** @@ -1295,7 +1328,7 @@ public Globally withoutDefaults() { * that will be used. */ public Globally withFanout(int fanout) { - return new Globally<>(name, fn, insertDefault, fanout); + return new Globally<>(name, fn, fnClass, insertDefault, fanout); } /** @@ -1305,7 +1338,7 @@ public Globally withFanout(int fanout) { public Globally withSideInputs( Iterable> sideInputs) { Preconditions.checkState(fn instanceof RequiresContextInternal); - return new Globally(name, fn, insertDefault, fanout, + return new Globally(name, fn, fnClass, insertDefault, fanout, ImmutableList.>copyOf(sideInputs)); } @@ -1316,7 +1349,7 @@ public PCollection apply(PCollection input) { .setCoder(KvCoder.of(VoidCoder.of(), input.getCoder())); Combine.PerKey combine = - Combine.fewKeys(fn.asKeyedFn()); + Combine.fewKeys(fn.asKeyedFn(), fnClass); if (!sideInputs.isEmpty()) { combine = combine.withSideInputs(sideInputs); } @@ -1340,6 +1373,12 @@ public PCollection apply(PCollection input) { } } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + Combine.populateDisplayData(builder, fn, fnClass); + Combine.populateGlobalDisplayData(builder, fanout, insertDefault); + } + private PCollection insertDefaultValueIfEmpty(PCollection maybeEmpty) { final PCollectionView> maybeEmptyView = maybeEmpty.apply( View.asIterable()); @@ -1366,6 +1405,20 @@ public void processElement(DoFn.ProcessContext c) { } } + private static void populateDisplayData( + DisplayData.Builder builder, HasDisplayData fn, ClassForDisplay fnClass) { + builder + .include(fn, fnClass) + .add("combineFn", fnClass); + } + + private static void populateGlobalDisplayData( + DisplayData.Builder builder, int fanout, boolean insertDefault) { + builder + .addIfNotDefault("fanout", fanout, 0) + .add("emitDefaultOnEmptyInput", insertDefault); + } + /** * {@code Combine.GloballyAsSingletonView} takes a {@code PCollection} * and returns a {@code PCollectionView} whose elements are the result of @@ -1409,12 +1462,15 @@ public static class GloballyAsSingletonView extends PTransform, PCollectionView> { private final GlobalCombineFn fn; + private final ClassForDisplay fnClass; private final boolean insertDefault; private final int fanout; private GloballyAsSingletonView( - GlobalCombineFn fn, boolean insertDefault, int fanout) { + GlobalCombineFn fn, ClassForDisplay fnClass, + boolean insertDefault, int fanout) { this.fn = fn; + this.fnClass = fnClass; this.insertDefault = insertDefault; this.fanout = fanout; } @@ -1445,6 +1501,12 @@ public boolean getInsertDefault() { public GlobalCombineFn getCombineFn() { return fn; } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + Combine.populateDisplayData(builder, fn, fnClass); + Combine.populateGlobalDisplayData(builder, fanout, insertDefault); + } } /** @@ -1524,6 +1586,11 @@ public List compact(List accumulator) { return accumulator.size() > 1 ? mergeToSingleton(accumulator) : accumulator; } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.add("combineFn", combiner.getClass()); + } + private List mergeToSingleton(Iterable values) { List singleton = new ArrayList<>(); singleton.add(combiner.apply(values)); @@ -1597,30 +1664,35 @@ public static class PerKey extends PTransform>, PCollection>> { private final transient PerKeyCombineFn fn; + private final ClassForDisplay fnClass; private final boolean fewKeys; private final List> sideInputs; private PerKey( - PerKeyCombineFn fn, boolean fewKeys) { + PerKeyCombineFn fn, ClassForDisplay fnClass, + boolean fewKeys) { this.fn = fn; + this.fnClass = fnClass; this.fewKeys = fewKeys; this.sideInputs = ImmutableList.of(); } private PerKey(String name, - PerKeyCombineFn fn, + PerKeyCombineFn fn, ClassForDisplay fnClass, boolean fewKeys, List> sideInputs) { super(name); this.fn = fn; + this.fnClass = fnClass; this.fewKeys = fewKeys; this.sideInputs = sideInputs; } private PerKey( String name, PerKeyCombineFn fn, - boolean fewKeys) { + ClassForDisplay fnClass, boolean fewKeys) { super(name); this.fn = fn; + this.fnClass = fnClass; this.fewKeys = fewKeys; this.sideInputs = ImmutableList.of(); } @@ -1630,7 +1702,7 @@ private PerKey( * specified name. Does not modify this transform. */ public PerKey named(String name) { - return new PerKey(name, fn, fewKeys); + return new PerKey(name, fn, fnClass, fewKeys); } /** @@ -1640,7 +1712,7 @@ public PerKey named(String name) { public PerKey withSideInputs( Iterable> sideInputs) { Preconditions.checkState(fn instanceof RequiresContextInternal); - return new PerKey(name, fn, fewKeys, + return new PerKey(name, fn, fnClass, fewKeys, ImmutableList.>copyOf(sideInputs)); } @@ -1657,7 +1729,7 @@ public PerKey withSideInputs( */ public PerKeyWithHotKeyFanout withHotKeyFanout( SerializableFunction hotKeyFanout) { - return new PerKeyWithHotKeyFanout(name, fn, hotKeyFanout); + return new PerKeyWithHotKeyFanout(name, fn, fnClass, hotKeyFanout); } /** @@ -1665,7 +1737,7 @@ public PerKeyWithHotKeyFanout withHotKeyFanout( * constant value for every key. */ public PerKeyWithHotKeyFanout withHotKeyFanout(final int hotKeyFanout) { - return new PerKeyWithHotKeyFanout(name, fn, + return new PerKeyWithHotKeyFanout(name, fn, fnClass, new SerializableFunction(){ @Override public Integer apply(K unused) { @@ -1694,6 +1766,11 @@ public PCollection> apply(PCollection> input) { .apply(GroupByKey.create(fewKeys)) .apply(Combine.groupedValues(fn).withSideInputs(sideInputs)); } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + Combine.populateDisplayData(builder, fn, fnClass); + } } /** @@ -1703,13 +1780,16 @@ public static class PerKeyWithHotKeyFanout extends PTransform>, PCollection>> { private final transient PerKeyCombineFn fn; + private final ClassForDisplay fnClass; private final SerializableFunction hotKeyFanout; private PerKeyWithHotKeyFanout(String name, PerKeyCombineFn fn, + ClassForDisplay fnClass, SerializableFunction hotKeyFanout) { super(name); this.fn = fn; + this.fnClass = fnClass; this.hotKeyFanout = hotKeyFanout; } @@ -1992,6 +2072,12 @@ public void processElement(ProcessContext c) { .apply("PostCombine", Combine.perKey(postCombine)); } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + Combine.populateDisplayData(builder, fn, fnClass); + builder.add("fanoutFn", hotKeyFanout.getClass()); + } + /** * Used to store either an input or accumulator value, for flattening * the hot and cold key paths. @@ -2133,23 +2219,28 @@ public static class GroupedValues PCollection>> { private final PerKeyCombineFn fn; + private final ClassForDisplay fnClass; private final List> sideInputs; - private GroupedValues(PerKeyCombineFn fn) { + private GroupedValues( + PerKeyCombineFn fn, ClassForDisplay fnClass) { this.fn = SerializableUtils.clone(fn); + this.fnClass = fnClass; this.sideInputs = ImmutableList.>of(); } private GroupedValues( PerKeyCombineFn fn, + ClassForDisplay fnClass, List> sideInputs) { this.fn = SerializableUtils.clone(fn); + this.fnClass = fnClass; this.sideInputs = sideInputs; } public GroupedValues withSideInputs( Iterable> sideInputs) { - return new GroupedValues<>(fn, ImmutableList.>copyOf(sideInputs)); + return new GroupedValues<>(fn, fnClass, ImmutableList.>copyOf(sideInputs)); } /** @@ -2236,5 +2327,10 @@ public Coder> getDefaultOutputCoder( kvCoder.getKeyCoder(), kvCoder.getValueCoder()); return KvCoder.of(kvCoder.getKeyCoder(), outputValueCoder); } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + Combine.populateDisplayData(builder, fn, fnClass); + } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/CombineFnBase.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/CombineFnBase.java index a0b06cf1fc..d278858478 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/CombineFnBase.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/CombineFnBase.java @@ -22,6 +22,8 @@ import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn; import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.CombineFnWithContext; import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; +import com.google.cloud.dataflow.sdk.transforms.display.HasDisplayData; import com.google.cloud.dataflow.sdk.values.TypeDescriptor; import com.google.common.collect.ImmutableMap; @@ -49,7 +51,7 @@ public class CombineFnBase { * @param type of mutable accumulator values * @param type of output values */ - public interface GlobalCombineFn extends Serializable { + public interface GlobalCombineFn extends Serializable, HasDisplayData { /** * Returns the {@code Coder} to use for accumulator {@code AccumT} @@ -114,7 +116,8 @@ public Coder getDefaultOutputCoder(CoderRegistry registry, Coder type of mutable accumulator values * @param type of output values */ - public interface PerKeyCombineFn extends Serializable { + public interface PerKeyCombineFn + extends Serializable, HasDisplayData { /** * Returns the {@code Coder} to use for accumulator {@code AccumT} * values, or null if it is not able to be inferred. @@ -214,6 +217,16 @@ public TypeVariable getOutputTVariable() { return (TypeVariable) new TypeDescriptor(AbstractGlobalCombineFn.class) {}.getType(); } + + /** + * {@inheritDoc} + * + *

    By default, does not register any display data. Implementors may override this method + * to provide their own display metadata. + */ + @Override + public void populateDisplayData(DisplayData.Builder builder) { + } } /** @@ -279,5 +292,15 @@ public TypeVariable getOutputTVariable() { return (TypeVariable) new TypeDescriptor(AbstractPerKeyCombineFn.class) {}.getType(); } + + /** + * {@inheritDoc} + * + *

    By default, does not register any display data. Implementors may override this method + * to provide their own display metadata. + */ + @Override + public void populateDisplayData(DisplayData.Builder builder) { + } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/CombineFns.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/CombineFns.java index 656c010d91..120dbbf573 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/CombineFns.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/CombineFns.java @@ -29,12 +29,17 @@ import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.CombineFnWithContext; import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.Context; import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; +import com.google.cloud.dataflow.sdk.transforms.display.HasDisplayData; import com.google.cloud.dataflow.sdk.util.PropertyNames; import com.google.cloud.dataflow.sdk.values.TupleTag; +import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.collect.Multimap; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -43,6 +48,7 @@ import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; +import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -451,6 +457,11 @@ public Coder getAccumulatorCoder(CoderRegistry registry, Coder } return new ComposedAccumulatorCoder(coders); } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + CombineFns.populateDisplayData(builder, combineFns); + } } /** @@ -584,6 +595,11 @@ public Coder getAccumulatorCoder(CoderRegistry registry, Coder } return new ComposedAccumulatorCoder(coders); } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + CombineFns.populateDisplayData(builder, combineFnWithContexts); + } } /** @@ -765,6 +781,11 @@ public Coder getAccumulatorCoder( } return new ComposedAccumulatorCoder(coders); } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + CombineFns.populateDisplayData(builder, keyedCombineFns); + } } /** @@ -911,6 +932,11 @@ public Coder getAccumulatorCoder( } return new ComposedAccumulatorCoder(coders); } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + CombineFns.populateDisplayData(builder, keyedCombineFns); + } } ///////////////////////////////////////////////////////////////////////////// @@ -1097,4 +1123,43 @@ private static void checkUniqueness( "Cannot compose with tuple tag %s because it is already present in the composition.", outputTag); } + + /** + * Populate display data for the {@code combineFns} that make up a composed combine transform. + * + *

    The same combineFn class may be used multiple times, in which case we must take special care + * to register display data with unique namespaces. + */ + private static void populateDisplayData( + DisplayData.Builder builder, List combineFns) { + + // NB: ArrayListMultimap necessary to maintain ordering of combineFns of the same type. + Multimap, HasDisplayData> combineFnMap = ArrayListMultimap.create(); + + for (int i = 0; i < combineFns.size(); i++) { + HasDisplayData combineFn = combineFns.get(i); + builder.add("combineFn" + (i + 1), combineFn.getClass()); + combineFnMap.put(combineFn.getClass(), combineFn); + } + + for (Map.Entry, Collection> combineFnEntries : + combineFnMap.asMap().entrySet()) { + + Collection classCombineFns = combineFnEntries.getValue(); + if (classCombineFns.size() == 1) { + // Only one combineFn of this type, include it directly. + builder.include(Iterables.getOnlyElement(classCombineFns)); + + } else { + // Multiple combineFns of same type, add a namespace suffix so display data is + // unique and ordered. + String baseNamespace = combineFnEntries.getKey().getName(); + for (int i = 0; i < combineFns.size(); i++) { + HasDisplayData combineFn = combineFns.get(i); + String namespace = String.format("%s#%d", baseNamespace, i + 1); + builder.include(combineFn, namespace); + } + } + } + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/CombineWithContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/CombineWithContext.java index fdf56e33c0..b26636ee20 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/CombineWithContext.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/CombineWithContext.java @@ -21,6 +21,7 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn; import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.values.PCollectionView; /** @@ -165,6 +166,11 @@ public Coder getDefaultOutputCoder(CoderRegistry registry, Coder key public CombineFnWithContext forKey(K key, Coder keyCoder) { return CombineFnWithContext.this; } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + CombineFnWithContext.this.populateDisplayData(builder); + } }; } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnReflector.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnReflector.java index 1bb05fb340..678eb1d1be 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnReflector.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnReflector.java @@ -21,6 +21,7 @@ import com.google.cloud.dataflow.sdk.transforms.DoFnWithContext.FinishBundle; import com.google.cloud.dataflow.sdk.transforms.DoFnWithContext.ProcessElement; import com.google.cloud.dataflow.sdk.transforms.DoFnWithContext.StartBundle; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo; import com.google.cloud.dataflow.sdk.util.UserCodeException; @@ -650,6 +651,11 @@ protected TypeDescriptor getOutputTypeDescriptor() { return fn.getOutputTypeDescriptor(); } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + fn.populateDisplayData(builder); + } + private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException { in.defaultReadObject(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnWithContext.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnWithContext.java index 4f131adc91..452de2d10e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnWithContext.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/DoFnWithContext.java @@ -24,6 +24,8 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn; import com.google.cloud.dataflow.sdk.transforms.DoFn.DelegatingAggregator; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; +import com.google.cloud.dataflow.sdk.transforms.display.HasDisplayData; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo; import com.google.cloud.dataflow.sdk.util.WindowingInternals; @@ -81,7 +83,7 @@ * @param the type of the (main) output elements */ @Experimental -public abstract class DoFnWithContext implements Serializable { +public abstract class DoFnWithContext implements Serializable, HasDisplayData { /** Information accessible to all methods in this {@code DoFnWithContext}. */ public abstract class Context { @@ -413,4 +415,14 @@ public final Aggregator createAggregator( void prepareForProcessing() { aggregatorsAreFinal = true; } + + /** + * {@inheritDoc} + * + *

    By default, does not register any display data. Implementors may override this method + * to provide their own display metadata. + */ + @Override + public void populateDisplayData(DisplayData.Builder builder) { + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Filter.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Filter.java index 9e123a19fc..f0860d841f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Filter.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Filter.java @@ -17,6 +17,7 @@ package com.google.cloud.dataflow.sdk.transforms; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.values.PCollection; /** @@ -98,9 +99,15 @@ public void processElement(ProcessContext c) { c.output(c.element()); } } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + Filter.populateDisplayData(builder, String.format("x < %s", value)); + } }); } + /** * Returns a {@code PTransform} that takes an input * {@code PCollection} and returns a {@code PCollection} with @@ -130,6 +137,11 @@ public void processElement(ProcessContext c) { c.output(c.element()); } } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + Filter.populateDisplayData(builder, String.format("x > %s", value)); + } }); } @@ -162,6 +174,11 @@ public void processElement(ProcessContext c) { c.output(c.element()); } } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + Filter.populateDisplayData(builder, String.format("x ≤ %s", value)); + } }); } @@ -194,6 +211,11 @@ public void processElement(ProcessContext c) { c.output(c.element()); } } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + Filter.populateDisplayData(builder, String.format("x ≥ %s", value)); + } }); } @@ -231,4 +253,9 @@ public void processElement(ProcessContext c) { protected Coder getDefaultOutputCoder(PCollection input) { return input.getCoder(); } + + private static void populateDisplayData( + DisplayData.Builder builder, String predicateDescription) { + builder.add("predicate", predicateDescription); + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java index 8fde3e0869..1125ea5e50 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/GroupByKey.java @@ -25,6 +25,7 @@ import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner.ValueWithMetadata; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.DefaultTrigger; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; @@ -315,6 +316,13 @@ static KvCoder> getOutputKvCoder(Coder> inputCode return KvCoder.of(getKeyCoder(inputCoder), getOutputValueCoder(inputCoder)); } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + if (fewKeys) { + builder.add("fewKeys", true); + } + } + ///////////////////////////////////////////////////////////////////////////// /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/IntraBundleParallelization.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/IntraBundleParallelization.java index b6497b71c4..335cc6e412 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/IntraBundleParallelization.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/IntraBundleParallelization.java @@ -19,6 +19,7 @@ import com.google.cloud.dataflow.sdk.options.GcsOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo; import com.google.cloud.dataflow.sdk.util.WindowingInternals; @@ -170,6 +171,14 @@ public PCollection apply(PCollection input) { return input.apply( ParDo.of(new MultiThreadedIntraBundleProcessingDoFn<>(doFn, maxParallelism))); } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder + .add("maxParallelism", maxParallelism) + .add("fn", doFn.getClass()) + .include(doFn); + } } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Max.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Max.java index 8678e4f33e..70e541c42e 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Max.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Max.java @@ -17,6 +17,7 @@ package com.google.cloud.dataflow.sdk.transforms; import com.google.cloud.dataflow.sdk.transforms.Combine.BinaryCombineFn; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.util.common.Counter; import com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind; import com.google.cloud.dataflow.sdk.util.common.CounterProvider; @@ -203,6 +204,11 @@ public T identity() { public T apply(T left, T right) { return comparator.compare(left, right) >= 0 ? left : right; } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.add("comparer", comparator.getClass()); + } } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Min.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Min.java index 47ab3a0ad2..5e9d7041c8 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Min.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Min.java @@ -17,6 +17,7 @@ package com.google.cloud.dataflow.sdk.transforms; import com.google.cloud.dataflow.sdk.transforms.Combine.BinaryCombineFn; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.util.common.Counter; import com.google.cloud.dataflow.sdk.util.common.Counter.AggregationKind; import com.google.cloud.dataflow.sdk.util.common.CounterProvider; @@ -203,6 +204,11 @@ public T identity() { public T apply(T left, T right) { return comparator.compare(left, right) <= 0 ? left : right; } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.add("comparer", comparator.getClass()); + } } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java index c77ac4447c..2ab61ba5ee 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/ParDo.java @@ -22,6 +22,7 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.CoderException; import com.google.cloud.dataflow.sdk.runners.DirectPipelineRunner; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.transforms.display.DisplayData.Builder; import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn; import com.google.cloud.dataflow.sdk.util.DirectModeExecutionContext; @@ -554,7 +555,12 @@ public static UnboundMulti withOutputTags( * properties can be set on it first. */ public static Bound of(DoFn fn) { - return new Unbound().of(fn); + return of(fn, fn.getClass()); + } + + private static Bound of( + DoFn fn, Class fnClass) { + return new Unbound().of(fn, fnClass); } private static DoFn @@ -577,7 +583,7 @@ public static Bound of(DoFn */ @Experimental public static Bound of(DoFnWithContext fn) { - return of(adapt(fn)); + return of(adapt(fn), fn.getClass()); } /** @@ -664,9 +670,15 @@ public UnboundMulti withOutputTags(TupleTag mainOutp * still be specified. */ public Bound of(DoFn fn) { - return new Bound<>(name, sideInputs, fn); + return of(fn, fn.getClass()); + } + + private Bound of( + DoFn fn, Class fnClass) { + return new Bound<>(name, sideInputs, fn, fnClass); } + /** * Returns a new {@link ParDo} {@link PTransform} that's like this * transform but which will invoke the given {@link DoFnWithContext} @@ -676,7 +688,7 @@ public Bound of(DoFn fn) { * still be specified. */ public Bound of(DoFnWithContext fn) { - return of(adapt(fn)); + return of(adapt(fn), fn.getClass()); } } @@ -697,13 +709,16 @@ public static class Bound // Inherits name. private final List> sideInputs; private final DoFn fn; + private final Class fnClass; Bound(String name, List> sideInputs, - DoFn fn) { + DoFn fn, + Class fnClass) { super(name); this.sideInputs = sideInputs; this.fn = SerializableUtils.clone(fn); + this.fnClass = fnClass; } /** @@ -714,7 +729,7 @@ public static class Bound *

    See the discussion of Naming above for more explanation. */ public Bound named(String name) { - return new Bound<>(name, sideInputs, fn); + return new Bound<>(name, sideInputs, fn, fnClass); } /** @@ -742,7 +757,7 @@ public Bound withSideInputs( ImmutableList.Builder> builder = ImmutableList.builder(); builder.addAll(this.sideInputs); builder.addAll(sideInputs); - return new Bound<>(name, builder.build(), fn); + return new Bound<>(name, builder.build(), fn, fnClass); } /** @@ -756,7 +771,7 @@ public Bound withSideInputs( public BoundMulti withOutputTags(TupleTag mainOutputTag, TupleTagList sideOutputTags) { return new BoundMulti<>( - name, sideInputs, mainOutputTag, sideOutputTags, fn); + name, sideInputs, mainOutputTag, sideOutputTags, fn, fnClass); } @Override @@ -797,7 +812,7 @@ protected String getKindString() { */ @Override public void populateDisplayData(Builder builder) { - builder.include(fn); + ParDo.populateDisplayData(builder, fn, fnClass); } public DoFn getFn() { @@ -889,8 +904,12 @@ public UnboundMulti withSideInputs( * more properties can still be specified. */ public BoundMulti of(DoFn fn) { + return of(fn, fn.getClass()); + } + + public BoundMulti of(DoFn fn, Class fnClass) { return new BoundMulti<>( - name, sideInputs, mainOutputTag, sideOutputTags, fn); + name, sideInputs, mainOutputTag, sideOutputTags, fn, fnClass); } /** @@ -902,7 +921,7 @@ public BoundMulti of(DoFn fn) { * more properties can still be specified. */ public BoundMulti of(DoFnWithContext fn) { - return of(adapt(fn)); + return of(adapt(fn), fn.getClass()); } } @@ -924,17 +943,20 @@ public static class BoundMulti private final TupleTag mainOutputTag; private final TupleTagList sideOutputTags; private final DoFn fn; + private final Class fnClass; BoundMulti(String name, List> sideInputs, TupleTag mainOutputTag, TupleTagList sideOutputTags, - DoFn fn) { + DoFn fn, + Class fnClass) { super(name); this.sideInputs = sideInputs; this.mainOutputTag = mainOutputTag; this.sideOutputTags = sideOutputTags; this.fn = SerializableUtils.clone(fn); + this.fnClass = fnClass; } /** @@ -946,7 +968,7 @@ public static class BoundMulti */ public BoundMulti named(String name) { return new BoundMulti<>( - name, sideInputs, mainOutputTag, sideOutputTags, fn); + name, sideInputs, mainOutputTag, sideOutputTags, fn, fnClass); } /** @@ -977,7 +999,7 @@ public BoundMulti withSideInputs( builder.addAll(sideInputs); return new BoundMulti<>( name, builder.build(), - mainOutputTag, sideOutputTags, fn); + mainOutputTag, sideOutputTags, fn, fnClass); } @@ -1025,6 +1047,11 @@ protected String getKindString() { } } + @Override + public void populateDisplayData(Builder builder) { + ParDo.populateDisplayData(builder, fn, fnClass); + } + public DoFn getFn() { return fn; } @@ -1231,6 +1258,13 @@ private static SideInputReader makeSideInputReader( return DirectSideInputReader.of(sideInputValues); } + private static void populateDisplayData( + DisplayData.Builder builder, DoFn fn, Class fnClass) { + builder + .include(fn, fnClass) + .add("fn", fnClass); + } + /** * A {@code DoFnRunner.OutputManager} that provides facilities for checking output values for * illegal mutations. diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Partition.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Partition.java index bbbccbc75d..623875caa8 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Partition.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Partition.java @@ -17,6 +17,7 @@ package com.google.cloud.dataflow.sdk.transforms; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PCollectionList; import com.google.cloud.dataflow.sdk.values.PCollectionTuple; @@ -120,6 +121,11 @@ public PCollectionList apply(PCollection in) { return pcs; } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.include(partitionDoFn); + } + private final transient PartitionDoFn partitionDoFn; private Partition(PartitionDoFn partitionDoFn) { @@ -169,5 +175,12 @@ public void processElement(ProcessContext c) { partition + " not in [0.." + numPartitions + ")"); } } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder + .add("numPartitions", numPartitions) + .add("partitionFn", partitionFn.getClass()); + } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sample.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sample.java index c5b6e7ec14..a72443ce49 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sample.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sample.java @@ -23,6 +23,7 @@ import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.coders.VoidCoder; import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PCollectionView; @@ -151,6 +152,11 @@ public PCollection apply(PCollection in) { .of(new SampleAnyDoFn<>(limit, iterableView))) .setCoder(in.getCoder()); } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.add("sampleSize", limit); + } } /** @@ -186,6 +192,7 @@ public static class FixedSizedSampleFn extends CombineFn, SerializableComparator>>, Iterable> { + private final int sampleSize; private final Top.TopCombineFn, SerializableComparator>> topCombineFn; private final Random rand = new Random(); @@ -194,6 +201,8 @@ private FixedSizedSampleFn(int sampleSize) { if (sampleSize < 0) { throw new IllegalArgumentException("sample size must be >= 0"); } + + this.sampleSize = sampleSize; topCombineFn = new Top.TopCombineFn, SerializableComparator>>( sampleSize, new KV.OrderByKey()); } @@ -242,5 +251,10 @@ public Coder> getDefaultOutputCoder( CoderRegistry registry, Coder inputCoder) { return IterableCoder.of(inputCoder); } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.add("sampleSize", sampleSize); + } } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java index 98fe53c0a8..e46da47539 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/Top.java @@ -24,6 +24,7 @@ import com.google.cloud.dataflow.sdk.transforms.Combine.AccumulatingCombineFn; import com.google.cloud.dataflow.sdk.transforms.Combine.AccumulatingCombineFn.Accumulator; import com.google.cloud.dataflow.sdk.transforms.Combine.PerKey; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; @@ -390,6 +391,13 @@ public Coder> getAccumulatorCoder( return new BoundedHeapCoder<>(count, compareFn, inputCoder); } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder + .add("count", count) + .add("comparer", compareFn.getClass()); + } + @Override public String getIncompatibleGlobalWindowErrorMessage() { return "Default values are not supported in Top.[of, smallest, largest]() if the output " diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/ClassForDisplay.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/ClassForDisplay.java new file mode 100644 index 0000000000..7316c2a272 --- /dev/null +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/ClassForDisplay.java @@ -0,0 +1,91 @@ +/* + * Copyright (C) 2016 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.display; + +import static com.google.common.base.Preconditions.checkNotNull; + +import java.io.Serializable; +import java.util.Objects; + +/** + * Display metadata representing a Java class. + *

    + *

    Java classes can be registered as display metadata via + * {@link DisplayData.Builder#add(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) { + return new ClassForDisplay(checkNotNull(clazz)); + } + + /** + * Create a {@link ClassForDisplay} from the class of the specified object instance. + */ + public static ClassForDisplay fromInstance(Object obj) { + checkNotNull(obj); + 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/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayData.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayData.java index dadc7309da..eefbfb6078 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayData.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayData.java @@ -28,6 +28,7 @@ import com.fasterxml.jackson.annotation.JsonGetter; import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonValue; import org.apache.avro.reflect.Nullable; import org.joda.time.Duration; @@ -75,6 +76,31 @@ public static DisplayData from(HasDisplayData component) { return InternalBuilder.forRoot(component).build(); } + /** + * Infer the {@link Type} for the given object. + * + *

    Use this method if the type of metadata is not known at compile time. For example: + * + *

    +   * {@code
    +   * @Override
    +   * public void populateDisplayData(DisplayData.Builder builder) {
    +   *   Optional type = DisplayData.inferType(foo);
    +   *   if (type.isPresent()) {
    +   *     builder.add("foo", type.get(), foo);
    +   *   }
    +   * }
    +   * }
    +   * 
    + * + * @return The inferred {@link Type}, or null if the type cannot be inferred, + */ + @Nullable + public static Type inferType(@Nullable Object value) { + return Type.tryInferFrom(value); + } + + @JsonValue public Collection items() { return entries.values(); } @@ -83,11 +109,26 @@ public Map asMap() { return entries; } + @Override + public int hashCode() { + return entries.hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof DisplayData) { + DisplayData that = (DisplayData) obj; + return Objects.equals(this.entries, that.entries); + } + + return false; + } + @Override public String toString() { StringBuilder builder = new StringBuilder(); boolean isFirstLine = true; - for (Map.Entry entry : entries.entrySet()) { + for (Item entry : entries.values()) { if (isFirstLine) { isFirstLine = false; } else { @@ -100,19 +141,44 @@ public String toString() { return builder.toString(); } + private static String namespaceOf(ClassForDisplay clazz) { + return clazz.getName(); + } + /** * Utility to build up display metadata from a component and its included * subcomponents. */ public interface Builder { /** - * Include display metadata from the specified subcomponent. For example, a {@link ParDo} - * transform includes display metadata from the encapsulated {@link DoFn}. + * Register display metadata from the specified subcomponent. * - * @return A builder instance to continue to build in a fluent-style. + * @see #include(HasDisplayData, String) */ Builder include(HasDisplayData subComponent); + /** + * Register display metadata from the specified subcomponent, using the specified namespace. + * + * @see #include(HasDisplayData, String) + */ + Builder include(HasDisplayData subComponent, Class namespace); + + /** + * Register display metadata from the specified subcomponent, using the specified namespace. + * + * @see #include(HasDisplayData, String) + */ + Builder include(HasDisplayData subComponent, ClassForDisplay namespace); + + /** + * 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, String 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 @@ -120,6 +186,20 @@ public interface Builder { */ ItemBuilder add(String key, String value); + /** + * Register the given string display data if the value is not null. + * + * @see DisplayData.Builder#add(String, String) + */ + ItemBuilder addIfNotNull(String key, @Nullable String value); + + /** + * Register the given string display data if the value is different than the specified default. + * + * @see DisplayData.Builder#add(String, String) + */ + ItemBuilder addIfNotDefault(String key, @Nullable String value, @Nullable String defaultValue); + /** * Register the given numeric display metadata. The metadata item will be registered with type * {@link DisplayData.Type#INTEGER}, and is identified by the specified key and namespace from @@ -127,6 +207,20 @@ public interface Builder { */ ItemBuilder add(String key, long value); + /** + * Register the given numeric display data if the value is not null. + * + * @see DisplayData.Builder#add(String, long) + */ + ItemBuilder addIfNotNull(String key, @Nullable Long value); + + /** + * Register the given numeric display data if the value is different than the specified default. + * + * @see DisplayData.Builder#add(String, long) + */ + ItemBuilder addIfNotDefault(String key, long value, long defaultValue); + /** * Register the given floating point display metadata. The metadata item will be registered with * type {@link DisplayData.Type#FLOAT}, and is identified by the specified key and namespace @@ -134,6 +228,42 @@ public interface Builder { */ ItemBuilder add(String key, double value); + /** + * Register the given floating point display data if the value is not null. + * + * @see DisplayData.Builder#add(String, double) + */ + ItemBuilder addIfNotNull(String key, @Nullable Double value); + + /** + * Register the given floating point display data if the value is different than the specified + * default. + * + * @see DisplayData.Builder#add(String, double) + */ + ItemBuilder addIfNotDefault(String key, double value, double defaultValue); + + /** + * Register the given boolean 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 boolean display data if the value is not null. + * + * @see DisplayData.Builder#add(String, boolean) + */ + ItemBuilder addIfNotNull(String key, @Nullable Boolean value); + + /** + * Register the given boolean display data if the value is different than the specified default. + * + * @see DisplayData.Builder#add(String, boolean) + */ + ItemBuilder addIfNotDefault(String key, boolean value, boolean defaultValue); + /** * 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 @@ -141,6 +271,22 @@ public interface Builder { */ ItemBuilder add(String key, Instant value); + /** + * Register the given timestamp display data if the value is not null. + * + * @see DisplayData.Builder#add(String, Instant) + */ + ItemBuilder addIfNotNull(String key, @Nullable Instant value); + + /** + * Register the given timestamp display data if the value is different than the specified + * default. + * + * @see DisplayData.Builder#add(String, Instant) + */ + ItemBuilder addIfNotDefault( + String key, @Nullable Instant value, @Nullable Instant defaultValue); + /** * Register the given duration display metadata. The metadata item will be registered with type * {@link DisplayData.Type#DURATION}, and is identified by the specified key and namespace from @@ -148,12 +294,76 @@ public interface Builder { */ ItemBuilder add(String key, Duration value); + /** + * Register the given duration display data if the value is not null. + * + * @see DisplayData.Builder#add(String, Duration) + */ + ItemBuilder addIfNotNull(String key, @Nullable Duration value); + + /** + * Register the given duration display data if the value is different than the specified + * default. + * + * @see DisplayData.Builder#add(String, Duration) + */ + ItemBuilder addIfNotDefault( + String key, @Nullable Duration value, @Nullable Duration defaultValue); + /** * Register the given class display metadata. The metadata item will be registered with type * {@link DisplayData.Type#JAVA_CLASS}, and is identified by the specified key and namespace * from the current transform or component. */ ItemBuilder add(String key, Class value); + + /** + * Register the given class display metadata. The metadata item will be registered with type + * {@link DisplayData.Type#JAVA_CLASS}, and is identified by the specified key and namespace + * from the current transform or component. + */ + ItemBuilder add(String key, ClassForDisplay value); + + /** + * Register the given class display data if the value is not null. + * + * @see DisplayData.Builder#add(String, Class) + */ + ItemBuilder addIfNotNull(String key, @Nullable Class value); + + /** + * Register the given class display data if the value is not null. + * + * @see DisplayData.Builder#add(String, ClassForDisplay) + */ + ItemBuilder addIfNotNull(String key, @Nullable ClassForDisplay value); + + /** + * Register the given class display data if the value is different than the specified default. + * + * @see DisplayData.Builder#add(String, Class) + */ + ItemBuilder addIfNotDefault( + String key, @Nullable Class value, @Nullable Class defaultValue); + + /** + * Register the given class display data if the value is different than the specified default. + * + * @see DisplayData.Builder#add(String, ClassForDisplay) + */ + ItemBuilder addIfNotDefault( + String key, @Nullable ClassForDisplay value, @Nullable ClassForDisplay defaultValue); + /** + * Register the given display metadata with the specified type. + * + *

    The added display data is identified by the specified key and namespace from the current + * transform or component. + * + * @throws ClassCastException if the value cannot be safely cast to the specified type. + * + * @see DisplayData#inferType(Object) + */ + ItemBuilder add(String key, Type type, Object value); } /** @@ -177,6 +387,22 @@ public interface ItemBuilder extends Builder { *

    Specifying a null value will clear the URL if it was previously defined. */ ItemBuilder withLinkUrl(@Nullable String url); + + /** + * Adds an explicit namespace to the most-recently added display metadata. The namespace + * and key uniquely identify the display metadata. + * + *

    Leaving the namespace unspecified will default to the registering instance's class. + */ + ItemBuilder withNamespace(Class namespace); + + /** + * Adds an explicit namespace to the most-recently added display metadata. The namespace + * and key uniquely identify the display metadata. + * + *

    Leaving the namespace unspecified will default to the registering instance's class. + */ + ItemBuilder withNamespace(ClassForDisplay namespace); } /** @@ -189,23 +415,23 @@ public static class Item { private final String key; private final String ns; private final Type type; - private final String value; - private final String shortValue; + private final Object value; + private final Object shortValue; private final String label; private final String url; - private static Item create(String namespace, String key, Type type, T value) { + private static Item create(String nsClass, String key, Type type, Object value) { FormattedItemValue formatted = type.format(value); return new Item( - namespace, key, type, formatted.getLongValue(), formatted.getShortValue(), null, null); + nsClass, key, type, formatted.getLongValue(), formatted.getShortValue(), null, null); } private Item( String namespace, String key, Type type, - String value, - String shortValue, + Object value, + Object shortValue, String url, String label) { this.ns = namespace; @@ -240,7 +466,7 @@ public Type getType() { * Retrieve the value of the metadata item. */ @JsonGetter("value") - public String getValue() { + public Object getValue() { return value; } @@ -254,7 +480,7 @@ public String getValue() { @JsonGetter("shortValue") @JsonInclude(JsonInclude.Include.NON_NULL) @Nullable - public String getShortValue() { + public Object getShortValue() { return shortValue; } @@ -286,7 +512,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) { @@ -296,6 +550,12 @@ private Item withLabel(String label) { private Item withUrl(String url) { return new Item(this.ns, this.key, this.type, this.value, this.shortValue, url, this.label); } + + private Item withNamespace(ClassForDisplay nsClass) { + String namespace = namespaceOf(nsClass); + return new Item( + namespace, this.key, this.type, this.value, this.shortValue, this.url, this.label); + } } /** @@ -312,8 +572,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(ClassForDisplay namespace, String key) { + return of(namespaceOf(namespace), key); + } + + public static Identifier of(String namespace, String key) { + return new Identifier(namespace, key); } private Identifier(String ns, String key) { @@ -354,45 +618,74 @@ public String toString() { /** * Display metadata type. */ - enum Type { + public enum Type { STRING { @Override FormattedItemValue format(Object value) { - return new FormattedItemValue((String) value); + return new FormattedItemValue(checkType(value, String.class, STRING)); } }, INTEGER { @Override FormattedItemValue format(Object value) { - return new FormattedItemValue(Long.toString((long) value)); + if (value instanceof Integer) { + long l = ((Integer) value).longValue(); + return format(l); + } + + return new FormattedItemValue(checkType(value, Long.class, INTEGER)); } }, FLOAT { @Override FormattedItemValue format(Object value) { - return new FormattedItemValue(Double.toString((Double) value)); + return new FormattedItemValue(checkType(value, Number.class, FLOAT)); + } + }, + BOOLEAN() { + @Override + FormattedItemValue format(Object value) { + return new FormattedItemValue(checkType(value, Boolean.class, BOOLEAN)); } }, TIMESTAMP() { @Override FormattedItemValue format(Object value) { - return new FormattedItemValue((TIMESTAMP_FORMATTER.print((Instant) value))); + Instant instant = checkType(value, Instant.class, TIMESTAMP); + return new FormattedItemValue((TIMESTAMP_FORMATTER.print(instant))); } }, DURATION { @Override FormattedItemValue format(Object value) { - return new FormattedItemValue(Long.toString(((Duration) value).getMillis())); + Duration duration = checkType(value, Duration.class, DURATION); + return new FormattedItemValue(duration.getMillis()); } }, JAVA_CLASS { @Override FormattedItemValue format(Object value) { - Class clazz = (Class) value; + if (value instanceof Class) { + ClassForDisplay classForDisplay = ClassForDisplay.of((Class) value); + return format(classForDisplay); + } + + ClassForDisplay clazz = checkType(value, ClassForDisplay.class, JAVA_CLASS); return new FormattedItemValue(clazz.getName(), clazz.getSimpleName()); } }; + private static T checkType(Object value, Class clazz, DisplayData.Type expectedType) { + if (!clazz.isAssignableFrom(value.getClass())) { + throw new ClassCastException(String.format( + "Value is not valid for DisplayData type %s: %s", expectedType, value)); + } + + @SuppressWarnings("unchecked") // type checked above. + T typedValue = (T) value; + return typedValue; + } + /** * Format the display metadata value into a long string representation, and optionally * a shorter representation for display. @@ -400,26 +693,47 @@ FormattedItemValue format(Object value) { *

    Internal-only. Value objects can be safely cast to the expected Java type. */ abstract FormattedItemValue format(Object value); + + @Nullable + private static Type tryInferFrom(@Nullable Object value) { + if (value instanceof Integer || value instanceof Long) { + return INTEGER; + } else if (value instanceof Double || value instanceof Float) { + return FLOAT; + } else if (value instanceof Boolean) { + return BOOLEAN; + } else if (value instanceof Instant) { + return TIMESTAMP; + } else if (value instanceof Duration) { + return DURATION; + } else if (value instanceof Class || value instanceof ClassForDisplay) { + return JAVA_CLASS; + } else if (value instanceof String) { + return STRING; + } else { + return null; + } + } } - private static class FormattedItemValue { - private final String shortValue; - private final String longValue; + static class FormattedItemValue { + private final Object shortValue; + private final Object longValue; - private FormattedItemValue(String longValue) { + private FormattedItemValue(Object longValue) { this(longValue, null); } - private FormattedItemValue(String longValue, String shortValue) { + private FormattedItemValue(Object longValue, Object shortValue) { this.longValue = longValue; this.shortValue = shortValue; } - private String getLongValue () { + Object getLongValue() { return this.longValue; } - private String getShortValue() { + Object getShortValue() { return this.shortValue; } } @@ -428,9 +742,10 @@ private static class InternalBuilder implements ItemBuilder { private final Map entries; private final Set visited; - private Class latestNs; + private String latestNs; + + @Nullable private Item latestItem; - private Identifier latestIdentifier; private InternalBuilder() { this.entries = Maps.newHashMap(); @@ -446,10 +761,31 @@ 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(namespace); + return include(subComponent, ClassForDisplay.of(namespace)); + } + + @Override + public Builder include(HasDisplayData subComponent, ClassForDisplay namespace) { + checkNotNull(namespace); + return include(subComponent, namespaceOf(namespace)); + } + + @Override + public Builder include(HasDisplayData subComponent, String namespace) { + checkNotNull(subComponent); + checkNotNull(namespace); + + commitLatest(); boolean newComponent = visited.add(subComponent); if (newComponent) { - Class prevNs = this.latestNs; - this.latestNs = subComponent.getClass(); + String prevNs = this.latestNs; + this.latestNs = namespace; subComponent.populateDisplayData(this); this.latestNs = prevNs; } @@ -460,70 +796,199 @@ public Builder include(HasDisplayData subComponent) { @Override public ItemBuilder add(String key, String value) { checkNotNull(value); - return addItem(key, Type.STRING, value); + return addItemIf(true, key, Type.STRING, value); + } + + @Override + public ItemBuilder addIfNotNull(String key, @Nullable String value) { + return addItemIf(value != null, key, Type.STRING, value); + } + + @Override + public ItemBuilder addIfNotDefault( + String key, @Nullable String value, @Nullable String defaultValue) { + return addItemIf(!Objects.equals(value, defaultValue), key, Type.STRING, value); } @Override public ItemBuilder add(String key, long value) { - return addItem(key, Type.INTEGER, value); + return addItemIf(true, key, Type.INTEGER, value); + } + + @Override + public ItemBuilder addIfNotNull(String key, @Nullable Long value) { + return addItemIf(value != null, key, Type.INTEGER, value); + } + + @Override + public ItemBuilder addIfNotDefault(String key, long value, long defaultValue) { + return addItemIf(!Objects.equals(value, defaultValue), key, Type.INTEGER, value); } @Override public ItemBuilder add(String key, double value) { - return addItem(key, Type.FLOAT, value); + return addItemIf(true, key, Type.FLOAT, value); + } + + @Override + public ItemBuilder addIfNotNull(String key, @Nullable Double value) { + return addItemIf(value != null, key, Type.FLOAT, value); + } + + @Override + public ItemBuilder addIfNotDefault(String key, double value, double defaultValue) { + return addItemIf(!Objects.equals(value, defaultValue), key, Type.FLOAT, value); + } + + @Override + public ItemBuilder add(String key, boolean value) { + return addItemIf(true, key, Type.BOOLEAN, value); + } + + @Override + public ItemBuilder addIfNotNull(String key, @Nullable Boolean value) { + return addItemIf(value != null, key, Type.BOOLEAN, value); + } + + @Override + public ItemBuilder addIfNotDefault(String key, boolean value, boolean defaultValue) { + return addItemIf(!Objects.equals(value, defaultValue), key, Type.BOOLEAN, value); } @Override public ItemBuilder add(String key, Instant value) { - checkNotNull(value); - return addItem(key, Type.TIMESTAMP, value); + return addItemIf(true, key, Type.TIMESTAMP, value); + } + + @Override + public ItemBuilder addIfNotNull(String key, @Nullable Instant value) { + return addItemIf(value != null, key, Type.TIMESTAMP, value); + } + + @Override + public ItemBuilder addIfNotDefault( + String key, @Nullable Instant value, @Nullable Instant defaultValue) { + return addItemIf(!Objects.equals(value, defaultValue), key, Type.TIMESTAMP, value); } @Override public ItemBuilder add(String key, Duration value) { - checkNotNull(value); - return addItem(key, Type.DURATION, value); + return addItemIf(true, key, Type.DURATION, value); + } + + @Override + public ItemBuilder addIfNotNull(String key, @Nullable Duration value) { + return addItemIf(value != null, key, Type.DURATION, value); + } + + @Override + public ItemBuilder addIfNotDefault( + String key, @Nullable Duration value, @Nullable Duration defaultValue) { + return addItemIf(!Objects.equals(value, defaultValue), key, Type.DURATION, value); } @Override public ItemBuilder add(String key, Class value) { + return addItemIf(true, key, Type.JAVA_CLASS, value); + } + + @Override + public ItemBuilder add(String key, ClassForDisplay value) { checkNotNull(value); - return addItem(key, Type.JAVA_CLASS, value); + return addItemIf(true, key, Type.JAVA_CLASS, value); } - private ItemBuilder addItem(String key, Type type, T value) { - checkNotNull(key); - checkArgument(!key.isEmpty()); + @Override + public ItemBuilder addIfNotNull(String key, @Nullable Class value) { + return addItemIf(value != null, key, Type.JAVA_CLASS, value); + } - Identifier id = Identifier.of(latestNs, key); + @Override + public ItemBuilder addIfNotNull(String key, @Nullable ClassForDisplay value) { + return addItemIf(value != null, key, Type.JAVA_CLASS, value); + } + + @Override + public ItemBuilder addIfNotDefault( + String key, @Nullable Class value, @Nullable Class defaultValue) { + return addItemIf(!Objects.equals(value, defaultValue), key, Type.JAVA_CLASS, value); + } + + @Override + public ItemBuilder addIfNotDefault( + String key, @Nullable ClassForDisplay value, @Nullable ClassForDisplay defaultValue) { + return addItemIf(!Objects.equals(value, defaultValue), key, Type.JAVA_CLASS, value); + } + + @Override + public ItemBuilder add(String key, Type type, Object value) { + checkNotNull(type); + return addItemIf(true, key, type, value); + } + + private ItemBuilder addItemIf(boolean condition, String key, Type type, Object value) { + checkNotNull(key, "Display data keys cannot be null or empty."); + checkArgument(!key.isEmpty(), "Display data keys cannot be null or empty."); + commitLatest(); + + if (condition) { + checkNotNull(value, "Display data values cannot be null. Key: [%s]", key); + latestItem = Item.create(latestNs, key, type, value); + } + + return this; + } + + private void commitLatest() { + if (latestItem == null) { + return; + } + + Identifier id = Identifier.of(latestItem.getNamespace(), latestItem.getKey()); if (entries.containsKey(id)) { throw new IllegalArgumentException("DisplayData key already exists. All display data " + "for a component must be registered with a unique key.\nKey: " + id); } - Item item = Item.create(id.getNamespace(), key, type, value); - entries.put(id, item); - latestItem = item; - latestIdentifier = id; + entries.put(id, latestItem); + latestItem = null; + } + + @Override + public ItemBuilder withLabel(@Nullable String label) { + if (latestItem != null) { + latestItem = latestItem.withLabel(label); + } return this; } @Override - public ItemBuilder withLabel(String label) { - latestItem = latestItem.withLabel(label); - entries.put(latestIdentifier, latestItem); + public ItemBuilder withLinkUrl(@Nullable String url) { + if (latestItem != null) { + latestItem = latestItem.withUrl(url); + } + return this; } @Override - public ItemBuilder withLinkUrl(String url) { - latestItem = latestItem.withUrl(url); - entries.put(latestIdentifier, latestItem); + public ItemBuilder withNamespace(Class namespace) { + checkNotNull(namespace); + return withNamespace(ClassForDisplay.of(namespace)); + } + + @Override + public ItemBuilder withNamespace(ClassForDisplay namespace) { + if (latestItem != null) { + latestItem = latestItem.withNamespace(namespace); + } + return this; } private DisplayData build() { + commitLatest(); return new DisplayData(this.entries); } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/HasDisplayData.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/HasDisplayData.java index b2eca3d881..825b19267d 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/HasDisplayData.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/display/HasDisplayData.java @@ -39,6 +39,7 @@ public interface HasDisplayData { * builder * .include(subComponent) * .add("minFilter", 42) + * .addIfNotDefault("useTransactions", this.txn, false) * .add("topic", "projects/myproject/topics/mytopic") * .withLabel("Pub/Sub Topic") * .add("serviceInstance", "myservice.com/fizzbang") diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterAll.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterAll.java index bb43010ae5..d44cf6b073 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterAll.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterAll.java @@ -19,6 +19,7 @@ import com.google.cloud.dataflow.sdk.annotations.Experimental; import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.OnceTrigger; import com.google.cloud.dataflow.sdk.util.ExecutableTrigger; +import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import org.joda.time.Instant; @@ -114,4 +115,13 @@ public void onOnlyFiring(TriggerContext context) throws Exception { subtrigger.invokeOnFire(context); } } + + @Override + public String toString() { + StringBuilder builder = new StringBuilder("AfterAll.of("); + Joiner.on(", ").appendTo(builder, subTriggers); + builder.append(")"); + + return builder.toString(); + } } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterDelayFromFirstElement.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterDelayFromFirstElement.java index 71968e919c..fa4a113366 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterDelayFromFirstElement.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterDelayFromFirstElement.java @@ -34,10 +34,12 @@ import org.joda.time.Duration; import org.joda.time.Instant; +import org.joda.time.format.PeriodFormat; +import org.joda.time.format.PeriodFormatter; import java.util.List; +import java.util.Locale; import java.util.Objects; - import javax.annotation.Nullable; /** @@ -57,6 +59,8 @@ public abstract class AfterDelayFromFirstElement extend StateTags.makeSystemTagInternal(StateTags.combiningValueFromInputInternal( "delayed", InstantCoder.of(), Min.MinFn.naturalOrder())); + private static final PeriodFormatter PERIOD_FORMATTER = PeriodFormat.wordBased(Locale.ENGLISH); + /** * To complete an implementation, return the desired time from the TriggerContext. */ @@ -274,6 +278,11 @@ public boolean equals(Object object) { public int hashCode() { return Objects.hash(delay); } + + @Override + public String toString() { + return PERIOD_FORMATTER.print(delay.toPeriod()); + } } /** diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterEach.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterEach.java index 4b052faeb8..9adecdcab4 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterEach.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterEach.java @@ -21,6 +21,7 @@ import com.google.cloud.dataflow.sdk.annotations.Experimental; import com.google.cloud.dataflow.sdk.util.ExecutableTrigger; +import com.google.common.base.Joiner; import org.joda.time.Instant; import java.util.Arrays; @@ -129,6 +130,15 @@ public void onFire(Trigger.TriggerContext context) throws Exception { updateFinishedState(context); } + @Override + public String toString() { + StringBuilder builder = new StringBuilder("AfterEach.inOrder("); + Joiner.on(", ").appendTo(builder, subTriggers); + builder.append(")"); + + return builder.toString(); + } + private void updateFinishedState(TriggerContext context) { context.trigger().setFinished(context.trigger().firstUnfinishedSubTrigger() == null); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterFirst.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterFirst.java index 29b19bf9b9..d2603eaee5 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterFirst.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterFirst.java @@ -19,6 +19,7 @@ import com.google.cloud.dataflow.sdk.annotations.Experimental; import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger.OnceTrigger; import com.google.cloud.dataflow.sdk.util.ExecutableTrigger; +import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import org.joda.time.Instant; @@ -109,6 +110,15 @@ protected void onOnlyFiring(TriggerContext context) throws Exception { } } + @Override + public String toString() { + StringBuilder builder = new StringBuilder("AfterFirst.of("); + Joiner.on(", ").appendTo(builder, subTriggers); + builder.append(")"); + + return builder.toString(); + } + private void updateFinishedStatus(TriggerContext c) { boolean anyFinished = false; for (ExecutableTrigger subTrigger : c.trigger().subTriggers()) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterProcessingTime.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterProcessingTime.java index 7e89902741..5db290127f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterProcessingTime.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterProcessingTime.java @@ -24,7 +24,6 @@ import java.util.List; import java.util.Objects; - import javax.annotation.Nullable; /** @@ -75,7 +74,15 @@ protected Trigger getContinuationTrigger(List> continuationTrigger @Override public String toString() { - return "AfterProcessingTime.pastFirstElementInPane(" + timestampMappers + ")"; + StringBuilder builder = new StringBuilder("AfterProcessingTime.pastFirstElementInPane()"); + for (SerializableFunction delayFn : timestampMappers) { + builder + .append(".plusDelayOf(") + .append(delayFn) + .append(")"); + } + + return builder.toString(); } @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterWatermark.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterWatermark.java index fac2c2841b..77c422a414 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterWatermark.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterWatermark.java @@ -64,6 +64,8 @@ @Experimental(Experimental.Kind.TRIGGER) public class AfterWatermark { + private static final String TO_STRING = "AfterWatermark.pastEndOfWindow()"; + // Static factory class. private AfterWatermark() {} @@ -100,7 +102,7 @@ public interface AfterWatermarkLate extends TriggerBuil * A trigger which never fires. Used for the "early" trigger when only a late trigger was * specified. */ - private static class NeverTrigger extends OnceTrigger { + static class NeverTrigger extends OnceTrigger { protected NeverTrigger() { super(null); @@ -258,6 +260,26 @@ public void onFire(Trigger.TriggerContext context) throws Exception { } } + @Override + public String toString() { + StringBuilder builder = new StringBuilder(TO_STRING); + + Trigger earlyTrigger = subTriggers.get(EARLY_INDEX); + if (!(earlyTrigger instanceof NeverTrigger)) { + builder + .append(".withEarlyFirings(") + .append(earlyTrigger) + .append(")"); + } + + builder + .append(".withLateFirings(") + .append(subTriggers.get(LATE_INDEX)) + .append(")"); + + return builder.toString(); + } + private void onNonLateFiring(Trigger.TriggerContext context) throws Exception { // We have not yet transitioned to late firings. ExecutableTrigger earlySubtrigger = context.trigger().subTrigger(EARLY_INDEX); @@ -368,7 +390,7 @@ public FromEndOfWindow getContinuationTrigger(List> continuationTr @Override public String toString() { - return "AfterWatermark.pastEndOfWindow()"; + return TO_STRING; } @Override diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindows.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindows.java index de5140f2a5..42c97b8370 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindows.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindows.java @@ -17,6 +17,7 @@ package com.google.cloud.dataflow.sdk.transforms.windowing; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; @@ -34,6 +35,8 @@ */ public class CalendarWindows { + private static final DateTime DEFAULT_START_DATE = new DateTime(0, DateTimeZone.UTC); + /** * Returns a {@link WindowFn} that windows elements into periods measured by days. * @@ -41,7 +44,7 @@ public class CalendarWindows { * separate windows for each day. */ public static DaysWindows days(int number) { - return new DaysWindows(number, new DateTime(0, DateTimeZone.UTC), DateTimeZone.UTC); + return new DaysWindows(number, DEFAULT_START_DATE, DateTimeZone.UTC); } /** @@ -53,7 +56,7 @@ public static DaysWindows days(int number) { public static DaysWindows weeks(int number, int startDayOfWeek) { return new DaysWindows( 7 * number, - new DateTime(0, DateTimeZone.UTC).withDayOfWeek(startDayOfWeek), + DEFAULT_START_DATE.withDayOfWeek(startDayOfWeek), DateTimeZone.UTC); } @@ -66,7 +69,7 @@ public static DaysWindows weeks(int number, int startDayOfWeek) { * and the first window begins in January 2014. */ public static MonthsWindows months(int number) { - return new MonthsWindows(number, 1, new DateTime(0, DateTimeZone.UTC), DateTimeZone.UTC); + return new MonthsWindows(number, 1, DEFAULT_START_DATE, DateTimeZone.UTC); } /** @@ -78,7 +81,7 @@ public static MonthsWindows months(int number) { * America/Los_Angeles time zone. */ public static YearsWindows years(int number) { - return new YearsWindows(number, 1, 1, new DateTime(0, DateTimeZone.UTC), DateTimeZone.UTC); + return new YearsWindows(number, 1, 1, DEFAULT_START_DATE, DateTimeZone.UTC); } /** @@ -141,6 +144,14 @@ public boolean isCompatible(WindowFn other) { && timeZone == that.timeZone; } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder + .add("numDays", number) + .addIfNotDefault("startDate", new DateTime(startDate, timeZone).toInstant(), + new DateTime(DEFAULT_START_DATE, DateTimeZone.UTC).toInstant()); + } + public int getNumber() { return number; } @@ -228,6 +239,14 @@ public boolean isCompatible(WindowFn other) { && timeZone == that.timeZone; } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder + .add("numMonths", number) + .addIfNotDefault("startDate", new DateTime(startDate, timeZone).toInstant(), + new DateTime(DEFAULT_START_DATE, DateTimeZone.UTC).toInstant()); + } + public int getNumber() { return number; } @@ -324,6 +343,14 @@ public boolean isCompatible(WindowFn other) { && timeZone == that.timeZone; } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder + .add("numYears", number) + .addIfNotDefault("startDate", new DateTime(startDate, timeZone).toInstant(), + new DateTime(DEFAULT_START_DATE, DateTimeZone.UTC).toInstant()); + } + public DateTimeZone getTimeZone() { return timeZone; } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindows.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindows.java index 12a0f1b918..67d226102f 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindows.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindows.java @@ -17,6 +17,7 @@ package com.google.cloud.dataflow.sdk.transforms.windowing; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import org.joda.time.Duration; import org.joda.time.Instant; @@ -81,6 +82,13 @@ public IntervalWindow assignWindow(Instant timestamp) { return new IntervalWindow(new Instant(start), size); } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder + .add("size", size) + .addIfNotDefault("offset", offset, Duration.ZERO); + } + @Override public Coder windowCoder() { return IntervalWindow.getCoder(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/OrFinallyTrigger.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/OrFinallyTrigger.java index 652092ad6e..afbbb45343 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/OrFinallyTrigger.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/OrFinallyTrigger.java @@ -90,6 +90,11 @@ public void onFire(Trigger.TriggerContext context) throws Exception { updateFinishedState(context); } + @Override + public String toString() { + return String.format("%s.orFinally(%s)", subTriggers.get(ACTUAL), subTriggers.get(UNTIL)); + } + private void updateFinishedState(TriggerContext c) throws Exception { boolean anyStillFinished = false; for (ExecutableTrigger subTrigger : c.trigger().subTriggers()) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Repeatedly.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Repeatedly.java index 5f29c8ffe3..c1000c2261 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Repeatedly.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Repeatedly.java @@ -42,7 +42,7 @@ public class Repeatedly extends Trigger { private static final int REPEATED = 0; /** - * Create a composite trigger that repeatedly executes the trigger {@code toRepeat}, firing each + * Create a composite trigger that repeatedly executes the trigger {@code repeated}, firing each * time it fires and ignoring any indications to finish. * *

    Unless used with {@link Trigger#orFinally} the composite trigger will never finish. @@ -94,6 +94,11 @@ public void onFire(TriggerContext context) throws Exception { } } + @Override + public String toString() { + return String.format("Repeatedly.forever(%s)", subTriggers.get(REPEATED)); + } + private ExecutableTrigger getRepeated(TriggerContext context) { return context.trigger().subTrigger(REPEATED); } diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Sessions.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Sessions.java index da137c1f47..0915ae7ea4 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Sessions.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Sessions.java @@ -19,6 +19,7 @@ import com.google.cloud.dataflow.sdk.annotations.Experimental; import com.google.cloud.dataflow.sdk.annotations.Experimental.Kind; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import org.joda.time.Duration; @@ -96,6 +97,11 @@ public Duration getGapDuration() { return gapDuration; } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.add("gapDuration", gapDuration); + } + @Override public boolean equals(Object object) { if (!(object instanceof Sessions)) { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java index b0066d6124..ddb453b133 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindows.java @@ -19,6 +19,7 @@ import com.google.cloud.dataflow.sdk.annotations.Experimental; import com.google.cloud.dataflow.sdk.annotations.Experimental.Kind; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import org.joda.time.Duration; import org.joda.time.Instant; @@ -138,6 +139,14 @@ public boolean isCompatible(WindowFn other) { return equals(other); } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder + .add("size", size) + .add("period", period) + .add("offset", offset); + } + /** * Return the last start of a sliding window that contains the timestamp. */ diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java index 6793e7648f..b22457800a 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/Window.java @@ -24,6 +24,7 @@ import com.google.cloud.dataflow.sdk.transforms.GroupByKey; import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.util.AssignWindowsDoFn; import com.google.cloud.dataflow.sdk.util.WindowingStrategy; import com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode; @@ -603,6 +604,30 @@ private PCollection assignWindows( return input.apply("AssignWindows", ParDo.of(new AssignWindowsDoFn(windowFn))); } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder + .add("windowFn", windowFn.getClass()) + .include(windowFn) + .addIfNotNull("allowedLateness", allowedLateness); + + if (trigger != null && !(trigger instanceof DefaultTrigger)) { + builder.add("trigger", trigger.toString()); + } + + if (mode != null) { + builder.add("accumulationMode", mode.toString()); + } + + if (closingBehavior != null) { + builder.add("closingBehavior", closingBehavior.toString()); + } + + if (outputTimeFn != null) { + builder.add("outputTimeFn", outputTimeFn.getClass()); + } + } + @Override protected Coder getDefaultOutputCoder(PCollection input) { return input.getCoder(); diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowFn.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowFn.java index d51fc7ead4..3d43372103 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowFn.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowFn.java @@ -19,6 +19,8 @@ import com.google.cloud.dataflow.sdk.annotations.Experimental; import com.google.cloud.dataflow.sdk.annotations.Experimental.Kind; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; +import com.google.cloud.dataflow.sdk.transforms.display.HasDisplayData; import com.google.cloud.dataflow.sdk.util.WindowingStrategy; import com.google.common.collect.Ordering; @@ -48,7 +50,7 @@ * windows used by this {@code WindowFn} */ public abstract class WindowFn - implements Serializable { + implements Serializable, HasDisplayData { /** * Information available when running {@link #assignWindows}. */ @@ -176,6 +178,16 @@ public boolean assignsToSingleWindow() { return false; } + /** + * {@inheritDoc} + * + *

    By default, does not register any display data. Implementors may override this method + * to provide their own display metadata. + */ + @Override + public void populateDisplayData(DisplayData.Builder builder) { + } + /** * A compatibility adapter that will return the assigned timestamps according to the * {@link WindowFn}, which was the prior policy. Specifying the assigned output timestamps diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CombineFnUtil.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CombineFnUtil.java index d9744800b9..b895fa35e0 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CombineFnUtil.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/CombineFnUtil.java @@ -25,6 +25,7 @@ import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.CombineFnWithContext; import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.Context; import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.util.state.StateContext; import java.io.IOException; @@ -99,6 +100,10 @@ public Coder getDefaultOutputCoder( CoderRegistry registry, Coder inputCoder) throws CannotProvideCoderException { return combineFn.getDefaultOutputCoder(registry, inputCoder); } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + combineFn.populateDisplayData(builder); + } }; } } @@ -144,6 +149,10 @@ public Coder getDefaultOutputCoder(CoderRegistry registry, Coder key Coder inputCoder) throws CannotProvideCoderException { return combineFn.getDefaultOutputCoder(registry, keyCoder, inputCoder); } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + combineFn.populateDisplayData(builder); + } private void writeObject(@SuppressWarnings("unused") ObjectOutputStream out) throws IOException { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReshuffleTrigger.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReshuffleTrigger.java index 248f00589d..fb7efd9620 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReshuffleTrigger.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/util/ReshuffleTrigger.java @@ -58,4 +58,9 @@ public boolean shouldFire(Trigger.TriggerContext context) throws Exception { @Override public void onFire(Trigger.TriggerContext context) throws Exception { } + + @Override + public String toString() { + return "ReshuffleTrigger()"; + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/AvroSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/AvroSourceTest.java index 0990294d20..e2c4299fe5 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/AvroSourceTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/AvroSourceTest.java @@ -16,6 +16,8 @@ package com.google.cloud.dataflow.sdk.io; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; + import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; @@ -28,6 +30,7 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.testing.SourceTestUtils; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.common.base.MoreObjects; import org.apache.avro.Schema; @@ -506,6 +509,18 @@ public void testSeekerFindAllLocations() { } } + @Test + public void testDisplayData() { + AvroSource source = AvroSource + .from("foobar.txt") + .withSchema(Bird.class) + .withMinBundleSize(1234); + + DisplayData displayData = DisplayData.from(source); + assertThat(displayData, hasDisplayItem("filePattern", "foobar.txt")); + assertThat(displayData, hasDisplayItem("minBundleSize", 1234)); + } + /** * Class that will encode to a fixed size: 16 bytes. * diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/BoundedReadFromUnboundedSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/BoundedReadFromUnboundedSourceTest.java index d01c036344..963ceb9044 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/BoundedReadFromUnboundedSourceTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/BoundedReadFromUnboundedSourceTest.java @@ -16,6 +16,8 @@ package com.google.cloud.dataflow.sdk.io; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includes; + import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; @@ -28,6 +30,7 @@ import com.google.cloud.dataflow.sdk.testing.RunnableOnService; import com.google.cloud.dataflow.sdk.testing.TestPipeline; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; @@ -37,13 +40,14 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +import java.io.Serializable; import java.util.ArrayList; import java.util.Collections; import java.util.List; /** Unit tests for {@link BoundedReadFromUnboundedSource}. */ @RunWith(JUnit4.class) -public class BoundedReadFromUnboundedSourceTest { +public class BoundedReadFromUnboundedSourceTest implements Serializable{ private static final int NUM_RECORDS = 100; private static List finalizeTracker = null; @@ -65,6 +69,19 @@ public void testTimeBound() throws Exception { test(false, true); } + @Test + public void testForwardsDisplayData() { + TestCountingSource src = new TestCountingSource(1234) { + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.add("foo", "bar"); + } + }; + + BoundedReadFromUnboundedSource> read = Read.from(src).withMaxNumRecords(5); + assertThat(DisplayData.from(read), includes(src)); + } + private static class Checker implements SerializableFunction>, Void> { private final boolean dedup; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/CompressedSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/CompressedSourceTest.java index fe855101c6..fe0e59ad70 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/CompressedSourceTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/CompressedSourceTest.java @@ -16,9 +16,14 @@ package com.google.cloud.dataflow.sdk.io; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasKey; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includes; + import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.instanceOf; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import com.google.cloud.dataflow.sdk.Pipeline; @@ -31,6 +36,7 @@ import com.google.cloud.dataflow.sdk.testing.DataflowAssert; import com.google.cloud.dataflow.sdk.testing.SourceTestUtils; import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.common.io.Files; import com.google.common.primitives.Bytes; @@ -330,6 +336,27 @@ public void testCompressedReadMultipleFiles() throws Exception { p.run(); } + @Test + public void testDisplayData() { + ByteSource inputSource = new ByteSource("foobar.txt", 1) { + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.add("foo", "bar"); + } + }; + + CompressedSource compressedSource = CompressedSource.from(inputSource); + CompressedSource gzipSource = compressedSource.withDecompression(CompressionMode.GZIP); + + DisplayData compressedSourceDisplayData = DisplayData.from(compressedSource); + DisplayData gzipDisplayData = DisplayData.from(gzipSource); + + assertThat(compressedSourceDisplayData, hasDisplayItem(hasKey("compressionMode"))); + assertThat(gzipDisplayData, hasDisplayItem("compressionMode", CompressionMode.GZIP.toString())); + assertThat(compressedSourceDisplayData, hasDisplayItem("source", inputSource.getClass())); + assertThat(compressedSourceDisplayData, includes(inputSource)); + } + /** * Generate byte array of given size. */ diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/DatastoreIOTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/DatastoreIOTest.java index 4cc3ace1b5..edf7c243f1 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/DatastoreIOTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/DatastoreIOTest.java @@ -17,6 +17,8 @@ package com.google.cloud.dataflow.sdk.io; import static com.google.api.services.datastore.client.DatastoreHelper.makeKey; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; + import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.lessThanOrEqualTo; @@ -54,6 +56,7 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.testing.ExpectedLogs; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.util.TestCredential; import com.google.common.collect.Lists; @@ -193,6 +196,22 @@ public void testSourceValidationSucceedsNamespace() throws Exception { source.validate(); } + @Test + public void testSourceDipslayData() { + DatastoreIO.Source source = DatastoreIO.source() + .withDataset(DATASET) + .withQuery(QUERY) + .withHost(HOST) + .withNamespace(NAMESPACE); + + DisplayData displayData = DisplayData.from(source); + + assertThat(displayData, hasDisplayItem("dataset", DATASET)); + assertThat(displayData, hasDisplayItem("query", QUERY.toString())); + assertThat(displayData, hasDisplayItem("host", HOST)); + assertThat(displayData, hasDisplayItem("namespace", NAMESPACE)); + } + @Test public void testSinkDoesNotAllowNullHost() throws Exception { thrown.expect(NullPointerException.class); @@ -225,6 +244,18 @@ public void testSinkValidationSucceedsWithDataset() throws Exception { sink.validate(testPipelineOptions(null)); } + @Test + public void testSinkDipslayData() { + DatastoreIO.Sink sink = DatastoreIO.sink() + .withDataset(DATASET) + .withHost(HOST); + + DisplayData displayData = DisplayData.from(sink); + + assertThat(displayData, hasDisplayItem("dataset", DATASET)); + assertThat(displayData, hasDisplayItem("host", HOST)); + } + @Test public void testQuerySplitBasic() throws Exception { KindExpression mykind = KindExpression.newBuilder().setName("mykind").build(); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/ReadTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/ReadTest.java index 8dc517a8e8..530687be14 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/ReadTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/ReadTest.java @@ -16,10 +16,17 @@ package com.google.cloud.dataflow.sdk.io; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includes; + +import static org.hamcrest.MatcherAssert.assertThat; + import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.io.UnboundedSource.CheckpointMark; import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; +import org.joda.time.Duration; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -27,17 +34,17 @@ import org.junit.runners.JUnit4; import java.io.IOException; +import java.io.Serializable; import java.util.List; - import javax.annotation.Nullable; /** * Tests for {@link Read}. */ @RunWith(JUnit4.class) -public class ReadTest { +public class ReadTest implements Serializable{ @Rule - public ExpectedException thrown = ExpectedException.none(); + public transient ExpectedException thrown = ExpectedException.none(); @Test public void failsWhenCustomBoundedSourceIsNotSerializable() { @@ -61,6 +68,38 @@ public void succeedsWhenCustomUnboundedSourceIsSerializable() { Read.from(new SerializableUnboundedSource()); } + @Test + public void testDisplayData() { + SerializableBoundedSource boundedSource = new SerializableBoundedSource() { + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.add("foo", "bar"); + } + }; + SerializableUnboundedSource unboundedSource = new SerializableUnboundedSource() { + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.add("foo", "bar"); + } + }; + Duration maxReadTime = Duration.standardMinutes(2345); + + Read.Bounded bounded = Read.from(boundedSource); + BoundedReadFromUnboundedSource unbounded = Read.from(unboundedSource) + .withMaxNumRecords(1234) + .withMaxReadTime(maxReadTime); + + DisplayData boundedDisplayData = DisplayData.from(bounded); + assertThat(boundedDisplayData, hasDisplayItem("source", boundedSource.getClass())); + assertThat(boundedDisplayData, includes(boundedSource)); + + DisplayData unboundedDisplayData = DisplayData.from(unbounded); + assertThat(unboundedDisplayData, hasDisplayItem("source", unboundedSource.getClass())); + assertThat(unboundedDisplayData, includes(unboundedSource)); + assertThat(unboundedDisplayData, hasDisplayItem("maxRecords", 1234)); + assertThat(unboundedDisplayData, hasDisplayItem("maxReadTime", maxReadTime)); + } + private abstract static class CustomBoundedSource extends BoundedSource { @Override public List> splitIntoBundles( diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/WriteTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/WriteTest.java index 0d4a70fd19..ca1c7df00d 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/WriteTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/WriteTest.java @@ -14,6 +14,9 @@ package com.google.cloud.dataflow.sdk.io; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includes; + import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; @@ -38,6 +41,7 @@ import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.transforms.SimpleFunction; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions; import com.google.cloud.dataflow.sdk.transforms.windowing.Window; @@ -151,6 +155,23 @@ public void testWriteWithSessions() { new WindowAndReshuffle(Window.into(Sessions.withGapDuration(Duration.millis(1))))); } + @Test + public void testDisplayData() { + TestSink sink = new TestSink() { + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.add("foo", "bar"); + } + }; + Write.Bound write = Write.to(sink); + DisplayData displayData = DisplayData.from(write); + + assertThat(displayData, hasDisplayItem("sink", sink.getClass())); + assertThat(displayData, includes(sink)); + } + + + /** * Performs a Write transform and verifies the Write transform calls the appropriate methods on * a test sink in the correct order, as well as verifies that the elements of a PCollection are diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/XmlSinkTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/XmlSinkTest.java index 4b8380d89e..22c1b59f45 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/XmlSinkTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/XmlSinkTest.java @@ -16,6 +16,9 @@ package com.google.cloud.dataflow.sdk.io; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; + +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; @@ -23,6 +26,7 @@ import com.google.cloud.dataflow.sdk.io.XmlSink.XmlWriter; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.common.collect.Lists; import org.junit.Rule; @@ -40,7 +44,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; - import javax.xml.bind.annotation.XmlElement; import javax.xml.bind.annotation.XmlRootElement; import javax.xml.bind.annotation.XmlType; @@ -160,6 +163,20 @@ public void testCreateWriter() throws Exception { assertNotNull(writer.marshaller); } + @Test + public void testDisplayData() { + XmlSink.Bound sink = XmlSink.write() + .toFilenamePrefix("foobar") + .withRootElement("bird") + .ofRecordClass(Integer.class); + + DisplayData displayData = DisplayData.from(sink); + + assertThat(displayData, hasDisplayItem("fileNamePattern", "foobar-SSSSS-of-NNNNN.xml")); + assertThat(displayData, hasDisplayItem("rootElement", "bird")); + assertThat(displayData, hasDisplayItem("recordClass", Integer.class)); + } + /** * Write a bundle with an XmlWriter and verify the output is expected. */ diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/XmlSourceTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/XmlSourceTest.java index 5618ec7a10..198d3857dc 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/XmlSourceTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/io/XmlSourceTest.java @@ -17,6 +17,8 @@ import static com.google.cloud.dataflow.sdk.testing.SourceTestUtils.assertSplitAtFractionExhaustive; import static com.google.cloud.dataflow.sdk.testing.SourceTestUtils.assertSplitAtFractionFails; import static com.google.cloud.dataflow.sdk.testing.SourceTestUtils.assertSplitAtFractionSucceedsAndConsistent; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; + import static org.hamcrest.Matchers.both; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.containsString; @@ -30,6 +32,7 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.testing.DataflowAssert; import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.common.collect.ImmutableList; @@ -819,4 +822,23 @@ public void testReadXMLFilePattern() throws IOException { DataflowAssert.that(output).containsInAnyOrder(expectedResults); p.run(); } + + @Test + public void testDisplayData() { + + + XmlSource source = XmlSource + .from("foo.xml") + .withRootElement("bird") + .withRecordElement("cat") + .withMinBundleSize(1234) + .withRecordClass(Integer.class); + DisplayData displayData = DisplayData.from(source); + + assertThat(displayData, hasDisplayItem("filePattern", "foo.xml")); + assertThat(displayData, hasDisplayItem("rootElement", "bird")); + assertThat(displayData, hasDisplayItem("recordElement", "cat")); + assertThat(displayData, hasDisplayItem("recordClass", Integer.class)); + assertThat(displayData, hasDisplayItem("minBundleSize", 1234)); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsReflectorTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsReflectorTest.java new file mode 100644 index 0000000000..560fef1b38 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/PipelineOptionsReflectorTest.java @@ -0,0 +1,193 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.options; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.allOf; +import static org.hamcrest.Matchers.hasItem; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.isOneOf; +import static org.hamcrest.Matchers.not; + +import com.google.common.collect.ImmutableSet; +import org.hamcrest.FeatureMatcher; +import org.hamcrest.Matcher; +import org.hamcrest.Matchers; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.Set; + +/** + * Unit tests for {@link PipelineOptionsReflector}. + */ +@RunWith(JUnit4.class) +public class PipelineOptionsReflectorTest { + @Test + public void testGetOptionSpecs() throws NoSuchMethodException { + Set properties = + PipelineOptionsReflector.getOptionSpecs(SimpleOptions.class); + + assertThat(properties, Matchers.hasItems(PipelineOptionSpec.of( + SimpleOptions.class, "foo", SimpleOptions.class.getDeclaredMethod("getFoo")))); + } + + interface SimpleOptions extends PipelineOptions { + String getFoo(); + void setFoo(String value); + } + + @Test + public void testFiltersNonGetterMethods() { + Set properties = + PipelineOptionsReflector.getOptionSpecs(OnlyTwoValidGetters.class); + + assertThat(properties, not(hasItem(hasName(isOneOf("misspelled", "hasParameter", "prefix"))))); + } + + interface OnlyTwoValidGetters extends PipelineOptions { + String getFoo(); + void setFoo(String value); + + boolean isBar(); + void setBar(boolean value); + + String gtMisspelled(); + void setMisspelled(String value); + + String getHasParameter(String value); + void setHasParameter(String value); + + String noPrefix(); + void setNoPrefix(String value); + } + + @Test + public void testBaseClassOptions() { + Set props = + PipelineOptionsReflector.getOptionSpecs(ExtendsSimpleOptions.class); + + assertThat(props, Matchers.hasItem( + allOf(hasName("foo"), hasClass(SimpleOptions.class)))); + assertThat(props, Matchers.hasItem( + allOf(hasName("foo"), hasClass(ExtendsSimpleOptions.class)))); + assertThat(props, Matchers.hasItem( + allOf(hasName("bar"), hasClass(ExtendsSimpleOptions.class)))); + } + + interface ExtendsSimpleOptions extends SimpleOptions { + @Override String getFoo(); + @Override void setFoo(String value); + + String getBar(); + void setBar(String value); + } + + @Test + public void testExcludesNonPipelineOptionsMethods() { + Set properties = + PipelineOptionsReflector.getOptionSpecs(ExtendsNonPipelineOptions.class); + + assertThat(properties, not(hasItem(hasName("foo")))); + } + + interface NoExtendsClause { + String getFoo(); + void setFoo(String value); + } + + interface ExtendsNonPipelineOptions extends NoExtendsClause, PipelineOptions {} + + @Test + public void testExcludesHiddenInterfaces() { + Set properties = + PipelineOptionsReflector.getOptionSpecs(HiddenOptions.class); + + assertThat(properties, not(hasItem(hasName("foo")))); + } + + @Hidden + interface HiddenOptions extends PipelineOptions { + String getFoo(); + void setFoo(String value); + } + + @Test + public void testMultipleInputInterfaces() { + Set> interfaces = + ImmutableSet.>of( + BaseOptions.class, + ExtendOptions1.class, + ExtendOptions2.class); + + Set props = PipelineOptionsReflector.getOptionSpecs(interfaces); + + assertThat(props, Matchers.hasItem(allOf(hasName("baseOption"), hasClass(BaseOptions.class)))); + assertThat(props, Matchers.hasItem( + allOf(hasName("extendOption1"), hasClass(ExtendOptions1.class)))); + assertThat(props, Matchers.hasItem( + allOf(hasName("extendOption2"), hasClass(ExtendOptions2.class)))); + } + + interface BaseOptions extends PipelineOptions { + String getBaseOption(); + void setBaseOption(String value); + } + + interface ExtendOptions1 extends BaseOptions { + String getExtendOption1(); + void setExtendOption1(String value); + } + + interface ExtendOptions2 extends BaseOptions { + String getExtendOption2(); + void setExtendOption2(String value); + } + + private static Matcher hasName(String name) { + return hasName(is(name)); + } + + private static Matcher hasName(Matcher matcher) { + return new FeatureMatcher(matcher, "name", "name") { + @Override + protected String featureValueOf(PipelineOptionSpec actual) { + return actual.getName(); + } + }; + } + + private static Matcher hasClass(Class clazz) { + return new FeatureMatcher>( + Matchers.>is(clazz), "defining class", "class") { + @Override + protected Class featureValueOf(PipelineOptionSpec actual) { + return actual.getDefiningInterface(); + } + }; + } + + private static Matcher hasGetter(String methodName) { + return new FeatureMatcher( + is(methodName), "getter method", "name") { + @Override + protected String featureValueOf(PipelineOptionSpec actual) { + return actual.getGetterMethod().getName(); + } + }; + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java index 7eb5822b43..70e685d500 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/options/ProxyInvocationHandlerTest.java @@ -16,12 +16,23 @@ package com.google.cloud.dataflow.sdk.options; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasKey; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasNamespace; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasType; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasValue; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.allOf; +import static org.hamcrest.Matchers.hasItem; +import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; @@ -32,12 +43,19 @@ import com.fasterxml.jackson.databind.JsonMappingException; import com.fasterxml.jackson.databind.ObjectMapper; +import org.hamcrest.Matchers; +import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import org.junit.rules.ExternalResource; +import org.junit.rules.TestRule; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +import java.io.IOException; +import java.io.Serializable; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; @@ -47,6 +65,14 @@ @RunWith(JUnit4.class) public class ProxyInvocationHandlerTest { @Rule public ExpectedException expectedException = ExpectedException.none(); + @Rule public TestRule resetPipelineOptionsRegistry = new ExternalResource() { + @Override + protected void before() { + PipelineOptionsFactory.resetRegistry(); + } + }; + + private static final ObjectMapper MAPPER = new ObjectMapper(); /** A test interface with some primitives and objects. */ public static interface Simple extends PipelineOptions { @@ -431,6 +457,19 @@ public void testPartialMethodConflictProvidesSameValue() throws Exception { assertEquals(5, partialMethodConflict.as(Simple.class).getPrimitive()); } + @Test + public void testResetRegistry() { + Set> defaultRegistry = + new HashSet<>(PipelineOptionsFactory.getRegisteredOptions()); + assertThat(defaultRegistry, not(hasItem(FooOptions.class))); + + PipelineOptionsFactory.register(FooOptions.class); + assertThat(PipelineOptionsFactory.getRegisteredOptions(), hasItem(FooOptions.class)); + + PipelineOptionsFactory.resetRegistry(); + assertEquals(defaultRegistry, PipelineOptionsFactory.getRegisteredOptions()); + } + @Test public void testJsonConversionForDefault() throws Exception { PipelineOptions options = PipelineOptionsFactory.create(); @@ -682,10 +721,233 @@ public void testJsonConversionOfSerializableWithMetadataProperty() throws Except assertEquals("TestString", options2.getValue().getValue()); } + @Test + public void testDisplayDataItemProperties() { + PipelineOptions options = PipelineOptionsFactory.create(); + options.setTempLocation("myTemp"); + DisplayData displayData = DisplayData.from(options); + + assertThat(displayData, hasDisplayItem(allOf( + hasKey("tempLocation"), + hasType(DisplayData.Type.STRING), + hasValue("myTemp"), + hasNamespace(PipelineOptions.class) + ))); + } + + @Test + public void testDisplayDataTypes() { + Instant now = Instant.now(); + + TypedOptions options = PipelineOptionsFactory.as(TypedOptions.class); + options.setInteger(1234); + options.setTimestamp(now); + options.setJavaClass(ProxyInvocationHandlerTest.class); + options.setObject(new Serializable() { + @Override + public String toString() { + return "foobar"; + } + }); + + DisplayData displayData = DisplayData.from(options); + + assertThat(displayData, hasDisplayItem("integer", 1234)); + assertThat(displayData, hasDisplayItem("timestamp", now)); + assertThat(displayData, hasDisplayItem("javaClass", ProxyInvocationHandlerTest.class)); + assertThat(displayData, hasDisplayItem("object", "foobar")); + } + + interface TypedOptions extends PipelineOptions { + int getInteger(); + void setInteger(int value); + + Instant getTimestamp(); + void setTimestamp(Instant value); + + Class getJavaClass(); + void setJavaClass(Class value); + + Object getObject(); + void setObject(Object value); + } + + @Test + public void testDisplayDataInheritanceNamespace() { + ExtendsBaseOptions options = PipelineOptionsFactory.as(ExtendsBaseOptions.class); + options.setFoo("bar"); + + DisplayData displayData = DisplayData.from(options); + + assertThat(displayData, hasDisplayItem(allOf( + hasKey("foo"), + hasValue("bar"), + hasNamespace(ExtendsBaseOptions.class) + ))); + } + + interface BaseOptions extends PipelineOptions { + String getFoo(); + void setFoo(String value); + } + + interface ExtendsBaseOptions extends BaseOptions { + @Override String getFoo(); + @Override void setFoo(String value); + } + + @Test + public void testDisplayDataExcludedFromOverriddenBaseClass() { + ExtendsBaseOptions options = PipelineOptionsFactory.as(ExtendsBaseOptions.class); + options.setFoo("bar"); + + DisplayData displayData = DisplayData.from(options); + assertThat(displayData, not(hasDisplayItem(hasNamespace(BaseOptions.class)))); + } + + @Test + public void testDisplayDataIncludedForDisjointInterfaceHierarchies() { + FooOptions fooOptions = PipelineOptionsFactory.as(FooOptions.class); + fooOptions.setFoo("foo"); + + BarOptions barOptions = fooOptions.as(BarOptions.class); + barOptions.setBar("bar"); + + DisplayData data = DisplayData.from(barOptions); + assertThat(data, hasDisplayItem(allOf(hasKey("foo"), hasNamespace(FooOptions.class)))); + assertThat(data, hasDisplayItem(allOf(hasKey("bar"), hasNamespace(BarOptions.class)))); + } + + interface FooOptions extends PipelineOptions { + String getFoo(); + void setFoo(String value); + } + + interface BarOptions extends PipelineOptions { + String getBar(); + void setBar(String value); + } + + @Test + public void testDisplayDataExcludesDefaultValues() { + PipelineOptions options = PipelineOptionsFactory.as(HasDefaults.class); + DisplayData data = DisplayData.from(options); + + assertThat(data, not(hasDisplayItem(hasKey("foo")))); + } + + interface HasDefaults extends PipelineOptions { + @Default.String("bar") + String getFoo(); + void setFoo(String value); + } + + @Test + public void testDisplayDataExcludesValuesAccessedButNeverSet() { + HasDefaults options = PipelineOptionsFactory.as(HasDefaults.class); + assertEquals("bar", options.getFoo()); + + DisplayData data = DisplayData.from(options); + assertThat(data, not(hasDisplayItem(hasKey("foo")))); + } + + @Test + public void testDisplayDataIncludesExplicitlySetDefaults() { + HasDefaults options = PipelineOptionsFactory.as(HasDefaults.class); + String defaultValue = options.getFoo(); + options.setFoo(defaultValue); + + DisplayData data = DisplayData.from(options); + assertThat(data, hasDisplayItem("foo", defaultValue)); + } + + @Test + public void testDisplayDataNullValuesConvertedToEmptyString() { + FooOptions options = PipelineOptionsFactory.as(FooOptions.class); + options.setFoo(null); + + DisplayData data = DisplayData.from(options); + assertThat(data, hasDisplayItem("foo", "")); + } + + @Test + public void testDisplayDataJsonSerialization() throws IOException { + FooOptions options = PipelineOptionsFactory.as(FooOptions.class); + options.setFoo("bar"); + + @SuppressWarnings("unchecked") + Map map = MAPPER.readValue(MAPPER.writeValueAsBytes(options), Map.class); + + assertThat("main pipeline options data keyed as 'options'", map, Matchers.hasKey("options")); + assertThat("display data keyed as 'display_data'", map, Matchers.hasKey("display_data")); + + Map expectedDisplayItem = ImmutableMap.builder() + .put("namespace", FooOptions.class.getName()) + .put("key", "foo") + .put("value", "bar") + .put("type", "STRING") + .build(); + + @SuppressWarnings("unchecked") + List> deserializedDisplayData = (List>) map.get("display_data"); + assertThat(deserializedDisplayData, hasItem(expectedDisplayItem)); + } + + @Test + public void testDisplayDataFromDeserializedJson() throws Exception { + FooOptions options = PipelineOptionsFactory.as(FooOptions.class); + options.setFoo("bar"); + DisplayData data = DisplayData.from(options); + assertThat(data, hasDisplayItem("foo", "bar")); + + FooOptions deserializedOptions = serializeDeserialize(FooOptions.class, options); + DisplayData dataAfterDeserialization = DisplayData.from(deserializedOptions); + assertEquals(data, dataAfterDeserialization); + } + + @Test + public void testDisplayDataDeserializationWithRegistration() throws Exception { + PipelineOptionsFactory.register(HasClassOptions.class); + HasClassOptions options = PipelineOptionsFactory.as(HasClassOptions.class); + options.setClassOption(ProxyInvocationHandlerTest.class); + + PipelineOptions deserializedOptions = serializeDeserialize(PipelineOptions.class, options); + DisplayData displayData = DisplayData.from(deserializedOptions); + assertThat(displayData, hasDisplayItem("classOption", ProxyInvocationHandlerTest.class)); + } + + @Test + public void testDisplayDataMissingPipelineOptionsRegistration() throws Exception { + HasClassOptions options = PipelineOptionsFactory.as(HasClassOptions.class); + options.setClassOption(ProxyInvocationHandlerTest.class); + + PipelineOptions deserializedOptions = serializeDeserialize(PipelineOptions.class, options); + DisplayData displayData = DisplayData.from(deserializedOptions); + String expectedJsonValue = MAPPER.writeValueAsString(ProxyInvocationHandlerTest.class); + assertThat(displayData, hasDisplayItem("classOption", expectedJsonValue)); + } + + interface HasClassOptions extends PipelineOptions { + Class getClassOption(); + void setClassOption(Class value); + } + + @Test + public void testDisplayDataJsonValueSetAfterDeserialization() throws Exception { + FooOptions options = PipelineOptionsFactory.as(FooOptions.class); + options.setFoo("bar"); + DisplayData data = DisplayData.from(options); + assertThat(data, hasDisplayItem("foo", "bar")); + + FooOptions deserializedOptions = serializeDeserialize(FooOptions.class, options); + deserializedOptions.setFoo("baz"); + DisplayData dataAfterDeserialization = DisplayData.from(deserializedOptions); + assertThat(dataAfterDeserialization, hasDisplayItem("foo", "baz")); + } + private T serializeDeserialize(Class kls, PipelineOptions options) throws Exception { - ObjectMapper mapper = new ObjectMapper(); - String value = mapper.writeValueAsString(options); - return mapper.readValue(value, PipelineOptions.class).as(kls); + String value = MAPPER.writeValueAsString(options); + return MAPPER.readValue(value, PipelineOptions.class).as(kls); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java index 83cfc9b4f9..699f5022c2 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/DataflowPipelineTranslatorTest.java @@ -19,6 +19,7 @@ import static com.google.cloud.dataflow.sdk.util.Structs.addObject; import static com.google.cloud.dataflow.sdk.util.Structs.getDictionary; import static com.google.cloud.dataflow.sdk.util.Structs.getString; + import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.hasKey; import static org.hamcrest.core.IsInstanceOf.instanceOf; @@ -198,8 +199,9 @@ public void testSettingOfSdkPipelineOptions() throws IOException { settings.put("numberOfWorkerHarnessThreads", 0); settings.put("experiments", null); - assertEquals(ImmutableMap.of("options", settings), - job.getEnvironment().getSdkPipelineOptions()); + Map sdkPipelineOptions = job.getEnvironment().getSdkPipelineOptions(); + assertThat(sdkPipelineOptions, hasKey("options")); + assertEquals(settings, sdkPipelineOptions.get("options")); } @Test @@ -819,8 +821,8 @@ public void processElement(ProcessContext c) throws Exception { public void populateDisplayData(DisplayData.Builder builder) { builder .add("foo", "bar") - .add("foo2", 123) - .withLabel("Test Value") + .add("foo2", DataflowPipelineTranslatorTest.class) + .withLabel("Test Class") .withLinkUrl("http://www.google.com"); } }; @@ -833,7 +835,7 @@ public void processElement(ProcessContext c) throws Exception { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("foo3", "barge"); + builder.add("foo3", 1234); } }; @@ -854,33 +856,50 @@ public void populateDisplayData(DisplayData.Builder builder) { Map parDo2Properties = steps.get(2).getProperties(); assertThat(parDo1Properties, hasKey("display_data")); + @SuppressWarnings("unchecked") Collection> fn1displayData = (Collection>) parDo1Properties.get("display_data"); + @SuppressWarnings("unchecked") Collection> fn2displayData = (Collection>) parDo2Properties.get("display_data"); - ImmutableSet> expectedFn1DisplayData = ImmutableSet.of( - ImmutableMap.builder() + ImmutableSet> expectedFn1DisplayData = ImmutableSet.of( + ImmutableMap.builder() .put("key", "foo") .put("type", "STRING") .put("value", "bar") .put("namespace", fn1.getClass().getName()) .build(), - ImmutableMap.builder() + ImmutableMap.builder() + .put("key", "fn") + .put("type", "JAVA_CLASS") + .put("value", fn1.getClass().getName()) + .put("shortValue", fn1.getClass().getSimpleName()) + .put("namespace", parDo1.getClass().getName()) + .build(), + ImmutableMap.builder() .put("key", "foo2") - .put("type", "INTEGER") - .put("value", "123") + .put("type", "JAVA_CLASS") + .put("value", DataflowPipelineTranslatorTest.class.getName()) + .put("shortValue", DataflowPipelineTranslatorTest.class.getSimpleName()) .put("namespace", fn1.getClass().getName()) - .put("label", "Test Value") + .put("label", "Test Class") .put("linkUrl", "http://www.google.com") .build() ); - ImmutableSet> expectedFn2DisplayData = ImmutableSet.of( - ImmutableMap.builder() + ImmutableSet> expectedFn2DisplayData = ImmutableSet.of( + ImmutableMap.builder() + .put("key", "fn") + .put("type", "JAVA_CLASS") + .put("value", fn2.getClass().getName()) + .put("shortValue", fn2.getClass().getSimpleName()) + .put("namespace", parDo2.getClass().getName()) + .build(), + ImmutableMap.builder() .put("key", "foo3") - .put("type", "STRING") - .put("value", "barge") + .put("type", "INTEGER") + .put("value", 1234L) .put("namespace", fn2.getClass().getName()) .build() ); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/inprocess/ForwardingPTransformTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/inprocess/ForwardingPTransformTest.java index 2e283f50b8..00e9785462 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/inprocess/ForwardingPTransformTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/runners/inprocess/ForwardingPTransformTest.java @@ -23,6 +23,7 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.values.PCollection; import org.junit.Before; @@ -97,4 +98,13 @@ public void getDefaultOutputCoderDelegates() throws Exception { when(delegate.getDefaultOutputCoder(input, output)).thenReturn(outputCoder); assertThat(forwarding.getDefaultOutputCoder(input, output), equalTo(outputCoder)); } + + @Test + public void populateDisplayDataDelegates() { + DisplayData.Builder builder = mock(DisplayData.Builder.class); + doThrow(RuntimeException.class).when(delegate).populateDisplayData(builder); + + thrown.expect(RuntimeException.class); + forwarding.populateDisplayData(builder); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantilesTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantilesTest.java index e366e6948d..35cb1aa111 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantilesTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateQuantilesTest.java @@ -17,6 +17,9 @@ package com.google.cloud.dataflow.sdk.transforms; import static com.google.cloud.dataflow.sdk.TestUtils.checkCombineFn; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; + +import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.collection.IsIterableContainingInOrder.contains; import com.google.cloud.dataflow.sdk.Pipeline; @@ -27,6 +30,7 @@ import com.google.cloud.dataflow.sdk.testing.DataflowAssert; import com.google.cloud.dataflow.sdk.testing.TestPipeline; import com.google.cloud.dataflow.sdk.transforms.ApproximateQuantiles.ApproximateQuantilesCombineFn; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; @@ -233,6 +237,16 @@ public void testAlternateComparator() { Arrays.asList("b", "aaa", "ccccc")); } + @Test + public void testDisplayData() { + Top.Largest comparer = new Top.Largest(); + PTransform approxQuanitiles = ApproximateQuantiles.globally(20, comparer); + DisplayData displayData = DisplayData.from(approxQuanitiles); + + assertThat(displayData, hasDisplayItem("numQuantiles", 20)); + assertThat(displayData, hasDisplayItem("comparer", comparer.getClass())); + } + private Matcher> quantileMatcher( int size, int numQuantiles, int absoluteError) { List> quantiles = new ArrayList<>(); diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUniqueTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUniqueTest.java index 39731bb931..5f008079e9 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUniqueTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ApproximateUniqueTest.java @@ -16,6 +16,10 @@ package com.google.cloud.dataflow.sdk.transforms; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasKey; + +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -25,6 +29,7 @@ import com.google.cloud.dataflow.sdk.testing.DataflowAssert; import com.google.cloud.dataflow.sdk.testing.RunnableOnService; import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PCollectionView; @@ -288,4 +293,17 @@ public Void apply(Iterable> estimatePerKey) { return null; } } + + @Test + public void testDisplayData() { + ApproximateUnique.Globally specifiedSampleSize = ApproximateUnique.globally(1234); + ApproximateUnique.PerKey specifiedMaxError = ApproximateUnique.perKey(0.1234); + + assertThat(DisplayData.from(specifiedSampleSize), hasDisplayItem("sampleSize", 1234)); + + DisplayData maxErrorDisplayData = DisplayData.from(specifiedMaxError); + assertThat(maxErrorDisplayData, hasDisplayItem("maximumEstimationError", 0.1234)); + assertThat("calculated sampleSize should be included", maxErrorDisplayData, + hasDisplayItem(hasKey("sampleSize"))); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineFnsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineFnsTest.java index ad37708677..aa6e035f6b 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineFnsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineFnsTest.java @@ -15,6 +15,9 @@ */ package com.google.cloud.dataflow.sdk.transforms; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includes; + import static org.junit.Assert.assertThat; import com.google.cloud.dataflow.sdk.Pipeline; @@ -33,6 +36,7 @@ import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext; import com.google.cloud.dataflow.sdk.transforms.Max.MaxIntegerFn; import com.google.cloud.dataflow.sdk.transforms.Min.MinIntegerFn; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PCollectionView; @@ -58,7 +62,7 @@ * Unit tests for {@link CombineFns}. */ @RunWith(JUnit4.class) -public class CombineFnsTest { +public class CombineFnsTest { @Rule public ExpectedException expectedException = ExpectedException.none(); @Test @@ -275,6 +279,69 @@ public void testComposedCombineNullValues() { p.run(); } + @Test + public void testComposedCombineDisplayData() { + SimpleFunction extractFn = new SimpleFunction() { + @Override + public String apply(String input) { + return input; + } + }; + + DisplayDataCombineFn combineFn1 = new DisplayDataCombineFn("value1"); + DisplayDataCombineFn combineFn2 = new DisplayDataCombineFn("value2"); + + CombineFns.ComposedCombineFn composedCombine = CombineFns.compose() + .with(extractFn, combineFn1, new TupleTag()) + .with(extractFn, combineFn2, new TupleTag()); + + DisplayData displayData = DisplayData.from(composedCombine); + assertThat(displayData, hasDisplayItem("combineFn1", combineFn1.getClass())); + assertThat(displayData, hasDisplayItem("combineFn2", combineFn2.getClass())); + + String nsBase = DisplayDataCombineFn.class.getName(); + assertThat(displayData, includes(combineFn1, nsBase + "#1")); + assertThat(displayData, includes(combineFn2, nsBase + "#2")); + } + + private static class DisplayDataCombineFn extends Combine.CombineFn { + private final String value; + private static int i; + private final int id; + + DisplayDataCombineFn(String value) { + id = ++i; + this.value = value; + } + + @Override + public String createAccumulator() { + return null; + } + + @Override + public String addInput(String accumulator, String input) { + return null; + } + + @Override + public String mergeAccumulators(Iterable accumulators) { + return null; + } + + @Override + public String extractOutput(String accumulator) { + return null; + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder + .add("uniqueKey" + id, value) + .add("sharedKey", value); + } + } + private static class UserString implements Serializable { private String strValue; diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java index 37ed20415f..6dd838ad87 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/CombineTest.java @@ -17,6 +17,8 @@ package com.google.cloud.dataflow.sdk.transforms; import static com.google.cloud.dataflow.sdk.TestUtils.checkCombineFn; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includes; import static com.google.common.base.Preconditions.checkNotNull; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; @@ -41,6 +43,7 @@ import com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn; import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.Context; import com.google.cloud.dataflow.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.transforms.windowing.AfterPane; import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindows; @@ -678,6 +681,24 @@ public void testCombineGetName() { Combine.perKey(new TestKeyedCombineFn()).withHotKeyFanout(10).getName()); } + @Test + public void testDisplayData() { + UniqueInts combineFn = new UniqueInts() { + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.add("fnMetadata", "foobar"); + } + }; + Combine.Globally combine = Combine.globally(combineFn) + .withFanout(1234); + DisplayData displayData = DisplayData.from(combine); + + assertThat(displayData, hasDisplayItem("combineFn", combineFn.getClass())); + assertThat(displayData, hasDisplayItem("emitDefaultOnEmptyInput", true)); + assertThat(displayData, hasDisplayItem("fanout", 1234)); + assertThat(displayData, includes(combineFn)); + } + //////////////////////////////////////////////////////////////////////////// // Test classes, for different kinds of combining fns. diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/DoFnWithContextTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/DoFnWithContextTest.java index 9f54a6e0ee..081293a383 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/DoFnWithContextTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/DoFnWithContextTest.java @@ -17,8 +17,10 @@ package com.google.cloud.dataflow.sdk.transforms; import static org.hamcrest.CoreMatchers.isA; +import static org.hamcrest.Matchers.empty; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertThat; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; @@ -29,6 +31,7 @@ import com.google.cloud.dataflow.sdk.testing.TestPipeline; import com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn; import com.google.cloud.dataflow.sdk.transforms.Max.MaxIntegerFn; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import org.junit.Rule; import org.junit.Test; @@ -159,7 +162,15 @@ public void testDoFnWithContextUsingAggregators() { } @Test + public void testDefaultPopulateDisplayDataImplementation() { + DoFnWithContext fn = new DoFnWithContext() { + }; + DisplayData displayData = DisplayData.from(fn); + assertThat(displayData.items(), empty()); + } + @Category(RunnableOnService.class) + @Test public void testCreateAggregatorInStartBundleThrows() { TestPipeline p = createTestPipeline(new DoFnWithContext() { @StartBundle diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FilterTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FilterTest.java index 41a4ff2fbe..06f7821de5 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FilterTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/FilterTest.java @@ -16,9 +16,14 @@ package com.google.cloud.dataflow.sdk.transforms; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; + +import static org.hamcrest.MatcherAssert.assertThat; + import com.google.cloud.dataflow.sdk.testing.DataflowAssert; import com.google.cloud.dataflow.sdk.testing.RunnableOnService; import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.values.PCollection; import org.junit.Test; @@ -157,4 +162,19 @@ public void testFilterGreaterThan() { DataflowAssert.that(output).containsInAnyOrder(5, 6, 7); p.run(); } + + @Test + public void testDisplayData() { + ParDo.Bound lessThan = Filter.lessThan(123); + assertThat(DisplayData.from(lessThan), hasDisplayItem("predicate", "x < 123")); + + ParDo.Bound lessThanOrEqual = Filter.lessThanEq(234); + assertThat(DisplayData.from(lessThanOrEqual), hasDisplayItem("predicate", "x ≤ 234")); + + ParDo.Bound greaterThan = Filter.greaterThan(345); + assertThat(DisplayData.from(greaterThan), hasDisplayItem("predicate", "x > 345")); + + ParDo.Bound greaterThanOrEqual = Filter.greaterThanEq(456); + assertThat(DisplayData.from(greaterThanOrEqual), hasDisplayItem("predicate", "x ≥ 456")); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java index 75eb92fcb4..175cc02bb0 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/GroupByKeyTest.java @@ -17,6 +17,9 @@ package com.google.cloud.dataflow.sdk.transforms; import static com.google.cloud.dataflow.sdk.TestUtils.KvMatcher.isKv; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; + +import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder; import static org.hamcrest.core.Is.is; @@ -35,6 +38,7 @@ import com.google.cloud.dataflow.sdk.testing.DataflowAssert; import com.google.cloud.dataflow.sdk.testing.RunnableOnService; import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.InvalidWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFns; @@ -435,4 +439,16 @@ public void processElement(ProcessContext c) throws Exception { public void testGroupByKeyGetName() { Assert.assertEquals("GroupByKey", GroupByKey.create().getName()); } + + @Test + public void testDisplayData() { + GroupByKey groupByKey = GroupByKey.create(); + GroupByKey groupByFewKeys = GroupByKey.create(true); + + DisplayData gbkDisplayData = DisplayData.from(groupByKey); + DisplayData fewKeysDisplayData = DisplayData.from(groupByFewKeys); + + assertThat(gbkDisplayData.items(), empty()); + assertThat(fewKeysDisplayData, hasDisplayItem("fewKeys", true)); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/IntraBundleParallelizationTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/IntraBundleParallelizationTest.java index 76dd67e097..1ed1973819 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/IntraBundleParallelizationTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/IntraBundleParallelizationTest.java @@ -17,6 +17,9 @@ package com.google.cloud.dataflow.sdk.transforms; import static com.google.cloud.dataflow.sdk.testing.SystemNanoTimeSleeper.sleepMillis; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includes; + import static org.hamcrest.Matchers.both; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.greaterThanOrEqualTo; @@ -29,6 +32,7 @@ import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import org.junit.Before; import org.junit.Test; @@ -213,6 +217,29 @@ public void testIntraBundleParallelizationGetName() { IntraBundleParallelization.of(new DelayFn()).withMaxParallelism(1).getName()); } + @Test + public void testDisplayData() { + DoFn fn = new DoFn() { + @Override + public void processElement(ProcessContext c) throws Exception { + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.add("foo", "bar"); + } + }; + + PTransform transform = IntraBundleParallelization + .withMaxParallelism(1234) + .of(fn); + + DisplayData displayData = DisplayData.from(transform); + assertThat(displayData, includes(fn)); + assertThat(displayData, hasDisplayItem("fn", fn.getClass())); + assertThat(displayData, hasDisplayItem("maxParallelism", 1234)); + } + /** * Runs the provided doFn inside of an {@link IntraBundleParallelization} transform. * diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/MaxTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/MaxTest.java index e1ea33bcf2..0dc95b4cbe 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/MaxTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/MaxTest.java @@ -17,8 +17,12 @@ package com.google.cloud.dataflow.sdk.transforms; import static com.google.cloud.dataflow.sdk.TestUtils.checkCombineFn; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; + +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.common.collect.Lists; import org.junit.Test; @@ -63,4 +67,12 @@ public void testMaxDoubleFn() { Lists.newArrayList(1.0, 2.0, 3.0, 4.0), 4.0); } + + @Test + public void testDisplayData() { + Top.Largest comparer = new Top.Largest<>(); + + Combine.Globally max = Max.globally(comparer); + assertThat(DisplayData.from(max), hasDisplayItem("comparer", comparer.getClass())); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/MinTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/MinTest.java index a291537adf..8ef2a73ab6 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/MinTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/MinTest.java @@ -16,9 +16,14 @@ package com.google.cloud.dataflow.sdk.transforms; + import static com.google.cloud.dataflow.sdk.TestUtils.checkCombineFn; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; + +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.common.collect.Lists; import org.junit.Test; @@ -63,4 +68,12 @@ public void testMinDoubleFn() { Lists.newArrayList(1.0, 2.0, 3.0, 4.0), 1.0); } + + @Test + public void testDisplayData() { + Top.Smallest comparer = new Top.Smallest<>(); + + Combine.Globally min = Min.globally(comparer); + assertThat(DisplayData.from(min), hasDisplayItem("comparer", comparer.getClass())); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java index 1ff46e41c2..04baade57c 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/ParDoTest.java @@ -18,9 +18,12 @@ import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasKey; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasType; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includes; import static com.google.cloud.dataflow.sdk.util.SerializableUtils.serializeToByteArray; import static com.google.cloud.dataflow.sdk.util.StringUtils.byteArrayToJsonString; import static com.google.cloud.dataflow.sdk.util.StringUtils.jsonStringToByteArray; +import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.isA; import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder; import static org.hamcrest.collection.IsIterableContainingInOrder.contains; @@ -44,6 +47,7 @@ import com.google.cloud.dataflow.sdk.transforms.ParDo.Bound; import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.transforms.display.DisplayData.Builder; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers; import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.Window; import com.google.cloud.dataflow.sdk.util.IllegalMutationException; @@ -1522,20 +1526,66 @@ public void testMutatingInputCoderDoFnError() throws Exception { } @Test - public void testIncludesDoFnDisplayData() { - Bound parDo = - ParDo.of( - new DoFn() { - @Override - public void processElement(ProcessContext c) {} + public void testDoFnDisplayData() { + DoFn fn = new DoFn() { + @Override + public void processElement(ProcessContext c) { + } - @Override - public void populateDisplayData(Builder builder) { - builder.add("foo", "bar"); - } - }); + @Override + public void populateDisplayData(Builder builder) { + builder.add("doFnMetadata", "bar"); + } + }; + + Bound parDo = ParDo.of(fn); + + DisplayData displayData = DisplayData.from(parDo); + assertThat(displayData, hasDisplayItem(allOf( + hasKey("fn"), + hasType(DisplayData.Type.JAVA_CLASS), + DisplayDataMatchers.hasValue(fn.getClass().getName())))); + + assertThat(displayData, includes(fn)); + } + + @Test + public void testDoFnWithContextDisplayData() { + DoFnWithContext fn = new DoFnWithContext() { + @ProcessElement + public void proccessElement(ProcessContext c) {} + + @Override + public void populateDisplayData(Builder builder) { + builder.add("fnMetadata", "foobar"); + } + }; + + Bound parDo = ParDo.of(fn); + + DisplayData displayData = DisplayData.from(parDo); + assertThat(displayData, includes(fn)); + assertThat(displayData, hasDisplayItem("fn", fn.getClass())); + } + + @Test + public void testWithOutputTagsDisplayData() { + DoFnWithContext fn = new DoFnWithContext() { + @ProcessElement + public void proccessElement(ProcessContext c) {} + + @Override + public void populateDisplayData(Builder builder) { + builder.add("fnMetadata", "foobar"); + } + }; + + ParDo.BoundMulti parDo = ParDo + .withOutputTags(new TupleTag(), TupleTagList.empty()) + .of(fn); DisplayData displayData = DisplayData.from(parDo); - assertThat(displayData, hasDisplayItem(hasKey("foo"))); + assertThat(displayData, includes(fn)); + assertThat(displayData, hasDisplayItem("fn", fn.getClass())); } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/PartitionTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/PartitionTest.java index 5121a0a881..c24b3690c8 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/PartitionTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/PartitionTest.java @@ -16,6 +16,9 @@ package com.google.cloud.dataflow.sdk.transforms; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; + +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -24,6 +27,7 @@ import com.google.cloud.dataflow.sdk.testing.RunnableOnService; import com.google.cloud.dataflow.sdk.testing.TestPipeline; import com.google.cloud.dataflow.sdk.transforms.Partition.PartitionFn; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PCollectionList; @@ -137,4 +141,13 @@ public void testDroppedPartition() { public void testPartitionGetName() { assertEquals("Partition", Partition.of(3, new ModFn()).getName()); } + + @Test + public void testDisplayData() { + Partition partition = Partition.of(123, new IdentityFn()); + DisplayData displayData = DisplayData.from(partition); + + assertThat(displayData, hasDisplayItem("numPartitions", 123)); + assertThat(displayData, hasDisplayItem("partitionFn", IdentityFn.class)); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/SampleTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/SampleTest.java index d860513463..415f6d4aa8 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/SampleTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/SampleTest.java @@ -18,6 +18,9 @@ import static com.google.cloud.dataflow.sdk.TestUtils.LINES; import static com.google.cloud.dataflow.sdk.TestUtils.NO_LINES; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; + +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -27,6 +30,7 @@ import com.google.cloud.dataflow.sdk.testing.DataflowAssert; import com.google.cloud.dataflow.sdk.testing.RunnableOnService; import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.common.base.Joiner; import com.google.common.base.Preconditions; @@ -257,4 +261,15 @@ public void testPickAnyWhenEmpty() { public void testSampleGetName() { assertEquals("Sample.SampleAny", Sample.any(1).getName()); } + + @Test + public void testDisplayData() { + PTransform sampleAny = Sample.any(1234); + DisplayData sampleAnyDisplayData = DisplayData.from(sampleAny); + assertThat(sampleAnyDisplayData, hasDisplayItem("sampleSize", 1234)); + + PTransform samplePerKey = Sample.fixedSizePerKey(2345); + DisplayData perKeyDisplayData = DisplayData.from(samplePerKey); + assertThat(perKeyDisplayData, hasDisplayItem("sampleSize", 2345)); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/TopTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/TopTest.java index ac06bff213..8a0cfe2f16 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/TopTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/TopTest.java @@ -16,6 +16,9 @@ package com.google.cloud.dataflow.sdk.transforms; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; + +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import com.google.cloud.dataflow.sdk.Pipeline; @@ -24,6 +27,7 @@ import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.testing.DataflowAssert; import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.Window; import com.google.cloud.dataflow.sdk.transforms.windowing.Window.Bound; @@ -232,6 +236,16 @@ public void testTopGetNames() { assertEquals("Largest.PerKey", Top.largestPerKey(2).getName()); } + @Test + public void testDisplayData() { + Top.Largest comparer = new Top.Largest(); + Combine.Globally> top = Top.of(1234, comparer); + DisplayData displayData = DisplayData.from(top); + + assertThat(displayData, hasDisplayItem("count", 1234)); + assertThat(displayData, hasDisplayItem("comparer", comparer.getClass())); + } + private static class OrderByLength implements Comparator, Serializable { @Override public int compare(String a, String b) { diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/ClassForDisplayTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/ClassForDisplayTest.java new file mode 100644 index 0000000000..9c6102b7f9 --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/ClassForDisplayTest.java @@ -0,0 +1,64 @@ +/* + * Copyright (C) 2016 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.display; + +import static org.junit.Assert.assertEquals; + +import com.google.cloud.dataflow.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/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchers.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchers.java index 2753aafa7b..aa7b66f0b6 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchers.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchers.java @@ -16,13 +16,19 @@ package com.google.cloud.dataflow.sdk.transforms.display; +import static org.hamcrest.Matchers.allOf; + 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; @@ -42,6 +48,71 @@ public static Matcher 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 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 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 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 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 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 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 hasDisplayItem(String key, Instant value) { + return hasDisplayItem(key, DisplayData.Type.TIMESTAMP, value); + } + + private static Matcher 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}. @@ -68,13 +139,104 @@ protected boolean matchesSafely(DisplayData data, Description mismatchDescriptio Collection 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; } } + /** @see #includes(HasDisplayData, String) */ + public static Matcher includes(HasDisplayData subComponent) { + return includes(subComponent, subComponent.getClass()); + } + + /** @see #includes(HasDisplayData, String) */ + public static Matcher includes( + HasDisplayData subComponent, Class namespace) { + return includes(subComponent, namespace.getName()); + } + + /** + * Create a matcher that matches if the examined {@link DisplayData} contains all display data + * registered from the specified subcomponent and namespace. + */ + public static Matcher includes( + final HasDisplayData subComponent, final String namespace) { + return new CustomTypeSafeMatcher("includes subcomponent") { + @Override + protected boolean matchesSafely(DisplayData displayData) { + DisplayData subComponentData = subComponentData(); + if (subComponentData.items().size() == 0) { + throw new UnsupportedOperationException("subComponent contains no display data; " + + "cannot verify whether it is included"); + } + + DisplayDataComparison comparison = checkSubset(displayData, subComponentData); + return comparison.missingItems.isEmpty(); + } + + @Override + protected void describeMismatchSafely( + DisplayData displayData, Description mismatchDescription) { + DisplayData subComponentDisplayData = subComponentData(); + DisplayDataComparison comparison = checkSubset( + displayData, subComponentDisplayData); + + mismatchDescription + .appendText("did not include:\n") + .appendValue(comparison.missingItems) + .appendText("\nNon-matching items:\n") + .appendValue(comparison.unmatchedItems); + } + + private DisplayData subComponentData() { + return DisplayData.from(new HasDisplayData() { + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.include(subComponent, namespace); + } + }); + } + + private DisplayDataComparison checkSubset( + DisplayData displayData, DisplayData included) { + DisplayDataComparison comparison = new DisplayDataComparison(displayData.items()); + for (Item item : included.items()) { + Item matchedItem = displayData.asMap().get( + DisplayData.Identifier.of(item.getNamespace(), item.getKey())); + + if (matchedItem != null) { + comparison.matched(matchedItem); + } else { + comparison.missing(item); + } + } + + return comparison; + } + + class DisplayDataComparison { + Collection missingItems; + Collection unmatchedItems; + + DisplayDataComparison(Collection 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. @@ -95,4 +257,77 @@ protected String featureValueOf(DisplayData.Item actual) { } }; } + + /** + * Creates a matcher that matches if the examined {@link DisplayData.Item} contains the + * specified namespace. + */ + public static Matcher hasNamespace(Class namespace) { + return hasNamespace(Matchers.>is(namespace)); + } + + /** + * Creates a matcher that matches if the examined {@link DisplayData.Item} contains a namespace + * matching the specified namespace matcher. + */ + public static Matcher hasNamespace(Matcher> namespaceMatcher) { + return new FeatureMatcher>( + namespaceMatcher, "display item with namespace", "namespace") { + @Override + protected Class featureValueOf(DisplayData.Item actual) { + try { + return Class.forName(actual.getNamespace()); + } catch (ClassNotFoundException e) { + return null; + } + } + }; + } + + /** + * Creates a matcher that matches if the examined {@link DisplayData.Item} matches the + * specified type. + */ + public static Matcher hasType(DisplayData.Type type) { + return hasType(Matchers.is(type)); + } + + /** + * Creates a matcher that matches if the examined {@link DisplayData.Item} has a type + * matching the specified type matcher. + */ + public static Matcher hasType(Matcher typeMatcher) { + return new FeatureMatcher( + typeMatcher, "with type", "type") { + @Override + protected DisplayData.Type featureValueOf(DisplayData.Item actual) { + return actual.getType(); + } + }; + } + + /** + * Creates a matcher that matches if the examined {@link DisplayData.Item} has the specified + * value. + */ + + public static Matcher hasValue(Object value) { + return hasValue(Matchers.is(value)); + } + + /** + * Creates a matcher that matches if the examined {@link DisplayData.Item} contains a value + * matching the specified value matcher. + */ + public static Matcher hasValue(Matcher valueMatcher) { + return new FeatureMatcher( + valueMatcher, "with value", "value") { + @Override + protected T featureValueOf(DisplayData.Item actual) { + @SuppressWarnings("unchecked") + T value = (T) actual.getValue(); + return value; + } + }; + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchersTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchersTest.java index 2636cf85c8..4ee76d9c2f 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchersTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataMatchersTest.java @@ -18,11 +18,14 @@ import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasKey; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasNamespace; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasType; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasValue; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includes; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.core.StringStartsWith.startsWith; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.transforms.display.DisplayData.Builder; @@ -45,7 +48,7 @@ public void testHasDisplayItem() { Matcher matcher = hasDisplayItem(); assertFalse(matcher.matches(DisplayData.none())); - assertTrue(matcher.matches(createDisplayDataWithItem("foo", "bar"))); + assertThat(createDisplayDataWithItem("foo", "bar"), matcher); } @Test @@ -58,24 +61,96 @@ public void testHasDisplayItemDescription() { matcher.describeMismatch(DisplayData.none(), mismatchDesc); assertThat(desc.toString(), startsWith("display data with item: ")); - assertThat(mismatchDesc.toString(), containsString("found 0 non-matching items")); + assertThat(mismatchDesc.toString(), containsString("found 0 non-matching item(s)")); } @Test public void testHasKey() { Matcher matcher = hasDisplayItem(hasKey("foo")); - assertTrue(matcher.matches(createDisplayDataWithItem("foo", "bar"))); assertFalse(matcher.matches(createDisplayDataWithItem("fooz", "bar"))); + + assertThat(createDisplayDataWithItem("foo", "bar"), matcher); + } + + @Test + public void testHasType() { + Matcher matcher = hasDisplayItem(hasType(DisplayData.Type.JAVA_CLASS)); + + DisplayData data = DisplayData.from(new PTransform, PCollection>() { + @Override + public void populateDisplayData(Builder builder) { + builder.add("foo", DisplayDataMatchersTest.class); + } + }); + + assertFalse(matcher.matches(createDisplayDataWithItem("fooz", "bar"))); + assertThat(data, matcher); + } + + @Test + public void testHasValue() { + Matcher matcher = hasDisplayItem(hasValue("bar")); + + assertFalse(matcher.matches(createDisplayDataWithItem("foo", "baz"))); + assertThat(createDisplayDataWithItem("foo", "bar"), matcher); + } + + @Test + public void testHasNamespace() { + Matcher matcher = hasDisplayItem(hasNamespace(SampleTransform.class)); + + assertFalse(matcher.matches(DisplayData.from( + new PTransform, PCollection>(){}))); + assertThat(createDisplayDataWithItem("foo", "bar"), matcher); + } + + @Test + public void testIncludes() { + final HasDisplayData subComponent = new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder.add("foo", "bar"); + } + }; + HasDisplayData hasSubcomponent = new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder + .include(subComponent) + .add("foo2", "bar2"); + } + }; + HasDisplayData sameKeyDifferentNamespace = new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder.add("foo", "bar"); + } + }; + Matcher matcher = includes(subComponent); + + assertFalse(matcher.matches(DisplayData.from(sameKeyDifferentNamespace))); + assertThat(DisplayData.from(hasSubcomponent), matcher); + assertThat(DisplayData.from(subComponent), matcher); } + private DisplayData createDisplayDataWithItem(final String key, final String value) { - return DisplayData.from( - new PTransform, PCollection>() { - @Override - public void populateDisplayData(Builder builder) { - builder.add(key, value); - } - }); + return DisplayData.from(new SampleTransform(key, value)); + } + + static class SampleTransform extends PTransform, PCollection> { + private final String key; + private final String value; + + SampleTransform(String key, String value) { + this.key = key; + this.value = value; + } + + @Override + public void populateDisplayData(Builder builder) { + builder.add(key, value); + } } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataTest.java index e8128c7788..625bfa168f 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/display/DisplayDataTest.java @@ -18,6 +18,10 @@ import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasKey; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasNamespace; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasType; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasValue; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includes; import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.everyItem; @@ -28,14 +32,21 @@ import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; +import static org.hamcrest.Matchers.startsWith; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.transforms.display.DisplayData.Builder; import com.google.cloud.dataflow.sdk.transforms.display.DisplayData.Item; import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableMultimap; +import com.google.common.collect.Multimap; import com.google.common.testing.EqualsTester; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; import org.hamcrest.CustomTypeSafeMatcher; import org.hamcrest.FeatureMatcher; @@ -63,6 +74,7 @@ public class DisplayDataTest { @Rule public ExpectedException thrown = ExpectedException.none(); private static final DateTimeFormatter ISO_FORMATTER = ISODateTimeFormat.dateTime(); + private static final ObjectMapper MAPPER = new ObjectMapper(); @Test public void testTypicalUsage() { @@ -82,20 +94,25 @@ public void populateDisplayData(DisplayData.Builder builder) { } }; + PTransform transform = new PTransform, PCollection>() { + final Instant defaultStartTime = new Instant(0); + Instant startTime = defaultStartTime; + @Override public void populateDisplayData(DisplayData.Builder builder) { builder .include(subComponent1) .include(subComponent2) - .add("MinSproggles", 200) - .withLabel("Mimimum Required Sproggles") - .add("LazerOrientation", "NORTH") - .add("TimeBomb", Instant.now().plus(Duration.standardDays(1))) - .add("FilterLogic", subComponent1.getClass()) - .add("ServiceUrl", "google.com/fizzbang") - .withLinkUrl("http://www.google.com/fizzbang"); + .add("minSproggles", 200) + .withLabel("Mimimum Required Sproggles") + .add("fireLazers", true) + .addIfNotDefault("startTime", startTime, defaultStartTime) + .add("timeBomb", Instant.now().plus(Duration.standardDays(1))) + .add("filterLogic", subComponent1.getClass()) + .add("serviceUrl", "google.com/fizzbang") + .withLinkUrl("http://www.google.com/fizzbang"); } }; @@ -126,12 +143,12 @@ public void testCanBuild() { DisplayData.from(new HasDisplayData() { @Override public void populateDisplayData(DisplayData.Builder builder) { - builder.add("Foo", "bar"); + builder.add("foo", "bar"); } }); assertThat(data.items(), hasSize(1)); - assertThat(data, hasDisplayItem(hasKey("Foo"))); + assertThat(data, hasDisplayItem("foo", "bar")); } @Test @@ -147,42 +164,37 @@ public void populateDisplayData(DisplayData.Builder builder) { Map map = data.asMap(); assertEquals(map.size(), 1); - assertThat(data, hasDisplayItem(hasKey("foo"))); + assertThat(data, hasDisplayItem("foo", "bar")); assertEquals(map.values(), data.items()); } @Test public void testItemProperties() { final Instant value = Instant.now(); - DisplayData data = DisplayData.from(new ConcreteComponent(value)); + DisplayData data = DisplayData.from(new HasDisplayData() { + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.add("now", value) + .withLabel("the current instant") + .withLinkUrl("http://time.gov") + .withNamespace(DisplayDataTest.class); + } + }); @SuppressWarnings("unchecked") DisplayData.Item item = (DisplayData.Item) data.items().toArray()[0]; assertThat( item, - allOf( - hasNamespace(Matchers.>is(ConcreteComponent.class)), + Matchers.allOf( + hasNamespace(DisplayDataTest.class), hasKey("now"), - hasType(is(DisplayData.Type.TIMESTAMP)), - hasValue(is(ISO_FORMATTER.print(value))), + hasType(DisplayData.Type.TIMESTAMP), + hasValue(ISO_FORMATTER.print(value)), hasShortValue(nullValue(String.class)), hasLabel(is("the current instant")), hasUrl(is("http://time.gov")))); } - static class ConcreteComponent implements HasDisplayData { - private Instant value; - - ConcreteComponent(Instant value) { - this.value = value; - } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - builder.add("now", value).withLabel("the current instant").withLinkUrl("http://time.gov"); - } - } - @Test public void testUnspecifiedOptionalProperties() { DisplayData data = @@ -199,6 +211,76 @@ public void populateDisplayData(DisplayData.Builder builder) { hasDisplayItem(allOf(hasLabel(nullValue(String.class)), hasUrl(nullValue(String.class))))); } + @Test + public void testAddIfNotDefault() { + DisplayData data = DisplayData.from(new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder + .addIfNotDefault("defaultString", "foo", "foo") + .addIfNotDefault("notDefaultString", "foo", "notFoo") + .addIfNotDefault("defaultInteger", 1, 1) + .addIfNotDefault("notDefaultInteger", 1, 2) + .addIfNotDefault("defaultDouble", 123.4, 123.4) + .addIfNotDefault("notDefaultDouble", 123.4, 234.5) + .addIfNotDefault("defaultBoolean", true, true) + .addIfNotDefault("notDefaultBoolean", true, false) + .addIfNotDefault("defaultInstant", new Instant(0), new Instant(0)) + .addIfNotDefault("notDefaultInstant", new Instant(0), Instant.now()) + .addIfNotDefault("defaultDuration", Duration.ZERO, Duration.ZERO) + .addIfNotDefault("notDefaultDuration", Duration.millis(1234), Duration.ZERO) + .addIfNotDefault("defaultClass", DisplayDataTest.class, DisplayDataTest.class) + .addIfNotDefault("notDefaultClass", DisplayDataTest.class, null); + } + }); + + assertThat(data.items(), hasSize(7)); + assertThat(data.items(), everyItem(hasKey(startsWith("notDefault")))); + } + + @Test + public void testAddIfNotNull() { + DisplayData data = DisplayData.from(new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder + .addIfNotNull("nullString", (String) null) + .addIfNotNull("notNullString", "foo") + .addIfNotNull("nullLong", (Long) null) + .addIfNotNull("notNullLong", 1234L) + .addIfNotNull("nullDouble", (Double) null) + .addIfNotNull("notNullDouble", 123.4) + .addIfNotNull("nullBoolean", (Boolean) null) + .addIfNotNull("notNullBoolean", true) + .addIfNotNull("nullInstant", (Instant) null) + .addIfNotNull("notNullInstant", Instant.now()) + .addIfNotNull("nullDuration", (Duration) null) + .addIfNotNull("notNullDuration", Duration.ZERO) + .addIfNotNull("nullClass", (Class) null) + .addIfNotNull("notNullClass", DisplayDataTest.class); + } + }); + + assertThat(data.items(), hasSize(7)); + assertThat(data.items(), everyItem(hasKey(startsWith("notNull")))); + } + + @Test + public void testModifyingConditionalItemIsSafe() { + HasDisplayData component = new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder.addIfNotNull("nullItem", (Class) null) + .withLinkUrl("http://abc") + .withNamespace(DisplayDataTest.class) + .withLabel("Null item shoudl be safe"); + } + }; + + DisplayData.from(component); // should not throw + } + + @Test public void testIncludes() { final HasDisplayData subComponent = @@ -218,22 +300,107 @@ public void populateDisplayData(DisplayData.Builder builder) { } }); - assertThat( - data, - hasDisplayItem( - allOf( - hasKey("foo"), - hasNamespace(Matchers.>is(subComponent.getClass()))))); + assertThat(data, includes(subComponent)); + } + + @Test + public void testIncludesNamespaceOverride() { + final HasDisplayData subComponent = new HasDisplayData() { + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.add("foo", "bar"); + } + }; + + final HasDisplayData namespaceOverride = new HasDisplayData(){ + @Override + public void populateDisplayData(Builder builder) { + } + }; + + DisplayData data = DisplayData.from(new HasDisplayData() { + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.include(subComponent, namespaceOverride.getClass()); + } + }); + + assertThat(data, includes(subComponent, namespaceOverride.getClass())); + } + + @Test + public void testNullNamespaceOverride() { + thrown.expect(NullPointerException.class); + + DisplayData.from(new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder.add("foo", "bar") + .withNamespace((Class) null); + } + }); } @Test public void testIdentifierEquality() { new EqualsTester() .addEqualityGroup( - 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")) + 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")) + .testEquals(); + } + + @Test + public void testItemEquality() { + HasDisplayData component1 = new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder.add("foo", "bar"); + } + }; + HasDisplayData component2 = new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder.add("foo", "bar"); + } + }; + + DisplayData component1DisplayData1 = DisplayData.from(component1); + DisplayData component1DisplayData2 = DisplayData.from(component1); + DisplayData component2DisplayData = DisplayData.from(component2); + + new EqualsTester() + .addEqualityGroup( + component1DisplayData1.items().toArray()[0], + component1DisplayData2.items().toArray()[0]) + .addEqualityGroup(component2DisplayData.items().toArray()[0]) + .testEquals(); + } + + @Test + public void testDisplayDataEquality() { + HasDisplayData component1 = new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder.add("foo", "bar"); + } + }; + HasDisplayData component2 = new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder.add("foo", "bar"); + } + }; + + DisplayData component1DisplayData1 = DisplayData.from(component1); + DisplayData component1DisplayData2 = DisplayData.from(component1); + DisplayData component2DisplayData = DisplayData.from(component2); + + new EqualsTester() + .addEqualityGroup(component1DisplayData1, component1DisplayData2) + .addEqualityGroup(component2DisplayData) .testEquals(); } @@ -297,6 +464,22 @@ public void populateDisplayData(DisplayData.Builder builder) { }); } + @Test + public void testDuplicateKeyWithNamespaceOverrideDoesntThrow() { + DisplayData displayData = DisplayData.from( + new HasDisplayData() { + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder + .add("foo", "bar") + .add("foo", "baz") + .withNamespace(DisplayDataTest.class); + } + }); + + assertThat(displayData.items(), hasSize(2)); + } + @Test public void testToString() { HasDisplayData component = new HasDisplayData() { @@ -376,6 +559,7 @@ public int hashCode() { } @Override + @SuppressWarnings("EqualsWhichDoesntCheckParameterClass") public boolean equals(Object obj) { return true; } @@ -403,7 +587,9 @@ public void populateDisplayData(DisplayData.Builder builder) { .add("string", "foobar") .add("integer", 123) .add("float", 3.14) + .add("boolean", true) .add("java_class", DisplayDataTest.class) + .add("java_class2", ClassForDisplay.of(DisplayDataTest.class)) .add("timestamp", Instant.now()) .add("duration", Duration.standardHours(1)); } @@ -411,18 +597,119 @@ public void populateDisplayData(DisplayData.Builder builder) { Collection items = data.items(); assertThat( - items, hasItem(allOf(hasKey("string"), hasType(is(DisplayData.Type.STRING))))); + items, hasItem(allOf(hasKey("string"), hasType(DisplayData.Type.STRING)))); + assertThat( + items, hasItem(allOf(hasKey("integer"), hasType(DisplayData.Type.INTEGER)))); + assertThat(items, hasItem(allOf(hasKey("float"), hasType(DisplayData.Type.FLOAT)))); + assertThat(items, hasItem(allOf(hasKey("boolean"), hasType(DisplayData.Type.BOOLEAN)))); assertThat( - items, hasItem(allOf(hasKey("integer"), hasType(is(DisplayData.Type.INTEGER))))); - assertThat(items, hasItem(allOf(hasKey("float"), hasType(is(DisplayData.Type.FLOAT))))); + items, + hasItem(allOf(hasKey("java_class"), hasType(DisplayData.Type.JAVA_CLASS)))); assertThat( items, - hasItem(allOf(hasKey("java_class"), hasType(is(DisplayData.Type.JAVA_CLASS))))); + hasItem(allOf(hasKey("java_class2"), hasType(DisplayData.Type.JAVA_CLASS)))); assertThat( items, - hasItem(allOf(hasKey("timestamp"), hasType(is(DisplayData.Type.TIMESTAMP))))); + hasItem(allOf(hasKey("timestamp"), hasType(DisplayData.Type.TIMESTAMP)))); assertThat( - items, hasItem(allOf(hasKey("duration"), hasType(is(DisplayData.Type.DURATION))))); + items, hasItem(allOf(hasKey("duration"), hasType(DisplayData.Type.DURATION)))); + } + + @Test + public void testExplicitItemType() { + DisplayData data = DisplayData.from(new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder + .add("integer", DisplayData.Type.INTEGER, 1234L) + .add("string", DisplayData.Type.STRING, "foobar"); + } + }); + + assertThat(data, hasDisplayItem("integer", 1234L)); + assertThat(data, hasDisplayItem("string", "foobar")); + } + + @Test + public void testFormatIncompatibleTypes() { + Map invalidPairs = ImmutableMap.builder() + .put(DisplayData.Type.STRING, 1234) + .put(DisplayData.Type.INTEGER, "string value") + .put(DisplayData.Type.FLOAT, "string value") + .put(DisplayData.Type.BOOLEAN, "string value") + .put(DisplayData.Type.TIMESTAMP, "string value") + .put(DisplayData.Type.DURATION, "string value") + .put(DisplayData.Type.JAVA_CLASS, "string value") + .build(); + + for (Map.Entry pair : invalidPairs.entrySet()) { + try { + DisplayData.Type type = pair.getKey(); + Object invalidValue = pair.getValue(); + + type.format(invalidValue); + fail(String.format( + "Expected exception not thrown for invalid %s value: %s", type, invalidValue)); + } catch (ClassCastException e) { + // Expected + } + } + } + + @Test + public void testFormatCompatibleTypes() { + Multimap validPairs = ImmutableMultimap + .builder() + .put(DisplayData.Type.INTEGER, 1234) + .put(DisplayData.Type.INTEGER, 1234L) + .put(DisplayData.Type.FLOAT, 123.4f) + .put(DisplayData.Type.FLOAT, 123.4) + .put(DisplayData.Type.FLOAT, 1234) + .put(DisplayData.Type.FLOAT, 1234L) + .build(); + + for (Map.Entry pair : validPairs.entries()) { + DisplayData.Type type = pair.getKey(); + Object value = pair.getValue(); + + try { + type.format(value); + } catch (ClassCastException e) { + fail(String.format("Failed to format %s for DisplayData.%s", + value.getClass().getSimpleName(), type)); + } + } + } + + @Test + public void testInvalidExplicitItemType() { + HasDisplayData component = new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder.add("integer", DisplayData.Type.INTEGER, "foobar"); + } + }; + + thrown.expect(ClassCastException.class); + DisplayData.from(component); + } + + @Test + public void testKnownTypeInference() { + assertEquals(DisplayData.Type.INTEGER, DisplayData.inferType(1234)); + assertEquals(DisplayData.Type.INTEGER, DisplayData.inferType(1234L)); + assertEquals(DisplayData.Type.FLOAT, DisplayData.inferType(12.3)); + assertEquals(DisplayData.Type.FLOAT, DisplayData.inferType(12.3f)); + 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")); + + assertEquals(null, DisplayData.inferType(null)); + assertEquals(null, DisplayData.inferType(new Object() {})); } @Test @@ -437,6 +724,7 @@ public void populateDisplayData(DisplayData.Builder builder) { .add("string", "foobar") .add("integer", 123) .add("float", 3.14) + .add("boolean", true) .add("java_class", DisplayDataTest.class) .add("timestamp", now) .add("duration", oneHour); @@ -444,17 +732,13 @@ public void populateDisplayData(DisplayData.Builder builder) { }; DisplayData data = DisplayData.from(component); - Collection items = data.items(); - assertThat(items, hasItem(allOf(hasKey("string"), hasValue(is("foobar"))))); - assertThat(items, hasItem(allOf(hasKey("integer"), hasValue(is("123"))))); - assertThat(items, hasItem(allOf(hasKey("float"), hasValue(is("3.14"))))); - assertThat(items, hasItem(allOf(hasKey("java_class"), - hasValue(is(DisplayDataTest.class.getName())), - hasShortValue(is(DisplayDataTest.class.getSimpleName()))))); - assertThat(items, hasItem(allOf(hasKey("timestamp"), - hasValue(is(ISO_FORMATTER.print(now)))))); - assertThat(items, hasItem(allOf(hasKey("duration"), - hasValue(is(Long.toString(oneHour.getMillis())))))); + assertThat(data, hasDisplayItem("string", "foobar")); + assertThat(data, hasDisplayItem("integer", 123)); + assertThat(data, hasDisplayItem("float", 3.14)); + assertThat(data, hasDisplayItem("boolean", true)); + assertThat(data, hasDisplayItem("java_class", DisplayDataTest.class)); + assertThat(data, hasDisplayItem("timestamp", now)); + assertThat(data, hasDisplayItem("duration", oneHour)); } @Test @@ -483,7 +767,7 @@ public void populateDisplayData(DisplayData.Builder builder) { hasItem( allOf( hasKey("alpha"), - hasNamespace(Matchers.>is(component.getClass()))))); + hasNamespace(component.getClass())))); } @Test @@ -504,6 +788,23 @@ public void populateDisplayData(Builder builder) { }); } + @Test + public void testIncludeNullNamespace() { + final HasDisplayData subComponent = new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + } + }; + + thrown.expect(NullPointerException.class); + DisplayData.from(new HasDisplayData() { + @Override + public void populateDisplayData(Builder builder) { + builder.include(subComponent, (ClassForDisplay) null); + } + }); + } + @Test public void testNullKey() { thrown.expect(NullPointerException.class); @@ -560,36 +861,94 @@ public void testAcceptsNullOptionalValues() { @Override public void populateDisplayData(Builder builder) { builder.add("key", "value") - .withLabel(null) - .withLinkUrl(null); + .withLabel(null) + .withLinkUrl(null); } }); // Should not throw } - private static Matcher hasNamespace(Matcher> nsMatcher) { - return new FeatureMatcher>( - nsMatcher, "display item with namespace", "namespace") { + @Test + public void testJsonSerialization() throws IOException { + final String stringValue = "foobar"; + final int intValue = 1234; + final double floatValue = 123.4; + final boolean boolValue = true; + final int durationMillis = 1234; + + HasDisplayData component = new HasDisplayData() { @Override - protected Class featureValueOf(DisplayData.Item actual) { - try { - return Class.forName(actual.getNamespace()); - } catch (ClassNotFoundException e) { - return null; - } + public void populateDisplayData(Builder builder) { + builder + .add("string", stringValue) + .add("long", intValue) + .add("double", floatValue) + .add("boolean", boolValue) + .add("instant", new Instant(0)) + .add("duration", Duration.millis(durationMillis)) + .add("class", DisplayDataTest.class) + .withLinkUrl("http://abc") + .withLabel("baz") + ; } }; + DisplayData data = DisplayData.from(component); + + JsonNode json = MAPPER.readTree(MAPPER.writeValueAsBytes(data)); + assertThat(json, hasExpectedJson(component, "STRING", "string", quoted(stringValue))); + assertThat(json, hasExpectedJson(component, "INTEGER", "long", intValue)); + assertThat(json, hasExpectedJson(component, "FLOAT", "double", floatValue)); + assertThat(json, hasExpectedJson(component, "BOOLEAN", "boolean", boolValue)); + assertThat(json, hasExpectedJson(component, "DURATION", "duration", durationMillis)); + assertThat(json, hasExpectedJson( + component, "TIMESTAMP", "instant", quoted("1970-01-01T00:00:00.000Z"))); + assertThat(json, hasExpectedJson( + component, "JAVA_CLASS", "class", quoted(DisplayDataTest.class.getName()), + quoted("DisplayDataTest"), "baz", "http://abc")); } - private static Matcher hasType(Matcher typeMatcher) { - return new FeatureMatcher( - typeMatcher, "display item with type", "type") { - @Override - protected DisplayData.Type featureValueOf(DisplayData.Item actual) { - return actual.getType(); - } - }; + private String quoted(Object obj) { + return String.format("\"%s\"", obj); + } + + private Matcher> hasExpectedJson( + HasDisplayData component, String type, String key, Object value) + throws IOException { + return hasExpectedJson(component, type, key, value, null, null, null); + } + + private Matcher> hasExpectedJson( + HasDisplayData component, + String type, + String key, + Object value, + Object shortValue, + String label, + String linkUrl) throws IOException { + Class nsClass = component.getClass(); + + StringBuilder builder = new StringBuilder(); + builder.append("{"); + builder.append(String.format("\"namespace\":\"%s\",", nsClass.getName())); + builder.append(String.format("\"type\":\"%s\",", type)); + builder.append(String.format("\"key\":\"%s\",", key)); + builder.append(String.format("\"value\":%s", value)); + + if (shortValue != null) { + builder.append(String.format(",\"shortValue\":%s", shortValue)); + } + if (label != null) { + builder.append(String.format(",\"label\":\"%s\"", label)); + } + if (linkUrl != null) { + builder.append(String.format(",\"linkUrl\":\"%s\"", linkUrl)); + } + + builder.append("}"); + + JsonNode jsonNode = MAPPER.readTree(builder.toString()); + return hasItem(jsonNode); } private static Matcher hasLabel(Matcher labelMatcher) { @@ -612,22 +971,14 @@ protected String featureValueOf(DisplayData.Item actual) { }; } - private static Matcher hasValue(Matcher valueMatcher) { - return new FeatureMatcher( - valueMatcher, "display item with value", "value") { - @Override - protected String featureValueOf(DisplayData.Item actual) { - return actual.getValue(); - } - }; - } - - private static Matcher hasShortValue(Matcher valueStringMatcher) { - return new FeatureMatcher( + private static Matcher hasShortValue(Matcher valueStringMatcher) { + return new FeatureMatcher( valueStringMatcher, "display item with short value", "short value") { @Override - protected String featureValueOf(DisplayData.Item actual) { - return actual.getShortValue(); + protected T featureValueOf(DisplayData.Item actual) { + @SuppressWarnings("unchecked") + T shortValue = (T) actual.getShortValue(); + return shortValue; } }; } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterAllTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterAllTest.java index 06f0c3f759..3e8ab333e1 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterAllTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterAllTest.java @@ -148,4 +148,9 @@ public void testContinuation() throws Exception { afterAll.getContinuationTrigger()); } + @Test + public void testToString() { + Trigger trigger = AfterAll.of(StubTrigger.named("t1"), StubTrigger.named("t2")); + assertEquals("AfterAll.of(t1, t2)", trigger.toString()); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterEachTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterEachTest.java index 5d6632d1c4..9c9e79ef55 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterEachTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterEachTest.java @@ -119,4 +119,14 @@ public void testContinuation() throws Exception { trigger1.getContinuationTrigger(), trigger2.getContinuationTrigger())), afterEach.getContinuationTrigger()); } + + @Test + public void testToString() { + Trigger trigger = AfterEach.inOrder( + StubTrigger.named("t1"), + StubTrigger.named("t2"), + StubTrigger.named("t3")); + + assertEquals("AfterEach.inOrder(t1, t2, t3)", trigger.toString()); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterFirstTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterFirstTest.java index 135638c363..fc0a5cd5ae 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterFirstTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterFirstTest.java @@ -172,4 +172,10 @@ public void testContinuation() throws Exception { AfterFirst.of(trigger1.getContinuationTrigger(), trigger2.getContinuationTrigger()), afterFirst.getContinuationTrigger()); } + + @Test + public void testToString() { + Trigger trigger = AfterFirst.of(StubTrigger.named("t1"), StubTrigger.named("t2")); + assertEquals("AfterFirst.of(t1, t2)", trigger.toString()); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterPaneTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterPaneTest.java index 9139bc5884..7274a997d7 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterPaneTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterPaneTest.java @@ -123,4 +123,10 @@ public void testContinuation() throws Exception { AfterPane.elementCountAtLeast(1), AfterPane.elementCountAtLeast(100).getContinuationTrigger().getContinuationTrigger()); } + + @Test + public void testToString() { + Trigger trigger = AfterPane.elementCountAtLeast(5); + assertEquals("AfterPane.elementCountAtLeast(5)", trigger.toString()); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterProcessingTimeTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterProcessingTimeTest.java index a3bb3c372e..959802fd0d 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterProcessingTimeTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterProcessingTimeTest.java @@ -154,4 +154,35 @@ public void testCompatibilityIdentical() throws Exception { .plusDelayOf(Duration.standardMinutes(1L)); assertTrue(t1.isCompatible(t2)); } + + @Test + public void testToString() { + Trigger trigger = AfterProcessingTime.pastFirstElementInPane(); + assertEquals("AfterProcessingTime.pastFirstElementInPane()", trigger.toString()); + } + + @Test + public void testWithDelayToString() { + Trigger trigger = AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(Duration.standardMinutes(5)); + + assertEquals("AfterProcessingTime.pastFirstElementInPane().plusDelayOf(5 minutes)", + trigger.toString()); + } + + @Test + public void testBuiltUpToString() { + Trigger trigger = AfterWatermark.pastEndOfWindow() + .withLateFirings(AfterProcessingTime + .pastFirstElementInPane() + .plusDelayOf(Duration.standardMinutes(10))) + .buildTrigger(); + + String expected = "AfterWatermark.pastEndOfWindow()" + + ".withLateFirings(AfterProcessingTime" + + ".pastFirstElementInPane()" + + ".plusDelayOf(10 minutes))"; + + assertEquals(expected, trigger.toString()); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterWatermarkTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterWatermarkTest.java index fb610aadd1..e1a14c7cdf 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterWatermarkTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/AfterWatermarkTest.java @@ -16,6 +16,7 @@ package com.google.cloud.dataflow.sdk.transforms.windowing; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.doNothing; @@ -335,4 +336,30 @@ public void testEarlyAndLateOnMergeRewinds() throws Exception { tester.advanceInputWatermark(new Instant(15)); assertTrue(tester.shouldFire(mergedWindow)); } + + @Test + public void testFromEndOfWindowToString() { + Trigger trigger = AfterWatermark.pastEndOfWindow(); + assertEquals("AfterWatermark.pastEndOfWindow()", trigger.toString()); + } + + @Test + public void testLateFiringsToString() { + Trigger trigger = AfterWatermark.pastEndOfWindow() + .withLateFirings(StubTrigger.named("t1")) + .buildTrigger(); + + assertEquals("AfterWatermark.pastEndOfWindow().withLateFirings(t1)", trigger.toString()); + } + + @Test + public void testEarlyAndLateFiringsToString() { + Trigger trigger = AfterWatermark.pastEndOfWindow() + .withEarlyFirings(StubTrigger.named("t1")) + .withLateFirings(StubTrigger.named("t2")) + .buildTrigger(); + + assertEquals("AfterWatermark.pastEndOfWindow().withEarlyFirings(t1).withLateFirings(t2)", + trigger.toString()); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindowsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindowsTest.java index 512fcbc292..8b1fbcec85 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindowsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/CalendarWindowsTest.java @@ -18,8 +18,13 @@ import static com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils.runWindowFn; import static com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils.set; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; + +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; + import org.joda.time.DateTime; import org.joda.time.DateTimeConstants; import org.joda.time.DateTimeZone; @@ -257,4 +262,31 @@ public void testTimeZone() throws Exception { CalendarWindows.days(1).withTimeZone(timeZone), timestamps)); } + + @Test + public void testDisplayData() { + DateTimeZone timeZone = DateTimeZone.forID("America/Los_Angeles"); + Instant jan1 = new DateTime(1990, 1, 1, 0, 0, timeZone).toInstant(); + + CalendarWindows.DaysWindows daysWindow = CalendarWindows.days(5) + .withStartingDay(1990, 1, 1) + .withTimeZone(timeZone); + DisplayData daysDisplayData = DisplayData.from(daysWindow); + assertThat(daysDisplayData, hasDisplayItem("numDays", 5)); + assertThat(daysDisplayData, hasDisplayItem("startDate", jan1)); + + CalendarWindows.MonthsWindows monthsWindow = CalendarWindows.months(2) + .withStartingMonth(1990, 1) + .withTimeZone(timeZone); + DisplayData monthsDisplayData = DisplayData.from(monthsWindow); + assertThat(monthsDisplayData, hasDisplayItem("numMonths", 2)); + assertThat(monthsDisplayData, hasDisplayItem("startDate", jan1)); + + CalendarWindows.YearsWindows yearsWindow = CalendarWindows.years(4) + .withStartingYear(1990) + .withTimeZone(timeZone); + DisplayData yearsDisplayData = DisplayData.from(yearsWindow); + assertThat(yearsDisplayData, hasDisplayItem("numYears", 4)); + assertThat(yearsDisplayData, hasDisplayItem("startDate", jan1)); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindowsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindowsTest.java index 935f22e016..e143597c52 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindowsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/FixedWindowsTest.java @@ -18,6 +18,8 @@ import static com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils.runWindowFn; import static com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils.set; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; + import static org.hamcrest.CoreMatchers.containsString; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -26,6 +28,7 @@ import static org.junit.Assert.fail; import com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import org.joda.time.Duration; import org.joda.time.Instant; @@ -121,4 +124,16 @@ public void testValidOutputTimes() throws Exception { FixedWindows.of(new Duration(500)), timestamp); } } + + @Test + public void testDisplayData() { + Duration offset = Duration.standardSeconds(1234); + Duration size = Duration.standardSeconds(2345); + + FixedWindows fixedWindows = FixedWindows.of(size).withOffset(offset); + DisplayData displayData = DisplayData.from(fixedWindows); + + assertThat(displayData, hasDisplayItem("size", size)); + assertThat(displayData, hasDisplayItem("offset", offset)); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/OrFinallyTriggerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/OrFinallyTriggerTest.java index a416e60425..6a734fdb54 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/OrFinallyTriggerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/OrFinallyTriggerTest.java @@ -206,4 +206,10 @@ public void testContinuation() throws Exception { triggerB.getContinuationTrigger().orFinally(triggerA.getContinuationTrigger())), bOrFinallyA.getContinuationTrigger()); } + + @Test + public void testToString() { + Trigger trigger = StubTrigger.named("t1").orFinally(StubTrigger.named("t2")); + assertEquals("t1.orFinally(t2)", trigger.toString()); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/RepeatedlyTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/RepeatedlyTest.java index e14c15c4e2..0a0f99cbf9 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/RepeatedlyTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/RepeatedlyTest.java @@ -208,4 +208,17 @@ public void testRepeatedlyProcessingTime() throws Exception { assertFalse(tester.shouldFire(window)); } + + @Test + public void testToString() { + Trigger trigger = Repeatedly.forever(new StubTrigger() { + @Override + public String toString() { + return "innerTrigger"; + } + }); + + assertEquals("Repeatedly.forever(innerTrigger)", trigger.toString()); + } + } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SessionsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SessionsTest.java index 91049cd9d9..4649a6bcc7 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SessionsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SessionsTest.java @@ -18,6 +18,7 @@ import static com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils.runWindowFn; import static com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils.set; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.containsString; @@ -26,6 +27,7 @@ import static org.junit.Assert.assertTrue; import com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.common.collect.ImmutableList; import org.joda.time.Duration; @@ -153,4 +155,12 @@ public void testValidOutputAtEndTimes() throws Exception { (List) ImmutableList.of(1L, 3L), (List) ImmutableList.of(0L, 5L, 10L, 15L, 20L))); } + + @Test + public void testDisplayData() { + Duration gapDuration = Duration.standardMinutes(234); + Sessions session = Sessions.withGapDuration(gapDuration); + assertThat(DisplayData.from(session), + hasDisplayItem("gapDuration", gapDuration)); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindowsTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindowsTest.java index 33c4b8b816..2aae07804c 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindowsTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/SlidingWindowsTest.java @@ -18,11 +18,15 @@ import static com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils.runWindowFn; import static com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils.set; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; + import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import com.google.cloud.dataflow.sdk.testing.WindowFnTestUtils; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import org.joda.time.Duration; import org.joda.time.Instant; @@ -190,4 +194,21 @@ public void testOutputTimesNonInterference() throws Exception { SlidingWindows.of(new Duration(1000)).every(new Duration(500)), timestamp); } } + + @Test + public void testDisplayData() { + Duration windowSize = Duration.standardSeconds(1234); + Duration offset = Duration.standardSeconds(2345); + Duration period = Duration.standardSeconds(3456); + + SlidingWindows slidingWindowFn = SlidingWindows + .of(windowSize) + .every(period) + .withOffset(offset); + + DisplayData displayData = DisplayData.from(slidingWindowFn); + assertThat(displayData, hasDisplayItem("size", windowSize)); + assertThat(displayData, hasDisplayItem("period", period)); + assertThat(displayData, hasDisplayItem("offset", offset)); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/StubTrigger.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/StubTrigger.java new file mode 100644 index 0000000000..a9be77b37a --- /dev/null +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/StubTrigger.java @@ -0,0 +1,70 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.sdk.transforms.windowing; + +import com.google.api.client.util.Lists; + +import org.joda.time.Instant; + +import java.util.List; + +/** + * No-op {@link OnceTrigger} implementation for testing. + */ +abstract class StubTrigger extends Trigger.OnceTrigger { + /** + * Create a stub {@link Trigger} instance which returns the specified name on {@link #toString()}. + */ + static StubTrigger named(final String name) { + return new StubTrigger() { + @Override + public String toString() { + return name; + } + }; + } + + protected StubTrigger() { + super(Lists.>newArrayList()); + } + + @Override + protected void onOnlyFiring(TriggerContext context) throws Exception { + } + + @Override + public void onElement(OnElementContext c) throws Exception { + } + + @Override + public void onMerge(OnMergeContext c) throws Exception { + } + + @Override + public boolean shouldFire(TriggerContext context) throws Exception { + return false; + } + + @Override + protected Trigger getContinuationTrigger(List> continuationTriggers) { + return null; + } + + @Override + public Instant getWatermarkThatGuaranteesFiring(BoundedWindow window) { + return null; + } +} diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/TriggerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/TriggerTest.java index ddff33fbda..c6ee3517cc 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/TriggerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/TriggerTest.java @@ -36,7 +36,7 @@ public class TriggerTest { @Test public void testTriggerToString() throws Exception { assertEquals("AfterWatermark.pastEndOfWindow()", AfterWatermark.pastEndOfWindow().toString()); - assertEquals("Repeatedly(AfterWatermark.pastEndOfWindow())", + assertEquals("Repeatedly.forever(AfterWatermark.pastEndOfWindow())", Repeatedly.forever(AfterWatermark.pastEndOfWindow()).toString()); } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowTest.java index 72f2b4c12d..970b815ccc 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/transforms/windowing/WindowTest.java @@ -16,7 +16,13 @@ package com.google.cloud.dataflow.sdk.transforms.windowing; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.hasKey; +import static com.google.cloud.dataflow.sdk.transforms.display.DisplayDataMatchers.includes; + import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.isOneOf; +import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; @@ -32,6 +38,7 @@ import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.GroupByKey; import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; import com.google.cloud.dataflow.sdk.util.WindowingStrategy; import com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode; import com.google.cloud.dataflow.sdk.values.KV; @@ -223,4 +230,56 @@ public void processElement(ProcessContext c) throws Exception { pipeline.run(); } + + @Test + public void testDisplayData() { + FixedWindows windowFn = FixedWindows.of(Duration.standardHours(5)); + AfterWatermark.FromEndOfWindow triggerBuilder = AfterWatermark.pastEndOfWindow(); + Duration allowedLateness = Duration.standardMinutes(10); + Window.ClosingBehavior closingBehavior = Window.ClosingBehavior.FIRE_IF_NON_EMPTY; + OutputTimeFn outputTimeFn = OutputTimeFns.outputAtEndOfWindow(); + + Window.Bound window = Window + .into(windowFn) + .triggering(triggerBuilder) + .accumulatingFiredPanes() + .withAllowedLateness(allowedLateness, closingBehavior) + .withOutputTimeFn(outputTimeFn); + + DisplayData displayData = DisplayData.from(window); + + assertThat(displayData, hasDisplayItem("windowFn", windowFn.getClass())); + assertThat(displayData, includes(windowFn)); + + assertThat(displayData, hasDisplayItem("trigger", triggerBuilder.toString())); + assertThat(displayData, + hasDisplayItem("accumulationMode", AccumulationMode.ACCUMULATING_FIRED_PANES.toString())); + assertThat(displayData, + hasDisplayItem("allowedLateness", allowedLateness)); + assertThat(displayData, hasDisplayItem("closingBehavior", closingBehavior.toString())); + assertThat(displayData, hasDisplayItem("outputTimeFn", outputTimeFn.getClass())); + } + + @Test + public void testDisplayDataExcludesUnspecifiedProperties() { + Window.Bound window = Window.into(new GlobalWindows()); + + DisplayData displayData = DisplayData.from(window); + assertThat(displayData, not(hasDisplayItem(hasKey(isOneOf( + "trigger", + "outputTimeFn", + "accumulationMode", + "allowedLateness", + "closingBehavior"))))); + + } + + @Test + public void testDisplayDataExcludesDefaultTrigger() { + Window.Bound window = Window.into(new GlobalWindows()) + .triggering(DefaultTrigger.of()); + + DisplayData data = DisplayData.from(window); + assertThat(data, not(hasDisplayItem(hasKey("trigger")))); + } } diff --git a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/ReshuffleTriggerTest.java b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/ReshuffleTriggerTest.java index 4b3a77ce61..eb3262bcac 100644 --- a/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/ReshuffleTriggerTest.java +++ b/sdk/src/test/java/com/google/cloud/dataflow/sdk/util/ReshuffleTriggerTest.java @@ -15,12 +15,14 @@ */ package com.google.cloud.dataflow.sdk.util; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; +import com.google.cloud.dataflow.sdk.transforms.windowing.Trigger; import org.joda.time.Duration; import org.joda.time.Instant; @@ -34,7 +36,7 @@ @RunWith(JUnit4.class) public class ReshuffleTriggerTest { - /** Public so that other tests can instantiate ReshufleTrigger. */ + /** Public so that other tests can instantiate {@link ReshuffleTrigger}. */ public static ReshuffleTrigger forTest() { return new ReshuffleTrigger<>(); } @@ -55,4 +57,10 @@ public void testOnTimer() throws Exception { tester.fireIfShouldFire(arbitraryWindow); assertFalse(tester.isMarkedFinished(arbitraryWindow)); } + + @Test + public void testToString() { + Trigger trigger = new ReshuffleTrigger<>(); + assertEquals("ReshuffleTrigger()", trigger.toString()); + } } diff --git a/sdk/src/test/java8/com/google/cloud/dataflow/sdk/transforms/CombineJava8Test.java b/sdk/src/test/java8/com/google/cloud/dataflow/sdk/transforms/CombineJava8Test.java index b569e49c95..49476172e8 100644 --- a/sdk/src/test/java8/com/google/cloud/dataflow/sdk/transforms/CombineJava8Test.java +++ b/sdk/src/test/java8/com/google/cloud/dataflow/sdk/transforms/CombineJava8Test.java @@ -16,12 +16,21 @@ package com.google.cloud.dataflow.sdk.transforms; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.not; + import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.testing.DataflowAssert; import com.google.cloud.dataflow.sdk.testing.TestPipeline; +import com.google.cloud.dataflow.sdk.transforms.display.DisplayData; +import com.google.cloud.dataflow.sdk.util.SerializableUtils; import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.common.collect.Iterables; + +import org.junit.Assume; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -130,4 +139,37 @@ public void testCombinePerKeyInstanceMethodReference() { KV.of("c", 4)); pipeline.run(); } + + /** + * Tests that we can serialize {@link Combine.CombineFn CombineFns} constructed from a lambda. + * Lambdas can be problematic because the {@link Class} object is synthetic and cannot be + * deserialized. + */ + @Test + public void testLambdaSerialization() { + SerializableFunction, Object> combiner = xs -> Iterables.getFirst(xs, 0); + + boolean lambdaClassSerializationThrows; + try { + SerializableUtils.clone(combiner.getClass()); + lambdaClassSerializationThrows = false; + } catch (IllegalArgumentException e) { + // Expected + lambdaClassSerializationThrows = true; + } + Assume.assumeTrue("Expected lambda class serialization to fail. " + + "If it's fixed, we can remove special behavior in Combine.", + lambdaClassSerializationThrows); + + + Combine.Globally combine = Combine.globally(combiner); + SerializableUtils.clone(combine); // should not throw. + } + + @Test + public void testLambdaDisplayData() { + Combine.Globally combine = Combine.globally(xs -> Iterables.getFirst(xs, 0)); + DisplayData displayData = DisplayData.from(combine); + assertThat(displayData.items(), not(empty())); + } } diff --git a/sdk/src/test/java8/com/google/cloud/dataflow/sdk/transforms/display/ClassForDisplayJava8Test.java b/sdk/src/test/java8/com/google/cloud/dataflow/sdk/transforms/display/ClassForDisplayJava8Test.java new file mode 100644 index 0000000000..de6bd7df87 --- /dev/null +++ b/sdk/src/test/java8/com/google/cloud/dataflow/sdk/transforms/display/ClassForDisplayJava8Test.java @@ -0,0 +1,46 @@ +/* + * 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 com.google.cloud.dataflow.sdk.transforms.display; + +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.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); + } +}