[ 
https://issues.apache.org/jira/browse/BEAM-2085?focusedWorklogId=589950&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-589950
 ]

ASF GitHub Bot logged work on BEAM-2085:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 27/Apr/21 18:45
            Start Date: 27/Apr/21 18:45
    Worklog Time Spent: 10m 
      Work Description: tvalentyn commented on a change in pull request #14607:
URL: https://github.com/apache/beam/pull/14607#discussion_r621500680



##########
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:
       I think style guide says to use upper case for:
   `All static final object reference types that are never followed by "." 
(dot).`
   so yes, chagning.




-- 
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: 589950)
    Time Spent: 38h  (was: 37h 50m)

> 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: 38h
>  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)

Reply via email to