Skip to content
Original file line number Diff line number Diff line change
Expand Up @@ -874,10 +874,10 @@ public void processElement(ProcessContext c) throws Exception {
@Override
public void populateDisplayData(DisplayData.Builder builder) {
builder
.add("foo", "bar")
.add("foo2", DataflowPipelineTranslatorTest.class)
.add(DisplayData.item("foo", "bar"))
.add(DisplayData.item("foo2", DataflowPipelineTranslatorTest.class)
.withLabel("Test Class")
.withLinkUrl("http://www.google.com");
.withLinkUrl("http://www.google.com"));
}
};

Expand All @@ -889,7 +889,7 @@ public void processElement(ProcessContext c) throws Exception {

@Override
public void populateDisplayData(DisplayData.Builder builder) {
builder.add("foo3", 1234);
builder.add(DisplayData.item("foo3", 1234));
}
};

Expand All @@ -915,8 +915,10 @@ public void populateDisplayData(DisplayData.Builder builder) {
Map<String, Object> parDo2Properties = steps.get(2).getProperties();
assertThat(parDo1Properties, hasKey("display_data"));

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

Expand Down
18 changes: 10 additions & 8 deletions sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
Original file line number Diff line number Diff line change
Expand Up @@ -331,8 +331,8 @@ public PCollection<T> apply(PInput input) {
public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);
builder
.addIfNotNull("filePattern", filepattern)
.addIfNotDefault("validation", validate, true);
.addIfNotNull(DisplayData.item("filePattern", filepattern))
.addIfNotDefault(DisplayData.item("validation", validate), true);
}

@Override
Expand Down Expand Up @@ -694,12 +694,14 @@ public PDone apply(PCollection<T> input) {
public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);
builder
.add("schema", type)
.addIfNotNull("filePrefix", filenamePrefix)
.addIfNotDefault("shardNameTemplate", shardTemplate, DEFAULT_SHARD_TEMPLATE)
.addIfNotDefault("fileSuffix", filenameSuffix, "")
.addIfNotDefault("numShards", numShards, 0)
.addIfNotDefault("validation", validate, true);
.add(DisplayData.item("schema", type))
.addIfNotNull(DisplayData.item("filePrefix", filenamePrefix))
.addIfNotDefault(
DisplayData.item("shardNameTemplate", shardTemplate),
DEFAULT_SHARD_TEMPLATE)
.addIfNotDefault(DisplayData.item("fileSuffix", filenameSuffix), "")
.addIfNotDefault(DisplayData.item("numShards", numShards), 0)
.addIfNotDefault(DisplayData.item("validation", validate), true);
}

