You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/11/07 20:42:30 UTC

[GitHub] [beam] damccorm commented on a diff in pull request #23990: [Go] Pipeline Resource Hints

damccorm commented on code in PR #23990:
URL: https://github.com/apache/beam/pull/23990#discussion_r1015865539


##########
sdks/go/pkg/beam/options/jobopts/options.go:
##########
@@ -45,6 +46,12 @@ func init() {
 			"have no more than 1 override applied to it. If multiple "+
 			"overrides match a container image it is arbitrary which "+
 			"will be applied.")
+	flag.Var(&ResourceHints,
+		"resource_hints",
+		"Set whole pipeline level resource hints, accepting values of the format '<urn>=<value>'. "+
+			"Supports 'min_ram' and 'accelerator' hints. "+

Review Comment:
   Optional opinionated feedback - IMO its not a great idea to specify the full set of acceptable values like that if the accepted values are liable to change without updating the code since its really easy for them to get out of date.
   
   I'd personally prefer specifying these as example options



##########
sdks/go/pkg/beam/options/jobopts/options.go:
##########
@@ -169,3 +179,49 @@ func GetExperiments() []string {
 	}
 	return strings.Split(*Experiments, ",")
 }
+
+// GetPipelineResourceHints parses known standard hints and returns the flag set hints for the pipeline.
+// In case of duplicate hint URNs, the last value specified will be used.
+func GetPipelineResourceHints() resource.Hints {
+	hints := make([]resource.Hint, 0, len(ResourceHints))
+	for _, hint := range ResourceHints {
+		name, val, ok := strings.Cut(hint, "=")
+		if !ok {
+			panic(fmt.Sprintf("unparsable resource hint: %q", hint))
+		}
+		var h resource.Hint
+		switch name {
+		case "min_ram":

Review Comment:
   Should we also check for equality to the fully specified payload (`beam:resources:min_ram_bytes:v1`)? Same thing with accelerator



##########
sdks/go/pkg/beam/options/resource/hint.go:
##########
@@ -0,0 +1,194 @@
+// 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 resource supports Beam resource hints to specify scoped hints or annotations
+// to pipelines.
+//
+// See https://beam.apache.org/documentation/runtime/resource-hints/ for more information.
+package resource
+
+import (
+	"bytes"
+	"fmt"
+	"strconv"
+
+	"github.com/dustin/go-humanize"
+)
+
+// Hints contains a list of hints for a given scope.
+type Hints struct {
+	h map[string]Hint
+}
+
+// MergeWithOuter produces a new list of Hints from this Hints, and the Hints from the outer scope.

Review Comment:
   2 thoughts:
   1) Could you specify the merge strategy? Looks like it is hint dependent, specifying that would be enough
   2) Does I don't quite get the naming here - why is this MergeWithOuter and the analogous hint level operation is MergeWith? Personally, my vote would just be for Merge, but staying consistent is the more important piece.



##########
sdks/go/pkg/beam/core/runtime/graphx/translate.go:
##########
@@ -1122,7 +1126,12 @@ const defaultEnvId = "go"
 
 func (m *marshaller) addDefaultEnv() string {
 	if _, exists := m.environments[defaultEnvId]; !exists {
-		m.environments[defaultEnvId] = m.opt.Environment
+		env := proto.Clone(m.opt.Environment).(*pipepb.Environment)
+		// Add the pipeline level resource hints here for now.
+		// TODO(https://github.com/apache/beam/issues/23893) move to a better place for
+		// scoped hints in next pass, which affect number of environments set by Go pipelines.
+		env.ResourceHints = m.opt.PipelineResourceHints.Payloads()

Review Comment:
   I think `m.opt.PipelineResourceHints` is not guaranteed to be non-nil (e.g. on the direct runner I don't think this will ever get set). It would be good to do a nil check here



-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org