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

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -72,9 +72,9 @@
import com.google.api.services.dataflow.model.WorkerPool;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;

import org.hamcrest.Matcher;
import org.hamcrest.Matchers;
import org.junit.Assert;
import org.junit.Rule;
Expand Down Expand Up @@ -840,10 +840,12 @@ public void populateDisplayData(DisplayData.Builder builder) {
}
};

ParDo.Bound<Integer, Integer> parDo1 = ParDo.of(fn1);
ParDo.Bound<Integer, Integer> parDo2 = ParDo.of(fn2);
pipeline
.apply(Create.of(1, 2, 3))
.apply(ParDo.of(fn1))
.apply(ParDo.of(fn2));
.apply(parDo1)
.apply(parDo2);

Job job = translator.translate(
pipeline, pipeline.getRunner(), Collections.<DataflowPackage>emptyList()).getJob();
Expand All @@ -855,43 +857,53 @@ public void populateDisplayData(DisplayData.Builder builder) {
Map<String, Object> parDo2Properties = steps.get(2).getProperties();
assertThat(parDo1Properties, hasKey("display_data"));


@SuppressWarnings("unchecked")
Collection<Map<String, Object>> fn1displayData =
(Collection<Map<String, Object>>) parDo1Properties.get("display_data");
@SuppressWarnings("unchecked")
Collection<Map<String, Object>> fn2displayData =
(Collection<Map<String, Object>>) parDo2Properties.get("display_data");

@SuppressWarnings("unchecked")
Matcher<Iterable<? extends Map<String, Object>>> fn1expectedData =
Matchers.<Map<String, Object>>containsInAnyOrder(
ImmutableMap.<String, Object>builder()
.put("namespace", fn1.getClass().getName())
.put("key", "foo")
.put("type", "STRING")
.put("value", "bar")
.build(),
ImmutableMap.<String, Object>builder()
.put("namespace", fn1.getClass().getName())
.put("key", "foo2")
.put("type", "JAVA_CLASS")
.put("value", DataflowPipelineTranslatorTest.class.getName())
.put("shortValue", DataflowPipelineTranslatorTest.class.getSimpleName())
.put("label", "Test Class")
.put("linkUrl", "http://www.google.com")
.build());

@SuppressWarnings("unchecked")
Matcher<Iterable<? extends Map<String, Object>>> fn2expectedData =
Matchers.<Map<String, Object>>contains(
ImmutableMap.<String, Object>builder()
.put("namespace", fn2.getClass().getName())
.put("key", "foo3")
.put("type", "INTEGER")
.put("value", 1234L)
.build());
assertThat(fn1displayData, fn1expectedData);
assertThat(fn2displayData, fn2expectedData);
Collection<Map<String, String>> fn1displayData =
(Collection<Map<String, String>>) parDo1Properties.get("display_data");
Collection<Map<String, String>> fn2displayData =
(Collection<Map<String, String>>) parDo2Properties.get("display_data");

ImmutableSet<ImmutableMap<String, Object>> expectedFn1DisplayData = ImmutableSet.of(
ImmutableMap.<String, Object>builder()
.put("key", "foo")
.put("type", "STRING")
.put("value", "bar")
.put("namespace", fn1.getClass().getName())
.build(),
ImmutableMap.<String, Object>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.<String, Object>builder()
.put("key", "foo2")
.put("type", "JAVA_CLASS")
.put("value", DataflowPipelineTranslatorTest.class.getName())
.put("shortValue", DataflowPipelineTranslatorTest.class.getSimpleName())
.put("namespace", fn1.getClass().getName())
.put("label", "Test Class")
.put("linkUrl", "http://www.google.com")
.build()
);

ImmutableSet<ImmutableMap<String, Object>> expectedFn2DisplayData = ImmutableSet.of(
Copy link
Contributor

Choose a reason for hiding this comment

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

This shouldn't need to change anymore -- the Matcher based version should work, right?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is the version that existed in my branch before mergepocolypse. It includes a few more fields which are now being serialized, and is also a stronger assertion in that it is testing for set equality rather than subset.

ImmutableMap.<String, Object>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.<String, Object>builder()
.put("key", "foo3")
.put("type", "INTEGER")
.put("value", 1234L)
.put("namespace", fn2.getClass().getName())
.build()
);

assertEquals(expectedFn1DisplayData, ImmutableSet.copyOf(fn1displayData));
assertEquals(expectedFn2DisplayData, ImmutableSet.copyOf(fn2displayData));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import org.apache.beam.sdk.coders.CannotProvideCoderException;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.display.DisplayData;
import org.apache.beam.sdk.values.PInput;
import org.apache.beam.sdk.values.POutput;
import org.apache.beam.sdk.values.TypedPValue;
Expand Down Expand Up @@ -53,4 +54,9 @@ public <T> Coder<T> getDefaultOutputCoder(InputT input, @SuppressWarnings("unuse
TypedPValue<T> output) throws CannotProvideCoderException {
return delegate().getDefaultOutputCoder(input, output);
}

@Override
public void populateDisplayData(DisplayData.Builder builder) {
delegate().populateDisplayData(builder);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.beam.sdk.transforms.DoFnWithContext.FinishBundle;
import org.apache.beam.sdk.transforms.DoFnWithContext.ProcessElement;
import org.apache.beam.sdk.transforms.DoFnWithContext.StartBundle;
import org.apache.beam.sdk.transforms.display.DisplayData;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.transforms.windowing.PaneInfo;
import org.apache.beam.sdk.util.UserCodeException;
Expand Down Expand Up @@ -653,6 +654,11 @@ protected TypeDescriptor<OutputT> 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();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,8 @@
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.transforms.Combine.CombineFn;
import org.apache.beam.sdk.transforms.DoFn.DelegatingAggregator;
import org.apache.beam.sdk.transforms.display.DisplayData;
import org.apache.beam.sdk.transforms.display.HasDisplayData;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.transforms.windowing.PaneInfo;
import org.apache.beam.sdk.util.WindowingInternals;
Expand Down Expand Up @@ -82,7 +84,7 @@
* @param <OutputT> the type of the (main) output elements
*/
@Experimental
public abstract class DoFnWithContext<InputT, OutputT> implements Serializable {
public abstract class DoFnWithContext<InputT, OutputT> implements Serializable, HasDisplayData {

/** Information accessible to all methods in this {@code DoFnWithContext}. */
public abstract class Context {
Expand Down Expand Up @@ -414,4 +416,14 @@ public final <AggInputT> Aggregator<AggInputT, AggInputT> createAggregator(
void prepareForProcessing() {
aggregatorsAreFinal = true;
}

/**
* {@inheritDoc}
*
* <p>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) {
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.beam.sdk.transforms;

import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.transforms.display.DisplayData;
import org.apache.beam.sdk.values.PCollection;

/**
Expand Down Expand Up @@ -99,9 +100,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<T>} and returns a {@code PCollection<T>} with
Expand Down Expand Up @@ -131,6 +138,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));
}
});
}

Expand Down Expand Up @@ -163,6 +175,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));
}
});
}

Expand Down Expand Up @@ -195,6 +212,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));
}
});
}

Expand Down Expand Up @@ -232,4 +254,9 @@ public void processElement(ProcessContext c) {
protected Coder<T> getDefaultOutputCoder(PCollection<T> input) {
return input.getCoder();
}

private static void populateDisplayData(
DisplayData.Builder builder, String predicateDescription) {
builder.add("predicate", predicateDescription);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.beam.sdk.coders.Coder.NonDeterministicException;
import org.apache.beam.sdk.coders.IterableCoder;
import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.transforms.display.DisplayData;
import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
import org.apache.beam.sdk.transforms.windowing.InvalidWindows;
Expand Down Expand Up @@ -273,4 +274,12 @@ static <K, V> Coder<Iterable<V>> getOutputValueCoder(Coder<KV<K, V>> inputCoder)
public static <K, V> KvCoder<K, Iterable<V>> getOutputKvCoder(Coder<KV<K, V>> inputCoder) {
return KvCoder.of(getKeyCoder(inputCoder), getOutputValueCoder(inputCoder));
}

@Override
public void populateDisplayData(DisplayData.Builder builder) {
if (fewKeys) {
builder.add("fewKeys", true);
}
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import org.apache.beam.sdk.options.GcsOptions;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.transforms.Combine.CombineFn;
import org.apache.beam.sdk.transforms.display.DisplayData;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.transforms.windowing.PaneInfo;
import org.apache.beam.sdk.util.WindowingInternals;
Expand Down Expand Up @@ -172,6 +173,14 @@ public PCollection<OutputT> apply(PCollection<? extends InputT> 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);
}
}

/**
Expand Down
Loading