[
https://issues.apache.org/jira/browse/BEAM-2085?focusedWorklogId=586881&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-586881
]
ASF GitHub Bot logged work on BEAM-2085:
----------------------------------------
Author: ASF GitHub Bot
Created on: 21/Apr/21 20:36
Start Date: 21/Apr/21 20:36
Worklog Time Spent: 10m
Work Description: tvalentyn commented on a change in pull request #14607:
URL: https://github.com/apache/beam/pull/14607#discussion_r617859428
##########
File path:
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHints.java
##########
@@ -0,0 +1,270 @@
+/*
+ * 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;
+
+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());
+
+ private final ImmutableMap<String, ResourceHint> hints;
+
+ private ResourceHints(ImmutableMap<String, ResourceHint> hints) {
+ this.hints = hints;
+ }
+
+ public static ResourceHints create() {
+ return empty;
+ }
+
+ public static ResourceHints fromOptions(PipelineOptions options) {
+ ResourceHintsOptions resourceHintsOptions =
options.as(ResourceHintsOptions.class);
+ ResourceHints result = create();
+ if (resourceHintsOptions.getResourceHints() == null) {
+ return result;
+ }
+ Splitter splitter = Splitter.on('=').limit(2);
+ for (String hint : resourceHintsOptions.getResourceHints()) {
+ List<String> parts = splitter.splitToList(hint);
+ if (parts.size() != 2) {
+ throw new IllegalArgumentException("Unparsable resource hint: " +
hint);
+ }
+ String nameOrUrn = parts.get(0);
+ String stringValue = parts.get(1);
+ String urn;
+ if (hintNameToUrn.containsKey(nameOrUrn)) {
+ urn = hintNameToUrn.get(nameOrUrn);
+ } else if (!nameOrUrn.startsWith("beam:resources:")) {
+ // Allow unknown hints to be passed, but validate a little bit to
prevent typos.
+ throw new IllegalArgumentException("Unknown resource hint: " + hint);
+ } else {
+ urn = nameOrUrn;
+ }
+ ResourceHint value = parsers.getOrDefault(urn, s -> new
StringHint(s)).apply(stringValue);
+ result = result.withHint(urn, value);
+ }
+ return result;
+ }
+
+ /*package*/ static class BytesHint extends ResourceHint {
+ private static Map<String, Long> suffixes =
+ ImmutableMap.<String, Long>builder()
+ .put("B", 1L)
Review comment:
Replied on the other comment.
--
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]
Issue Time Tracking
-------------------
Worklog Id: (was: 586881)
Time Spent: 33.5h (was: 33h 20m)
> Transforms could give hints to runners on resource requirements
> ---------------------------------------------------------------
>
> Key: BEAM-2085
> URL: https://issues.apache.org/jira/browse/BEAM-2085
> Project: Beam
> Issue Type: Improvement
> Components: beam-model, runner-core, sdk-java-core
> Affects Versions: Not applicable
> Reporter: Ismaël Mejía
> Assignee: Valentyn Tymofieiev
> Priority: P3
> Labels: Clarified
> Time Spent: 33.5h
> Remaining Estimate: 0h
>
> As discussed in BEAM-673 runners can allocate workers to accomplish their
> work in a better way if they can take into account some hints from the
> transforms, e.g. a source can hint data locality and with this information
> the runner can allocate the workers in an better way, this can also be the
> case with a particular transform that can suggest to the runner to be
> executed in a worker with a specific resource, e.g. GPU.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)