/**
Expand Down
20 changes: 10 additions & 10 deletions sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
Original file line number Diff line number Diff line change
Expand Up @@ -522,13 +522,13 @@ public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);

if (table != null) {
builder.add("table", toTableSpec(table));
builder.add(DisplayData.item("table", toTableSpec(table)));
}

builder
.addIfNotNull("query", query)
.addIfNotNull("flattenResults", flattenResults)
.addIfNotDefault("validation", validate, true);
.addIfNotNull(DisplayData.item("query", query))
.addIfNotNull(DisplayData.item("flattenResults", flattenResults))
.addIfNotDefault(DisplayData.item("validation", validate), true);
}

static {
Expand Down Expand Up @@ -1067,17 +1067,17 @@ public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);

builder
.addIfNotNull("table", jsonTableRef)
.addIfNotNull("schema", jsonSchema);
.addIfNotNull(DisplayData.item("table", jsonTableRef))
.addIfNotNull(DisplayData.item("schema", jsonSchema));

if (tableRefFunction != null) {
builder.add("tableFn", tableRefFunction.getClass());
builder.add(DisplayData.item("tableFn", tableRefFunction.getClass()));
}

builder
.add("createDisposition", createDisposition.toString())
.add("writeDisposition", writeDisposition.toString())
.addIfNotDefault("validation", validate, true);
.add(DisplayData.item("createDisposition", createDisposition.toString()))
.add(DisplayData.item("writeDisposition", writeDisposition.toString()))
.addIfNotDefault(DisplayData.item("validation", validate), true);
}

/** Returns the create disposition. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -113,9 +113,9 @@ public String getKindString() {
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)
.add(DisplayData.item("source", source.getClass()))
.addIfNotDefault(DisplayData.item("maxRecords", maxNumRecords), Long.MAX_VALUE)
.addIfNotNull(DisplayData.item("maxReadTime", maxReadTime))
.include(source);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -326,14 +326,14 @@ 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());
.add(DisplayData.item("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());
builder.add(DisplayData.item("compressionMode", ((Enum) channelFactory).name()));
} else {
builder.add("compressionMode", channelFactory.getClass());
builder.add(DisplayData.item("compressionMode", channelFactory.getClass()));
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -119,7 +119,7 @@ public PCollection<Long> apply(PBegin begin) {
@Override
public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);
builder.add("upTo", numElements);
builder.add(DisplayData.item("upTo", numElements));
}
}

Expand Down Expand Up @@ -233,14 +233,14 @@ public PCollection<Long> apply(PBegin begin) {
public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);

builder.add("timestampFn", timestampFn.getClass());
builder.add(DisplayData.item("timestampFn", timestampFn.getClass()));

if (maxReadTime.isPresent()) {
builder.add("maxReadTime", maxReadTime.get());
builder.add(DisplayData.item("maxReadTime", maxReadTime.get()));
}

if (maxNumRecords.isPresent()) {
builder.add("maxRecords", maxNumRecords.get());
builder.add(DisplayData.item("maxRecords", maxNumRecords.get()));
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -391,12 +391,12 @@ public long getEstimatedSizeBytes(PipelineOptions options) throws Exception {
public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);
builder
.addIfNotDefault("host", host, DEFAULT_HOST)
.addIfNotNull("dataset", datasetId)
.addIfNotNull("namespace", namespace);
.addIfNotDefault(DisplayData.item("host", host), DEFAULT_HOST)
.addIfNotNull(DisplayData.item("dataset", datasetId))
.addIfNotNull(DisplayData.item("namespace", namespace));

if (query != null) {
builder.add("query", query.toString());
builder.add(DisplayData.item("query", query.toString()));
}
}

Expand Down Expand Up @@ -606,8 +606,8 @@ public DatastoreWriteOperation createWriteOperation(PipelineOptions options) {
public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);
builder
.addIfNotDefault("host", host, DEFAULT_HOST)
.addIfNotNull("dataset", datasetId);
.addIfNotDefault(DisplayData.item("host", host), DEFAULT_HOST)
.addIfNotNull(DisplayData.item("dataset", datasetId));
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -142,7 +142,7 @@ public void populateDisplayData(DisplayData.Builder builder) {

String fileNamePattern = String.format("%s%s%s",
baseOutputFilename, fileNamingTemplate, getFileExtension(extension));
builder.add("fileNamePattern", fileNamePattern);
builder.add(DisplayData.item("fileNamePattern", fileNamePattern));
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -277,7 +277,7 @@ private static long getEstimatedSizeOfFilesBySampling(
@Override
public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);
builder.add("filePattern", getFileOrPatternSpec());
builder.add(DisplayData.item("filePattern", getFileOrPatternSpec()));
}

private ListenableFuture<List<? extends FileBasedSource<T>>> createFutureForFileSplit(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -207,9 +207,9 @@ public boolean allowsDynamicSplitting() {
public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);
builder
.add("minBundleSize", minBundleSize)
.addIfNotDefault("startOffset", startOffset, 0)
.addIfNotDefault("endOffset", endOffset, Long.MAX_VALUE);
.add(DisplayData.item("minBundleSize", minBundleSize))
.addIfNotDefault(DisplayData.item("startOffset", startOffset), 0L)
.addIfNotDefault(DisplayData.item("endOffset", endOffset), Long.MAX_VALUE);
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -694,17 +694,17 @@ public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);

builder
.addIfNotNull("timestampLabel", timestampLabel)
.addIfNotNull("idLabel", idLabel)
.addIfNotNull("maxReadTime", maxReadTime)
.addIfNotDefault("maxNumRecords", maxNumRecords, 0);
.addIfNotNull(DisplayData.item("timestampLabel", timestampLabel))
.addIfNotNull(DisplayData.item("idLabel", idLabel))
.addIfNotNull(DisplayData.item("maxReadTime", maxReadTime))
.addIfNotDefault(DisplayData.item("maxNumRecords", maxNumRecords), 0);

if (topic != null) {
builder.add("topic", topic.asPath());
builder.add(DisplayData.item("topic", topic.asPath()));
}

if (subscription != null) {
builder.add("subscription", subscription.asPath());
builder.add(DisplayData.item("subscription", subscription.asPath()));
}
}

Expand Down Expand Up @@ -998,11 +998,11 @@ public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);

builder
.addIfNotNull("timestampLabel", timestampLabel)
.addIfNotNull("idLabel", idLabel);
.addIfNotNull(DisplayData.item("timestampLabel", timestampLabel))
.addIfNotNull(DisplayData.item("idLabel", idLabel));

if (topic != null) {
builder.add("topic", topic.asPath());
builder.add(DisplayData.item("topic", topic.asPath()));
}
}

Expand Down
4 changes: 2 additions & 2 deletions sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java
Original file line number Diff line number Diff line change
Expand Up @@ -149,7 +149,7 @@ public String getKindString() {
public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);
builder
.add("source", source.getClass())
.add(DisplayData.item("source", source.getClass()))
.include(source);
}

Expand Down Expand Up @@ -264,7 +264,7 @@ public String getKindString() {
public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);
builder
.add("source", source.getClass())
.add(DisplayData.item("source", source.getClass()))
.include(source);
}
}
Expand Down
18 changes: 10 additions & 8 deletions sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
Original file line number Diff line number Diff line change
Expand Up @@ -344,9 +344,9 @@ public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);

builder
.add("compressionType", compressionType.toString())
.addIfNotDefault("validation", validate, true)
.addIfNotNull("filePattern", filepattern);
.add(DisplayData.item("compressionType", compressionType.toString()))
.addIfNotDefault(DisplayData.item("validation", validate), true)
.addIfNotNull(DisplayData.item("filePattern", filepattern));
}

@Override
Expand Down Expand Up @@ -649,11 +649,13 @@ public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);

builder
.addIfNotNull("filePrefix", filenamePrefix)
.addIfNotDefault("fileSuffix", filenameSuffix, "")
.addIfNotDefault("shardNameTemplate", shardTemplate, DEFAULT_SHARD_TEMPLATE)
.addIfNotDefault("validation", validate, true)
.addIfNotDefault("numShards", numShards, 0);
.addIfNotNull(DisplayData.item("filePrefix", filenamePrefix))
.addIfNotDefault(DisplayData.item("fileSuffix", filenameSuffix), "")
.addIfNotDefault(
DisplayData.item("shardNameTemplate", shardTemplate),
DEFAULT_SHARD_TEMPLATE)
.addIfNotDefault(DisplayData.item("validation", validate), true)
.addIfNotDefault(DisplayData.item("numShards", numShards), 0);
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ public PDone apply(PCollection<T> input) {
public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);
builder
.add("sink", sink.getClass())
.add(DisplayData.item("sink", sink.getClass()))
.include(sink);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -227,8 +227,8 @@ public XmlWriteOperation<T> createWriteOperation(PipelineOptions options) {
public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);
builder
.addIfNotNull("rootElement", rootElementName)
.addIfNotNull("recordClass", classToBind);
.addIfNotNull(DisplayData.item("rootElement", rootElementName))
.addIfNotNull(DisplayData.item("recordClass", classToBind));
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -222,9 +222,9 @@ public void validate() {
public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);
builder
.addIfNotNull("rootElement", rootElement)
.addIfNotNull("recordElement", recordElement)
.addIfNotNull("recordClass", recordClass);
.addIfNotNull(DisplayData.item("rootElement", rootElement))
.addIfNotNull(DisplayData.item("recordElement", recordElement))
.addIfNotNull(DisplayData.item("recordClass", recordClass));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -283,11 +283,11 @@ private void populateDisplayData(DisplayData.Builder builder) {
for (PipelineOptionSpec optionSpec : specs) {
Class<?> pipelineInterface = optionSpec.getDefiningInterface();
if (type != null) {
builder.add(option.getKey(), type, value)
.withNamespace(pipelineInterface);
builder.add(DisplayData.item(option.getKey(), type, value)
.withNamespace(pipelineInterface));
} else {
builder.add(option.getKey(), value.toString())
.withNamespace(pipelineInterface);
builder.add(DisplayData.item(option.getKey(), value.toString())
.withNamespace(pipelineInterface));
}
}
}
Expand All @@ -300,18 +300,18 @@ private void populateDisplayData(DisplayData.Builder builder) {

HashSet<PipelineOptionSpec> specs = new HashSet<>(optionsMap.get(jsonOption.getKey()));
if (specs.isEmpty()) {
builder.add(jsonOption.getKey(), jsonOption.getValue().toString())
.withNamespace(UnknownPipelineOptions.class);
builder.add(DisplayData.item(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());
builder.add(DisplayData.item(jsonOption.getKey(), type, value)
.withNamespace(spec.getDefiningInterface()));
} else {
builder.add(jsonOption.getKey(), value.toString())
.withNamespace(spec.getDefiningInterface());
builder.add(DisplayData.item(jsonOption.getKey(), value.toString())
.withNamespace(spec.getDefiningInterface()));
}
}
}
Expand Down Expand Up @@ -542,7 +542,7 @@ public void serialize(PipelineOptions value, JsonGenerator jgen, SerializerProvi
jgen.writeObject(serializableOptions);

List<Map<String, Object>> serializedDisplayData = Lists.newArrayList();
for (DisplayData.Item item : DisplayData.from(value).items()) {
for (DisplayData.Item<?> item : DisplayData.from(value).items()) {
@SuppressWarnings("unchecked")
Map<String, Object> serializedItem = MAPPER.convertValue(item, Map.class);
serializedDisplayData.add(serializedItem);
Expand Down
Loading