pulumi/pkg/codegen/nodejs/gen_program_expressions.go

801 lines
24 KiB
Go
Raw Normal View History

// Copyright 2020-2024, Pulumi Corporation.
//
// Licensed 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 nodejs
import (
"bytes"
Enable perfsprint linter (#14813) <!--- Thanks so much for your contribution! If this is your first time contributing, please ensure that you have read the [CONTRIBUTING](https://github.com/pulumi/pulumi/blob/master/CONTRIBUTING.md) documentation. --> # Description <!--- Please include a summary of the change and which issue is fixed. Please also include relevant motivation and context. --> Prompted by a comment in another review: https://github.com/pulumi/pulumi/pull/14654#discussion_r1419995945 This lints that we don't use `fmt.Errorf` when `errors.New` will suffice, it also covers a load of other cases where `Sprintf` is sub-optimal. Most of these edits were made by running `perfsprint --fix`. ## Checklist - [x] I have run `make tidy` to update any new dependencies - [x] I have run `make lint` to verify my code passes the lint check - [x] I have formatted my code using `gofumpt` <!--- Please provide details if the checkbox below is to be left unchecked. --> - [ ] I have added tests that prove my fix is effective or that my feature works <!--- User-facing changes require a CHANGELOG entry. --> - [ ] I have run `make changelog` and committed the `changelog/pending/<file>` documenting my change <!-- If the change(s) in this PR is a modification of an existing call to the Pulumi Cloud, then the service should honor older versions of the CLI where this change would not exist. You must then bump the API version in /pkg/backend/httpstate/client/api.go, as well as add it to the service. --> - [ ] Yes, there are changes in this PR that warrants bumping the Pulumi Cloud API version <!-- @Pulumi employees: If yes, you must submit corresponding changes in the service repo. -->
2023-12-12 12:19:42 +00:00
"errors"
"fmt"
"io"
"math/big"
"strings"
"unicode"
"github.com/hashicorp/hcl/v2"
"github.com/hashicorp/hcl/v2/hclsyntax"
"github.com/zclconf/go-cty/cty"
"github.com/zclconf/go-cty/cty/convert"
"github.com/pulumi/pulumi/pkg/v3/codegen/hcl2/model"
"github.com/pulumi/pulumi/pkg/v3/codegen/pcl"
"github.com/pulumi/pulumi/pkg/v3/codegen/schema"
"github.com/pulumi/pulumi/sdk/v3/go/common/util/contract"
)
type nameInfo int
func (nameInfo) Format(name string) string {
return makeValidIdentifier(name)
}
func (g *generator) lowerExpression(expr model.Expression, typ model.Type) model.Expression {
// TODO(pdg): diagnostics
if g.asyncMain {
expr = g.awaitInvokes(expr)
}
expr = pcl.RewritePropertyReferences(expr)
[program-gen] Emit Output-returning JSON serialization methods without rewriting applies (#15371) ### Description A while ago we started implementing [specialized JSON serialization methods](https://github.com/pulumi/pulumi/issues/12519) for Pulumi programs which can accept nested outputs without having to rewrite and combine applies. - `Output.SerializeJson` in .NET - `pulumi.jsonStringify` in nodejs - `pulumi.Output.json_dumps` in Python This PR extends program-gen for TypeScript, C# and Python to start emitting these JSON serialization functions (when necessary). The PR special-cases the `toJSON` PCL function when rewriting applies so that nested outputs aren't rewritted. Example PCL program and generated results: > Also check out the downstream codegen tests to see improved generated examples ``` resource vpc "aws:ec2:Vpc" { cidrBlock = "10.100.0.0/16" instanceTenancy = "default" } resource policy "aws:iam/policy:Policy" { description = "test" policy = toJSON({ "Version" = "2012-10-17" "Interpolated" = "arn:${vpc.arn}:value" "Value" = vpc.id }) } ``` ### Generated TypeScript Before ```typescript import * as pulumi from "@pulumi/pulumi"; import * as aws from "@pulumi/aws"; const vpc = new aws.ec2.Vpc("vpc", { cidrBlock: "10.100.0.0/16", instanceTenancy: "default", }); const policy = new aws.iam.Policy("policy", { description: "test", policy: pulumi.all([vpc.arn, vpc.id]).apply(([arn, id]) => JSON.stringify({ Version: "2012-10-17", Interpolated: `arn:${arn}:value`, Value: id, })), }); ``` ### Generated TypeScript After ```typescript import * as pulumi from "@pulumi/pulumi"; import * as aws from "@pulumi/aws"; const vpc = new aws.ec2.Vpc("vpc", { cidrBlock: "10.100.0.0/16", instanceTenancy: "default", }); const policy = new aws.iam.Policy("policy", { description: "test", policy: pulumi.jsonStringify({ Version: "2012-10-17", Interpolated: pulumi.interpolate`arn:${vpc.arn}:value`, Value: vpc.id, }), }); ``` ### Generated Python Before ```python import pulumi import json import pulumi_aws as aws vpc = aws.ec2.Vpc("vpc", cidr_block="10.100.0.0/16", instance_tenancy="default") policy = aws.iam.Policy("policy", description="test", policy=pulumi.Output.all(vpc.arn, vpc.id).apply(lambda arn, id: json.dumps({ "Version": "2012-10-17", "Interpolated": f"arn:{arn}:value", "Value": id, }))) ``` ### Generated Python After ```python import pulumi import json import pulumi_aws as aws vpc = aws.ec2.Vpc("vpc", cidr_block="10.100.0.0/16", instance_tenancy="default") policy = aws.iam.Policy("policy", description="test", policy=pulumi.Output.json_dumps({ "Version": "2012-10-17", "Interpolated": vpc.arn.apply(lambda arn: f"arn:{arn}:value"), "Value": vpc.id, })) ``` ### Generated C# Before ```csharp using System.Collections.Generic; using System.Linq; using System.Text.Json; using Pulumi; using Aws = Pulumi.Aws; return await Deployment.RunAsync(() => { var vpc = new Aws.Ec2.Vpc("vpc", new() { CidrBlock = "10.100.0.0/16", InstanceTenancy = "default", }); var policy = new Aws.Iam.Policy("policy", new() { Description = "test", PolicyDocument = Output.Tuple(vpc.Arn, vpc.Id).Apply(values => { var arn = values.Item1; var id = values.Item2; return JsonSerializer.Serialize(new Dictionary<string, object?> { ["Version"] = "2012-10-17", ["Interpolated"] = $"arn:{arn}:value", ["Value"] = id, }); }), }); }); ``` ### Generated C# After ```csharp using System.Collections.Generic; using System.Linq; using System.Text.Json; using Pulumi; using Aws = Pulumi.Aws; return await Deployment.RunAsync(() => { var vpc = new Aws.Ec2.Vpc("vpc", new() { CidrBlock = "10.100.0.0/16", InstanceTenancy = "default", }); var policy = new Aws.Iam.Policy("policy", new() { Description = "test", PolicyDocument = Output.JsonSerialize(Output.Create(new Dictionary<string, object?> { ["Version"] = "2012-10-17", ["Interpolated"] = vpc.Arn.Apply(arn => $"arn:{arn}:value"), ["Value"] = vpc.Id, })), }); }); ``` ## Checklist - [ ] I have run `make tidy` to update any new dependencies - [x] I have run `make lint` to verify my code passes the lint check - [x] I have formatted my code using `gofumpt` <!--- Please provide details if the checkbox below is to be left unchecked. --> - [x] I have added tests that prove my fix is effective or that my feature works <!--- User-facing changes require a CHANGELOG entry. --> - [x] I have run `make changelog` and committed the `changelog/pending/<file>` documenting my change <!-- If the change(s) in this PR is a modification of an existing call to the Pulumi Cloud, then the service should honor older versions of the CLI where this change would not exist. You must then bump the API version in /pkg/backend/httpstate/client/api.go, as well as add it to the service. --> - [ ] Yes, there are changes in this PR that warrants bumping the Pulumi Cloud API version <!-- @Pulumi employees: If yes, you must submit corresponding changes in the service repo. -->
2024-02-20 15:48:46 +00:00
skipToJSONWhenRewritingApplies := true
expr, diags := pcl.RewriteAppliesWithSkipToJSON(expr, nameInfo(0), !g.asyncMain, skipToJSONWhenRewritingApplies)
if typ != nil {
var convertDiags hcl.Diagnostics
expr, convertDiags = pcl.RewriteConversions(expr, typ)
diags = diags.Extend(convertDiags)
}
expr, lowerProxyDiags := g.lowerProxyApplies(expr)
diags = diags.Extend(lowerProxyDiags)
g.diagnostics = g.diagnostics.Extend(diags)
return expr
}
func (g *generator) GetPrecedence(expr model.Expression) int {
// Precedence is derived from
// https://developer.mozilla.org/en-US/docs/Web/JavaScript/Reference/Operators/Operator_Precedence.
switch expr := expr.(type) {
case *model.ConditionalExpression:
return 4
case *model.BinaryOpExpression:
switch expr.Operation {
case hclsyntax.OpLogicalOr:
return 5
case hclsyntax.OpLogicalAnd:
return 6
case hclsyntax.OpEqual, hclsyntax.OpNotEqual:
return 11
case hclsyntax.OpGreaterThan, hclsyntax.OpGreaterThanOrEqual, hclsyntax.OpLessThan,
hclsyntax.OpLessThanOrEqual:
return 12
case hclsyntax.OpAdd, hclsyntax.OpSubtract:
return 14
case hclsyntax.OpMultiply, hclsyntax.OpDivide, hclsyntax.OpModulo:
return 15
default:
contract.Failf("unexpected binary expression %v", expr)
}
case *model.UnaryOpExpression:
return 17
case *model.FunctionCallExpression:
switch expr.Name {
case intrinsicAwait:
return 17
case intrinsicInterpolate:
return 22
default:
return 20
}
case *model.ForExpression, *model.IndexExpression, *model.RelativeTraversalExpression, *model.SplatExpression,
*model.TemplateJoinExpression:
return 20
case *model.AnonymousFunctionExpression, *model.LiteralValueExpression, *model.ObjectConsExpression,
*model.ScopeTraversalExpression, *model.TemplateExpression, *model.TupleConsExpression:
return 22
default:
contract.Failf("unexpected expression %v of type %T", expr, expr)
}
return 0
}
func (g *generator) GenAnonymousFunctionExpression(w io.Writer, expr *model.AnonymousFunctionExpression) {
switch len(expr.Signature.Parameters) {
case 0:
g.Fgen(w, "()")
case 1:
g.Fgenf(w, "%s", expr.Signature.Parameters[0].Name)
default:
g.Fgen(w, "([")
for i, p := range expr.Signature.Parameters {
if i > 0 {
g.Fgen(w, ", ")
}
g.Fgenf(w, "%s", p.Name)
}
g.Fgen(w, "])")
}
g.Fgenf(w, " => %.v", expr.Body)
}
func (g *generator) GenBinaryOpExpression(w io.Writer, expr *model.BinaryOpExpression) {
opstr, precedence := "", g.GetPrecedence(expr)
switch expr.Operation {
case hclsyntax.OpAdd:
opstr = "+"
case hclsyntax.OpDivide:
opstr = "/"
case hclsyntax.OpEqual:
opstr = "=="
case hclsyntax.OpGreaterThan:
opstr = ">"
case hclsyntax.OpGreaterThanOrEqual:
opstr = ">="
case hclsyntax.OpLessThan:
opstr = "<"
case hclsyntax.OpLessThanOrEqual:
opstr = "<="
case hclsyntax.OpLogicalAnd:
opstr = "&&"
case hclsyntax.OpLogicalOr:
opstr = "||"
case hclsyntax.OpModulo:
opstr = "%"
case hclsyntax.OpMultiply:
opstr = "*"
case hclsyntax.OpNotEqual:
opstr = "!="
case hclsyntax.OpSubtract:
opstr = "-"
default:
opstr, precedence = ",", 1
}
g.Fgenf(w, "%.[1]*[2]v %[3]v %.[1]*[4]o", precedence, expr.LeftOperand, opstr, expr.RightOperand)
}
func (g *generator) GenConditionalExpression(w io.Writer, expr *model.ConditionalExpression) {
g.Fgenf(w, "%.4v ? %.4v : %.4v", expr.Condition, expr.TrueResult, expr.FalseResult)
}
func (g *generator) GenForExpression(w io.Writer, expr *model.ForExpression) {
switch expr.Collection.Type().(type) {
case *model.ListType, *model.TupleType:
if expr.KeyVariable == nil {
g.Fgenf(w, "%.20v", expr.Collection)
} else {
g.Fgenf(w, "%.20v.map((v, k) => [k, v])", expr.Collection)
}
case *model.MapType, *model.ObjectType:
if expr.KeyVariable == nil {
g.Fgenf(w, "Object.values(%.v)", expr.Collection)
} else {
g.Fgenf(w, "Object.entries(%.v)", expr.Collection)
}
}
fnParams, reduceParams := expr.ValueVariable.Name, expr.ValueVariable.Name
if expr.KeyVariable != nil {
reduceParams = fmt.Sprintf("[%.v, %.v]", expr.KeyVariable.Name, expr.ValueVariable.Name)
fnParams = fmt.Sprintf("(%v)", reduceParams)
}
if expr.Condition != nil {
g.Fgenf(w, ".filter(%s => %.v)", fnParams, expr.Condition)
}
if expr.Key != nil {
// TODO(pdg): grouping
g.Fgenf(w, ".reduce((__obj, %s) => ({ ...__obj, [%.v]: %.v }))", reduceParams, expr.Key, expr.Value)
} else {
g.Fgenf(w, ".map(%s => (%.v))", fnParams, expr.Value)
}
}
func (g *generator) genApply(w io.Writer, expr *model.FunctionCallExpression) {
// Extract the list of outputs and the continuation expression from the `__apply` arguments.
applyArgs, then := pcl.ParseApplyCall(expr)
// If all of the arguments are promises, use promise methods. If any argument is an output, convert all other args
// to outputs and use output methods.
anyOutputs := false
for _, arg := range applyArgs {
if isOutputType(arg.Type()) {
anyOutputs = true
}
}
apply, all := "then", "Promise.all"
if anyOutputs {
apply, all = "apply", "pulumi.all"
}
if len(applyArgs) == 1 {
// If we only have a single output, just generate a normal `.apply` or `.then`.
g.Fgenf(w, "%.20v.%v(%.v)", applyArgs[0], apply, then)
} else {
// Otherwise, generate a call to `pulumi.all([]).apply()`.
g.Fgenf(w, "%v([", all)
for i, o := range applyArgs {
if i > 0 {
g.Fgen(w, ", ")
}
g.Fgenf(w, "%v", o)
}
g.Fgenf(w, "]).%v(%.v)", apply, then)
}
}
// functionName computes the NodeJS package, module, and name for the given function token.
func functionName(tokenArg model.Expression) (string, string, string, hcl.Diagnostics) {
token := tokenArg.(*model.TemplateExpression).Parts[0].(*model.LiteralValueExpression).Value.AsString()
tokenRange := tokenArg.SyntaxNode().Range()
// Compute the resource type from the Pulumi type token.
pkg, module, member, diagnostics := pcl.DecomposeToken(token, tokenRange)
return pkg, strings.ReplaceAll(module, "/", "."), member, diagnostics
}
func (g *generator) genRange(w io.Writer, call *model.FunctionCallExpression, entries bool) {
var from, to model.Expression
switch len(call.Args) {
case 1:
from, to = &model.LiteralValueExpression{Value: cty.NumberIntVal(0)}, call.Args[0]
case 2:
from, to = call.Args[0], call.Args[1]
default:
contract.Failf("expected range() to have exactly 1 or 2 args; got %v", len(call.Args))
}
genPrefix := func() { g.Fprint(w, "((from, to) => (new Array(to - from))") }
mapValue := "from + i"
genSuffix := func() { g.Fgenf(w, ")(%.v, %.v)", from, to) }
if litFrom, ok := from.(*model.LiteralValueExpression); ok {
fromV, err := convert.Convert(litFrom.Value, cty.Number)
contract.AssertNoErrorf(err, "conversion of %v to number failed", litFrom.Value.Type())
from, _ := fromV.AsBigFloat().Int64()
if litTo, ok := to.(*model.LiteralValueExpression); ok {
toV, err := convert.Convert(litTo.Value, cty.Number)
contract.AssertNoErrorf(err, "conversion of %v to number failed", litTo.Value.Type())
to, _ := toV.AsBigFloat().Int64()
if from == 0 {
mapValue = "i"
} else {
mapValue = fmt.Sprintf("%d + i", from)
}
genPrefix = func() { g.Fprintf(w, "(new Array(%d))", to-from) }
genSuffix = func() {}
} else if from == 0 {
genPrefix = func() { g.Fgenf(w, "(new Array(%.v))", to) }
mapValue = "i"
genSuffix = func() {}
}
}
if entries {
mapValue = fmt.Sprintf("{key: %[1]s, value: %[1]s}", mapValue)
}
genPrefix()
g.Fprintf(w, ".map((_, i) => %v)", mapValue)
genSuffix()
}
var functionImports = map[string][]string{
intrinsicInterpolate: {"@pulumi/pulumi"},
"fileArchive": {"@pulumi/pulumi"},
"remoteArchive": {"@pulumi/pulumi"},
"assetArchive": {"@pulumi/pulumi"},
"fileAsset": {"@pulumi/pulumi"},
"stringAsset": {"@pulumi/pulumi"},
"remoteAsset": {"@pulumi/pulumi"},
"filebase64": {"fs"},
"filebase64sha256": {"fs", "crypto"},
"readFile": {"fs"},
"readDir": {"fs"},
"sha1": {"crypto"},
}
func (g *generator) getFunctionImports(x *model.FunctionCallExpression) []string {
if x.Name != pcl.Invoke {
return functionImports[x.Name]
}
pkg, _, _, diags := functionName(x.Args[0])
contract.Assertf(len(diags) == 0, "unexpected diagnostics: %v", diags)
return []string{"@pulumi/" + pkg}
}
func enumName(enum *model.EnumType) (string, error) {
e, ok := pcl.GetSchemaForType(enum)
if !ok {
Enable perfsprint linter (#14813) <!--- Thanks so much for your contribution! If this is your first time contributing, please ensure that you have read the [CONTRIBUTING](https://github.com/pulumi/pulumi/blob/master/CONTRIBUTING.md) documentation. --> # Description <!--- Please include a summary of the change and which issue is fixed. Please also include relevant motivation and context. --> Prompted by a comment in another review: https://github.com/pulumi/pulumi/pull/14654#discussion_r1419995945 This lints that we don't use `fmt.Errorf` when `errors.New` will suffice, it also covers a load of other cases where `Sprintf` is sub-optimal. Most of these edits were made by running `perfsprint --fix`. ## Checklist - [x] I have run `make tidy` to update any new dependencies - [x] I have run `make lint` to verify my code passes the lint check - [x] I have formatted my code using `gofumpt` <!--- Please provide details if the checkbox below is to be left unchecked. --> - [ ] I have added tests that prove my fix is effective or that my feature works <!--- User-facing changes require a CHANGELOG entry. --> - [ ] I have run `make changelog` and committed the `changelog/pending/<file>` documenting my change <!-- If the change(s) in this PR is a modification of an existing call to the Pulumi Cloud, then the service should honor older versions of the CLI where this change would not exist. You must then bump the API version in /pkg/backend/httpstate/client/api.go, as well as add it to the service. --> - [ ] Yes, there are changes in this PR that warrants bumping the Pulumi Cloud API version <!-- @Pulumi employees: If yes, you must submit corresponding changes in the service repo. -->
2023-12-12 12:19:42 +00:00
return "", errors.New("Could not get associated enum")
}
pkgRef := e.(*schema.EnumType).PackageReference
return enumNameWithPackage(enum.Token, pkgRef)
}
func enumNameWithPackage(enumToken string, pkgRef schema.PackageReference) (string, error) {
components := strings.Split(enumToken, ":")
contract.Assertf(len(components) == 3, "malformed token %v", enumToken)
name := tokenToName(enumToken)
pkg := makeValidIdentifier(components[0])
if mod := components[1]; mod != "" && mod != "index" {
[program-gen] Fix enum resolution from types of the form Union[string, Enum] and emit fully qualified enum cases (#15696) # Description This PR improves enum type resolution from strings. When we try to resolve `Union[string, Enum]` for a string expression, we choose `string` because it is the more general type since not every string is assignable to `Enum`. However, here we spacial case strings that are actually part of that `Enum`. The result is that `pcl.LowerConversion` will choose `Enum` from `Union[string, Enum]` when the value of the input string is compatible with the enum. This greatly improves program-gen for all of typescript, python, csharp and go which now will emit the fully qualified enum cases instead of emitting strings. Closes https://github.com/pulumi/pulumi-dotnet/issues/41 which is supposed to be a duplicate of https://github.com/pulumi/pulumi-azure-native/issues/2616 but that is not the case (the former is about unions of objects, the latter is unions of enums and strings) ## Checklist - [ ] I have run `make tidy` to update any new dependencies - [x] I have run `make lint` to verify my code passes the lint check - [x] I have formatted my code using `gofumpt` <!--- Please provide details if the checkbox below is to be left unchecked. --> - [x] I have added tests that prove my fix is effective or that my feature works <!--- User-facing changes require a CHANGELOG entry. --> - [x] I have run `make changelog` and committed the `changelog/pending/<file>` documenting my change <!-- If the change(s) in this PR is a modification of an existing call to the Pulumi Cloud, then the service should honor older versions of the CLI where this change would not exist. You must then bump the API version in /pkg/backend/httpstate/client/api.go, as well as add it to the service. --> - [ ] Yes, there are changes in this PR that warrants bumping the Pulumi Cloud API version <!-- @Pulumi employees: If yes, you must submit corresponding changes in the service repo. -->
2024-03-15 17:49:12 +00:00
// if the token has the format {pkg}:{mod}/{name}:{Name}
// then we simplify into {pkg}:{mod}:{Name}
modParts := strings.Split(mod, "/")
if len(modParts) == 2 && strings.EqualFold(modParts[1], components[2]) {
mod = modParts[0]
}
if pkgRef != nil {
mod = moduleName(mod, pkgRef)
}
pkg += "." + mod
}
return fmt.Sprintf("%s.%s", pkg, name), nil
}
func (g *generator) genEntries(w io.Writer, expr *model.FunctionCallExpression) {
entriesArg := expr.Args[0]
entriesArgType := pcl.UnwrapOption(model.ResolveOutputs(entriesArg.Type()))
switch entriesArgType.(type) {
case *model.ListType, *model.TupleType:
if call, ok := expr.Args[0].(*model.FunctionCallExpression); ok && call.Name == "range" {
g.genRange(w, call, true)
return
}
// Mapping over a list with a tuple receiver accepts (value, index).
g.Fgenf(w, "%.20v.map((v, k)", expr.Args[0])
case *model.MapType, *model.ObjectType:
g.Fgenf(w, "Object.entries(%.v).map(([k, v])", expr.Args[0])
case *model.OpaqueType:
if entriesArgType.Equals(model.DynamicType) {
g.Fgenf(w, "Object.entries(%.v).map(([k, v])", expr.Args[0])
}
}
g.Fgenf(w, " => ({key: k, value: v}))")
}
func (g *generator) GenFunctionCallExpression(w io.Writer, expr *model.FunctionCallExpression) {
switch expr.Name {
case pcl.IntrinsicConvert:
from := expr.Args[0]
to := pcl.LowerConversion(from, expr.Signature.ReturnType)
output, isOutput := to.(*model.OutputType)
if isOutput {
to = output.ElementType
}
switch to := to.(type) {
case *model.EnumType:
if enum, err := enumName(to); err == nil {
if isOutput {
g.Fgenf(w, "%.v.apply((x) => %s[x])", from, enum)
} else {
diag := pcl.GenEnum(to, from, func(member *schema.Enum) {
memberTag, err := enumMemberName(tokenToName(to.Token), member)
contract.AssertNoErrorf(err, "Failed to get member name on enum '%s'", enum)
g.Fgenf(w, "%s.%s", enum, memberTag)
}, func(from model.Expression) {
g.Fgenf(w, "%s[%.v]", enum, from)
})
if diag != nil {
g.diagnostics = append(g.diagnostics, diag)
}
}
} else {
g.Fgenf(w, "%v", from)
}
default:
g.Fgenf(w, "%v", from)
}
case pcl.IntrinsicApply:
g.genApply(w, expr)
case intrinsicAwait:
g.Fgenf(w, "await %.17v", expr.Args[0])
case intrinsicInterpolate:
g.Fgen(w, "pulumi.interpolate`")
for _, part := range expr.Args {
if lit, ok := part.(*model.LiteralValueExpression); ok && model.StringType.AssignableFrom(lit.Type()) {
g.Fgen(w, lit.Value.AsString())
} else {
g.Fgenf(w, "${%.v}", part)
}
}
g.Fgen(w, "`")
case "element":
g.Fgenf(w, "%.20v[%.v]", expr.Args[0], expr.Args[1])
case "entries":
g.genEntries(w, expr)
case "fileArchive":
g.Fgenf(w, "new pulumi.asset.FileArchive(%.v)", expr.Args[0])
case "remoteArchive":
g.Fgenf(w, "new pulumi.asset.RemoteArchive(%.v)", expr.Args[0])
case "assetArchive":
g.Fgenf(w, "new pulumi.asset.AssetArchive(%.v)", expr.Args[0])
case "fileAsset":
g.Fgenf(w, "new pulumi.asset.FileAsset(%.v)", expr.Args[0])
case "stringAsset":
g.Fgenf(w, "new pulumi.asset.StringAsset(%.v)", expr.Args[0])
case "remoteAsset":
g.Fgenf(w, "new pulumi.asset.RemoteAsset(%.v)", expr.Args[0])
case "filebase64":
g.Fgenf(w, "fs.readFileSync(%v, { encoding: \"base64\" })", expr.Args[0])
case "filebase64sha256":
// Assuming the existence of the following helper method
g.Fgenf(w, "computeFilebase64sha256(%v)", expr.Args[0])
case "notImplemented":
g.Fgenf(w, "notImplemented(%v)", expr.Args[0])
case "singleOrNone":
g.Fgenf(w, "singleOrNone(%v)", expr.Args[0])
[program-gen] Fix generated utility functions for filebase64, filebase64sha256, sha1 and mimeType (#14857) # Description While writing program tests for generated helper utility functions `filebase64`, `filebase64sha256`, `sha1` and `mimeType` with the idea to increase code coverage, it turned out that those are completely broken in all of the languages containing syntax errors, missing imports and wrong indentation. This PR fixes them and extends the `functions` program to show how they now look like and to show that they compile. Also adding example usage of `stack()`, `project()` and `cwd()` in the test program. ## Checklist - [ ] I have run `make tidy` to update any new dependencies - [ ] I have run `make lint` to verify my code passes the lint check - [ ] I have formatted my code using `gofumpt` <!--- Please provide details if the checkbox below is to be left unchecked. --> - [x] I have added tests that prove my fix is effective or that my feature works <!--- User-facing changes require a CHANGELOG entry. --> - [x] I have run `make changelog` and committed the `changelog/pending/<file>` documenting my change <!-- If the change(s) in this PR is a modification of an existing call to the Pulumi Cloud, then the service should honor older versions of the CLI where this change would not exist. You must then bump the API version in /pkg/backend/httpstate/client/api.go, as well as add it to the service. --> - [ ] Yes, there are changes in this PR that warrants bumping the Pulumi Cloud API version <!-- @Pulumi employees: If yes, you must submit corresponding changes in the service repo. -->
2023-12-15 11:26:00 +00:00
case "mimeType":
g.Fgenf(w, "mimeType(%v)", expr.Args[0])
case pcl.Invoke:
pkg, module, fn, diags := functionName(expr.Args[0])
contract.Assertf(len(diags) == 0, "unexpected diagnostics: %v", diags)
if module != "" {
module = "." + module
}
isOut := pcl.IsOutputVersionInvokeCall(expr)
name := fmt.Sprintf("%s%s.%s", makeValidIdentifier(pkg), module, fn)
if isOut {
Enable perfsprint linter (#14813) <!--- Thanks so much for your contribution! If this is your first time contributing, please ensure that you have read the [CONTRIBUTING](https://github.com/pulumi/pulumi/blob/master/CONTRIBUTING.md) documentation. --> # Description <!--- Please include a summary of the change and which issue is fixed. Please also include relevant motivation and context. --> Prompted by a comment in another review: https://github.com/pulumi/pulumi/pull/14654#discussion_r1419995945 This lints that we don't use `fmt.Errorf` when `errors.New` will suffice, it also covers a load of other cases where `Sprintf` is sub-optimal. Most of these edits were made by running `perfsprint --fix`. ## Checklist - [x] I have run `make tidy` to update any new dependencies - [x] I have run `make lint` to verify my code passes the lint check - [x] I have formatted my code using `gofumpt` <!--- Please provide details if the checkbox below is to be left unchecked. --> - [ ] I have added tests that prove my fix is effective or that my feature works <!--- User-facing changes require a CHANGELOG entry. --> - [ ] I have run `make changelog` and committed the `changelog/pending/<file>` documenting my change <!-- If the change(s) in this PR is a modification of an existing call to the Pulumi Cloud, then the service should honor older versions of the CLI where this change would not exist. You must then bump the API version in /pkg/backend/httpstate/client/api.go, as well as add it to the service. --> - [ ] Yes, there are changes in this PR that warrants bumping the Pulumi Cloud API version <!-- @Pulumi employees: If yes, you must submit corresponding changes in the service repo. -->
2023-12-12 12:19:42 +00:00
name = name + "Output"
}
g.Fprintf(w, "%s(", name)
if len(expr.Args) >= 2 {
if expr.Signature.MultiArgumentInputs {
var invokeArgs *model.ObjectConsExpression
// extract invoke args in case we have the form invoke("token", __convert(args))
if converted, objectArgs, _ := pcl.RecognizeTypedObjectCons(expr.Args[1]); converted {
invokeArgs = objectArgs
} else {
// otherwise, we have the form invoke("token", args)
invokeArgs = expr.Args[1].(*model.ObjectConsExpression)
}
pcl.GenerateMultiArguments(g.Formatter, w, "undefined", invokeArgs, pcl.SortedFunctionParameters(expr))
} else {
g.Fgenf(w, "%.v", expr.Args[1])
}
}
if len(expr.Args) == 3 {
g.Fgenf(w, ", %.v", expr.Args[2])
}
g.Fprint(w, ")")
case "join":
g.Fgenf(w, "%.20v.join(%v)", expr.Args[1], expr.Args[0])
case "length":
g.Fgenf(w, "%.20v.length", expr.Args[0])
case "lookup":
g.Fgenf(w, "%v[%v]", expr.Args[0], expr.Args[1])
if len(expr.Args) == 3 {
g.Fgenf(w, " || %v", expr.Args[2])
}
case "range":
g.genRange(w, expr, false)
case "readFile":
g.Fgenf(w, "fs.readFileSync(%v, \"utf8\")", expr.Args[0])
case "readDir":
g.Fgenf(w, "fs.readdirSync(%v)", expr.Args[0])
case "secret":
g.Fgenf(w, "pulumi.secret(%v)", expr.Args[0])
2023-01-31 12:31:00 +00:00
case "unsecret":
g.Fgenf(w, "pulumi.unsecret(%v)", expr.Args[0])
case "split":
g.Fgenf(w, "%.20v.split(%v)", expr.Args[1], expr.Args[0])
case "toBase64":
g.Fgenf(w, "Buffer.from(%v).toString(\"base64\")", expr.Args[0])
2022-09-16 23:12:29 +00:00
case "fromBase64":
g.Fgenf(w, "Buffer.from(%v, \"base64\").toString(\"utf8\")", expr.Args[0])
case "toJSON":
[program-gen] Emit Output-returning JSON serialization methods without rewriting applies (#15371) ### Description A while ago we started implementing [specialized JSON serialization methods](https://github.com/pulumi/pulumi/issues/12519) for Pulumi programs which can accept nested outputs without having to rewrite and combine applies. - `Output.SerializeJson` in .NET - `pulumi.jsonStringify` in nodejs - `pulumi.Output.json_dumps` in Python This PR extends program-gen for TypeScript, C# and Python to start emitting these JSON serialization functions (when necessary). The PR special-cases the `toJSON` PCL function when rewriting applies so that nested outputs aren't rewritted. Example PCL program and generated results: > Also check out the downstream codegen tests to see improved generated examples ``` resource vpc "aws:ec2:Vpc" { cidrBlock = "10.100.0.0/16" instanceTenancy = "default" } resource policy "aws:iam/policy:Policy" { description = "test" policy = toJSON({ "Version" = "2012-10-17" "Interpolated" = "arn:${vpc.arn}:value" "Value" = vpc.id }) } ``` ### Generated TypeScript Before ```typescript import * as pulumi from "@pulumi/pulumi"; import * as aws from "@pulumi/aws"; const vpc = new aws.ec2.Vpc("vpc", { cidrBlock: "10.100.0.0/16", instanceTenancy: "default", }); const policy = new aws.iam.Policy("policy", { description: "test", policy: pulumi.all([vpc.arn, vpc.id]).apply(([arn, id]) => JSON.stringify({ Version: "2012-10-17", Interpolated: `arn:${arn}:value`, Value: id, })), }); ``` ### Generated TypeScript After ```typescript import * as pulumi from "@pulumi/pulumi"; import * as aws from "@pulumi/aws"; const vpc = new aws.ec2.Vpc("vpc", { cidrBlock: "10.100.0.0/16", instanceTenancy: "default", }); const policy = new aws.iam.Policy("policy", { description: "test", policy: pulumi.jsonStringify({ Version: "2012-10-17", Interpolated: pulumi.interpolate`arn:${vpc.arn}:value`, Value: vpc.id, }), }); ``` ### Generated Python Before ```python import pulumi import json import pulumi_aws as aws vpc = aws.ec2.Vpc("vpc", cidr_block="10.100.0.0/16", instance_tenancy="default") policy = aws.iam.Policy("policy", description="test", policy=pulumi.Output.all(vpc.arn, vpc.id).apply(lambda arn, id: json.dumps({ "Version": "2012-10-17", "Interpolated": f"arn:{arn}:value", "Value": id, }))) ``` ### Generated Python After ```python import pulumi import json import pulumi_aws as aws vpc = aws.ec2.Vpc("vpc", cidr_block="10.100.0.0/16", instance_tenancy="default") policy = aws.iam.Policy("policy", description="test", policy=pulumi.Output.json_dumps({ "Version": "2012-10-17", "Interpolated": vpc.arn.apply(lambda arn: f"arn:{arn}:value"), "Value": vpc.id, })) ``` ### Generated C# Before ```csharp using System.Collections.Generic; using System.Linq; using System.Text.Json; using Pulumi; using Aws = Pulumi.Aws; return await Deployment.RunAsync(() => { var vpc = new Aws.Ec2.Vpc("vpc", new() { CidrBlock = "10.100.0.0/16", InstanceTenancy = "default", }); var policy = new Aws.Iam.Policy("policy", new() { Description = "test", PolicyDocument = Output.Tuple(vpc.Arn, vpc.Id).Apply(values => { var arn = values.Item1; var id = values.Item2; return JsonSerializer.Serialize(new Dictionary<string, object?> { ["Version"] = "2012-10-17", ["Interpolated"] = $"arn:{arn}:value", ["Value"] = id, }); }), }); }); ``` ### Generated C# After ```csharp using System.Collections.Generic; using System.Linq; using System.Text.Json; using Pulumi; using Aws = Pulumi.Aws; return await Deployment.RunAsync(() => { var vpc = new Aws.Ec2.Vpc("vpc", new() { CidrBlock = "10.100.0.0/16", InstanceTenancy = "default", }); var policy = new Aws.Iam.Policy("policy", new() { Description = "test", PolicyDocument = Output.JsonSerialize(Output.Create(new Dictionary<string, object?> { ["Version"] = "2012-10-17", ["Interpolated"] = vpc.Arn.Apply(arn => $"arn:{arn}:value"), ["Value"] = vpc.Id, })), }); }); ``` ## Checklist - [ ] I have run `make tidy` to update any new dependencies - [x] I have run `make lint` to verify my code passes the lint check - [x] I have formatted my code using `gofumpt` <!--- Please provide details if the checkbox below is to be left unchecked. --> - [x] I have added tests that prove my fix is effective or that my feature works <!--- User-facing changes require a CHANGELOG entry. --> - [x] I have run `make changelog` and committed the `changelog/pending/<file>` documenting my change <!-- If the change(s) in this PR is a modification of an existing call to the Pulumi Cloud, then the service should honor older versions of the CLI where this change would not exist. You must then bump the API version in /pkg/backend/httpstate/client/api.go, as well as add it to the service. --> - [ ] Yes, there are changes in this PR that warrants bumping the Pulumi Cloud API version <!-- @Pulumi employees: If yes, you must submit corresponding changes in the service repo. -->
2024-02-20 15:48:46 +00:00
if model.ContainsOutputs(expr.Args[0].Type()) {
g.Fgenf(w, "pulumi.jsonStringify(%v)", expr.Args[0])
} else {
g.Fgenf(w, "JSON.stringify(%v)", expr.Args[0])
}
case "sha1":
g.Fgenf(w, "crypto.createHash('sha1').update(%v).digest('hex')", expr.Args[0])
case "stack":
g.Fgenf(w, "pulumi.getStack()")
case "project":
g.Fgenf(w, "pulumi.getProject()")
case "organization":
g.Fgenf(w, "pulumi.getOrganization()")
case "cwd":
g.Fgen(w, "process.cwd()")
Add StackReference conformance test (#15935) <!--- Thanks so much for your contribution! If this is your first time contributing, please ensure that you have read the [CONTRIBUTING](https://github.com/pulumi/pulumi/blob/master/CONTRIBUTING.md) documentation. --> # Description <!--- Please include a summary of the change and which issue is fixed. Please also include relevant motivation and context. --> Fixes https://github.com/pulumi/pulumi/issues/15932. This adds a conformance test that checks that StackReferences work. Tests a plain string and a secret string output. To support this test we add a new intrinsic `getOutput` that takes a stack reference resource and a string and calls the `get_output/getOutput/GetOutput` method on the stack reference resource type. ## Checklist - [x] I have run `make tidy` to update any new dependencies - [x] I have run `make lint` to verify my code passes the lint check - [ ] I have formatted my code using `gofumpt` <!--- Please provide details if the checkbox below is to be left unchecked. --> - [x] I have added tests that prove my fix is effective or that my feature works <!--- User-facing changes require a CHANGELOG entry. --> - [ ] I have run `make changelog` and committed the `changelog/pending/<file>` documenting my change <!-- If the change(s) in this PR is a modification of an existing call to the Pulumi Cloud, then the service should honor older versions of the CLI where this change would not exist. You must then bump the API version in /pkg/backend/httpstate/client/api.go, as well as add it to the service. --> - [ ] Yes, there are changes in this PR that warrants bumping the Pulumi Cloud API version <!-- @Pulumi employees: If yes, you must submit corresponding changes in the service repo. -->
2024-04-16 11:13:25 +00:00
case "getOutput":
g.Fgenf(w, "%s.getOutput(%v)", expr.Args[0], expr.Args[1])
default:
var rng hcl.Range
if expr.Syntax != nil {
rng = expr.Syntax.Range()
}
g.genNYI(w, "FunctionCallExpression: %v (%v)", expr.Name, rng)
}
}
func (g *generator) GenIndexExpression(w io.Writer, expr *model.IndexExpression) {
g.Fgenf(w, "%.20v[%.v]", expr.Collection, expr.Key)
}
func escapeRune(c rune) string {
if uint(c) <= 0xFF {
return fmt.Sprintf("\\x%02x", c)
} else if uint(c) <= 0xFFFF {
return fmt.Sprintf("\\u%04x", c)
}
return fmt.Sprintf("\\u{%x}", c)
}
func (g *generator) genStringLiteral(w io.Writer, v string) {
builder := strings.Builder{}
newlines := strings.Count(v, "\n")
if newlines == 0 || newlines == 1 && (v[0] == '\n' || v[len(v)-1] == '\n') {
// This string either does not contain newlines or contains a single leading or trailing newline, so we'll
// Generate a normal string literal. Quotes, backslashes, and newlines will be escaped in conformance with
// ECMA-262 11.8.4 ("String Literals").
builder.WriteRune('"')
for _, c := range v {
if c == '"' || c == '\\' {
builder.WriteRune('\\')
builder.WriteRune(c)
} else if c == '\n' {
builder.WriteString(`\n`)
} else if unicode.IsPrint(c) {
builder.WriteRune(c)
} else {
// This is a non-printable character. We'll emit an escape sequence for it.
builder.WriteString(escapeRune(c))
}
}
builder.WriteRune('"')
} else {
// This string does contain newlines, so we'll Generate a template string literal. "${", backquotes, and
// backslashes will be escaped in conformance with ECMA-262 11.8.6 ("Template Literal Lexical Components").
runes := []rune(v)
builder.WriteRune('`')
for i, c := range runes {
if c == '`' || c == '\\' {
builder.WriteRune('\\')
builder.WriteRune(c)
} else if c == '$' {
if i < len(runes)-1 && runes[i+1] == '{' {
builder.WriteRune('\\')
builder.WriteRune('$')
}
} else if c == '\n' {
builder.WriteRune('\n')
} else if unicode.IsPrint(c) {
builder.WriteRune(c)
} else {
// This is a non-printable character. We'll emit an escape sequence for it.
builder.WriteString(escapeRune(c))
}
}
builder.WriteRune('`')
}
g.Fgenf(w, "%s", builder.String())
}
func (g *generator) GenLiteralValueExpression(w io.Writer, expr *model.LiteralValueExpression) {
typ := expr.Type()
if cns, ok := typ.(*model.ConstType); ok {
typ = cns.Type
}
switch typ {
case model.BoolType:
g.Fgenf(w, "%v", expr.Value.True())
case model.NoneType:
g.Fgen(w, "undefined")
case model.NumberType:
bf := expr.Value.AsBigFloat()
if i, acc := bf.Int64(); acc == big.Exact {
g.Fgenf(w, "%d", i)
} else {
f, _ := bf.Float64()
g.Fgenf(w, "%g", f)
}
case model.StringType:
g.genStringLiteral(w, expr.Value.AsString())
default:
contract.Failf("unexpected literal type in GenLiteralValueExpression: %v (%v)", expr.Type(),
expr.SyntaxNode().Range())
}
}
func (g *generator) literalKey(x model.Expression) (string, bool) {
strKey := ""
switch x := x.(type) {
case *model.LiteralValueExpression:
if model.StringType.AssignableFrom(x.Type()) {
strKey = x.Value.AsString()
break
}
var buf bytes.Buffer
g.GenLiteralValueExpression(&buf, x)
return buf.String(), true
case *model.TemplateExpression:
if len(x.Parts) == 1 {
if lit, ok := x.Parts[0].(*model.LiteralValueExpression); ok && model.StringType.AssignableFrom(lit.Type()) {
strKey = lit.Value.AsString()
break
}
}
return "", false
default:
return "", false
}
if isLegalIdentifier(strKey) {
return strKey, true
}
return fmt.Sprintf("%q", strKey), true
}
func (g *generator) GenObjectConsExpression(w io.Writer, expr *model.ObjectConsExpression) {
if len(expr.Items) == 0 {
g.Fgen(w, "{}")
} else {
g.Fgen(w, "{")
g.Indented(func() {
for _, item := range expr.Items {
g.Fgenf(w, "\n%s", g.Indent)
if lit, ok := g.literalKey(item.Key); ok {
g.Fgenf(w, "%s", lit)
} else {
g.Fgenf(w, "[%.v]", item.Key)
}
g.Fgenf(w, ": %.v,", item.Value)
}
})
g.Fgenf(w, "\n%s}", g.Indent)
}
}
func (g *generator) genRelativeTraversal(w io.Writer, traversal hcl.Traversal, parts []model.Traversable) {
for i, part := range traversal {
var key cty.Value
switch part := part.(type) {
case hcl.TraverseAttr:
key = cty.StringVal(part.Name)
case hcl.TraverseIndex:
key = part.Key
default:
contract.Failf("unexpected traversal part of type %T (%v)", part, part.SourceRange())
}
2022-11-28 18:56:04 +00:00
var indexPrefix string
if model.IsOptionalType(model.GetTraversableType(parts[i])) {
g.Fgen(w, "?")
2022-11-28 18:56:04 +00:00
// `expr?[expr]` is not valid typescript, since it looks like a ternary
// operator.
//
// Typescript solves this by inserting a `.` in before the `[`: `expr?.[expr]`
//
// We need to do the same when generating index based expressions.
indexPrefix = "."
}
genIndex := func(inner string, value interface{}) {
g.Fgenf(w, "%s["+inner+"]", indexPrefix, value)
}
switch key.Type() {
case cty.String:
keyVal := key.AsString()
if isLegalIdentifier(keyVal) {
g.Fgenf(w, ".%s", keyVal)
} else {
2022-11-28 18:56:04 +00:00
genIndex("%q", keyVal)
}
case cty.Number:
idx, _ := key.AsBigFloat().Int64()
2022-11-28 18:56:04 +00:00
genIndex("%d", idx)
default:
2022-11-28 18:56:04 +00:00
genIndex("%q", key.AsString())
}
}
}
func (g *generator) GenRelativeTraversalExpression(w io.Writer, expr *model.RelativeTraversalExpression) {
g.Fgenf(w, "%.20v", expr.Source)
g.genRelativeTraversal(w, expr.Traversal, expr.Parts)
}
func (g *generator) GenScopeTraversalExpression(w io.Writer, expr *model.ScopeTraversalExpression) {
rootName := makeValidIdentifier(expr.RootName)
2023-03-08 23:34:15 +00:00
if g.isComponent {
if expr.RootName == "this" {
// special case for parent: this
g.Fgenf(w, "%s", expr.RootName)
return
}
configVars := map[string]*pcl.ConfigVariable{}
for _, configVar := range g.program.ConfigVariables() {
configVars[configVar.Name()] = configVar
}
if _, isConfig := configVars[expr.RootName]; isConfig {
if _, configReference := expr.Parts[0].(*pcl.ConfigVariable); configReference {
Enable perfsprint linter (#14813) <!--- Thanks so much for your contribution! If this is your first time contributing, please ensure that you have read the [CONTRIBUTING](https://github.com/pulumi/pulumi/blob/master/CONTRIBUTING.md) documentation. --> # Description <!--- Please include a summary of the change and which issue is fixed. Please also include relevant motivation and context. --> Prompted by a comment in another review: https://github.com/pulumi/pulumi/pull/14654#discussion_r1419995945 This lints that we don't use `fmt.Errorf` when `errors.New` will suffice, it also covers a load of other cases where `Sprintf` is sub-optimal. Most of these edits were made by running `perfsprint --fix`. ## Checklist - [x] I have run `make tidy` to update any new dependencies - [x] I have run `make lint` to verify my code passes the lint check - [x] I have formatted my code using `gofumpt` <!--- Please provide details if the checkbox below is to be left unchecked. --> - [ ] I have added tests that prove my fix is effective or that my feature works <!--- User-facing changes require a CHANGELOG entry. --> - [ ] I have run `make changelog` and committed the `changelog/pending/<file>` documenting my change <!-- If the change(s) in this PR is a modification of an existing call to the Pulumi Cloud, then the service should honor older versions of the CLI where this change would not exist. You must then bump the API version in /pkg/backend/httpstate/client/api.go, as well as add it to the service. --> - [ ] Yes, there are changes in this PR that warrants bumping the Pulumi Cloud API version <!-- @Pulumi employees: If yes, you must submit corresponding changes in the service repo. -->
2023-12-12 12:19:42 +00:00
rootName = "args." + expr.RootName
2023-03-08 23:34:15 +00:00
}
}
}
if _, ok := expr.Parts[0].(*model.SplatVariable); ok {
rootName = "__item"
}
g.Fgen(w, rootName)
g.genRelativeTraversal(w, expr.Traversal.SimpleSplit().Rel, expr.Parts)
}
func (g *generator) GenSplatExpression(w io.Writer, expr *model.SplatExpression) {
g.Fgenf(w, "%.20v.map(__item => %.v)", expr.Source, expr.Each)
}
func (g *generator) GenTemplateExpression(w io.Writer, expr *model.TemplateExpression) {
if len(expr.Parts) == 1 {
if lit, ok := expr.Parts[0].(*model.LiteralValueExpression); ok && model.StringType.AssignableFrom(lit.Type()) {
g.GenLiteralValueExpression(w, lit)
return
}
}
g.Fgen(w, "`")
for _, expr := range expr.Parts {
if lit, ok := expr.(*model.LiteralValueExpression); ok && model.StringType.AssignableFrom(lit.Type()) {
g.Fgen(w, lit.Value.AsString())
} else {
g.Fgenf(w, "${%.v}", expr)
}
}
g.Fgen(w, "`")
}
func (g *generator) GenTemplateJoinExpression(w io.Writer, expr *model.TemplateJoinExpression) {
g.genNYI(w, "TemplateJoinExpression")
}
func (g *generator) GenTupleConsExpression(w io.Writer, expr *model.TupleConsExpression) {
switch len(expr.Expressions) {
case 0:
g.Fgen(w, "[]")
case 1:
g.Fgenf(w, "[%.v]", expr.Expressions[0])
default:
g.Fgen(w, "[")
g.Indented(func() {
for _, v := range expr.Expressions {
g.Fgenf(w, "\n%s%.v,", g.Indent, v)
}
})
g.Fgen(w, "\n", g.Indent, "]")
}
}
func (g *generator) GenUnaryOpExpression(w io.Writer, expr *model.UnaryOpExpression) {
opstr, precedence := "", g.GetPrecedence(expr)
switch expr.Operation {
case hclsyntax.OpLogicalNot:
opstr = "!"
case hclsyntax.OpNegate:
opstr = "-"
}
g.Fgenf(w, "%[2]v%.[1]*[3]v", precedence, opstr, expr.Operand)
}