robertwb commented on a change in pull request #14607:
URL: https://github.com/apache/beam/pull/14607#discussion_r620746115
##########
File path:
runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
##########
@@ -760,6 +776,24 @@ private void addDisplayData(Step step, String stepName,
HasDisplayData hasDispla
List<Map<String, Object>> list = MAPPER.convertValue(displayData,
List.class);
addList(getProperties(), PropertyNames.DISPLAY_DATA, list);
}
+
+ private void addResourceHints(Step step, String stepName, ResourceHints
hints) {
+ Map<String, Object> urlEncodedHints = new HashMap<>();
+ PercentCodec codec = new PercentCodec();
+ for (Entry<String, ResourceHint> entry : hints.hints().entrySet()) {
+ try {
Review comment:
You could just use
https://docs.oracle.com/javase/7/docs/api/java/net/URLEncoder.html#encode(java.lang.String,%20java.lang.String)
##########
File path:
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java
##########
@@ -178,6 +179,34 @@ public String getName() {
return name != null ? name : getKindString();
}
+ /**
+ * Sets resource hints for the transform.
+ *
+ * @param resourceHints a {@link ResourceHints} instance.
+ * @return a reference to the same transfrom instance.
+ * <p>For example:
+ * <pre>{@code
+ * Pipeline p = ...
+ * ...
+ * p.apply(new
SomeTransform().setResourceHints(ResourceHints.create().withMinRam("6 GiB")))
+ * ...
+ *
+ * }</pre>
+ */
+ public PTransform<InputT, OutputT> setResourceHints(ResourceHints
resourceHints) {
+ this.resourceHints = resourceHints;
+ return this;
+ }
+
+ /** Returns resource hints set on the transform. */
+ public ResourceHints getResourceHints() {
+ if (resourceHints == null) {
Review comment:
Is this preferable to initializing it in the constructor (or at field
declaration)?
##########
File path:
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHints.java
##########
@@ -0,0 +1,291 @@
+/*
+ * 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 org.apache.beam.sdk.transforms.resourcehints;
+
+import static
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState;
+
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
+import org.apache.beam.model.pipeline.v1.RunnerApi.StandardResourceHints;
+import org.apache.beam.sdk.options.PipelineOptions;
+import
org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ProtocolMessageEnum;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Charsets;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Splitter;
+import
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Pipeline authors can use resource hints to provide additional information
to runners about the
+ * desired aspects of the execution environment. Resource hints can be
specified via {@link
+ * org.apache.beam.sdk.transforms.PTransform PTransform#setResourceHints} for
parts of the pipeline,
+ * or globally via {@link ResourceHintsOptions resourceHints} pipeline option.
+ *
+ * <p>Interpretation of hints is provided by Beam runners.
+ */
+public class ResourceHints {
+ private static final String MIN_RAM_URN = "beam:resources:min_ram_bytes:v1";
+ private static final String ACCELERATOR_URN =
"beam:resources:accelerator:v1";
+
+ // TODO: reference this from a common location in all packages that use this.
+ private static String getUrn(ProtocolMessageEnum value) {
+ return
value.getValueDescriptor().getOptions().getExtension(RunnerApi.beamUrn);
+ }
+
+ static {
+
checkState(MIN_RAM_URN.equals(getUrn(StandardResourceHints.Enum.MIN_RAM_BYTES)));
+
checkState(ACCELERATOR_URN.equals(getUrn(StandardResourceHints.Enum.ACCELERATOR)));
+ }
+
+ private static ImmutableMap<String, String> hintNameToUrn =
+ ImmutableMap.<String, String>builder()
+ .put("minRam", MIN_RAM_URN)
+ .put("min_ram", MIN_RAM_URN) // Courtesy alias.
+ .put("accelerator", ACCELERATOR_URN)
+ .build();
+
+ private static ImmutableMap<String, Function<String, ResourceHint>> parsers =
+ ImmutableMap.<String, Function<String, ResourceHint>>builder()
+ .put(MIN_RAM_URN, s -> new BytesHint(BytesHint.parse(s)))
+ .put(ACCELERATOR_URN, s -> new StringHint(s))
+ .build();
+
+ private static ResourceHints empty = new ResourceHints(ImmutableMap.of());
Review comment:
should be final (and probably EMPTY).
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]