Merge pull request #4455 from pulumi/komalali/increase-grpc-message-size

GH-4155 - Increase gRPC message size
This commit is contained in:
Komal 2020-04-24 19:02:49 -07:00 committed by GitHub
commit 21ccfb8de7
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
36 changed files with 383 additions and 29 deletions

View file

@ -22,6 +22,9 @@ CHANGELOG
- Ensure generated Python module names are keyword-safe.
[#4473](https://github.com/pulumi/pulumi/pull/4473)
- Increase the MaxCallRecvMsgSize for all RPC calls.
[#4455](https://github.com/pulumi/pulumi/pull/4455)
## 2.0.0 (2020-04-16)
- CLI behavior change. Commands in non-interactive mode (i.e. when `pulumi` has its output piped to

View file

@ -16,6 +16,7 @@ package deploytest
import (
"github.com/pkg/errors"
"github.com/pulumi/pulumi/sdk/v2/go/common/util/rpcutil"
"google.golang.org/grpc"
"github.com/pulumi/pulumi/sdk/v2/go/common/resource/plugin"
@ -48,7 +49,11 @@ func (p *languageRuntime) GetRequiredPlugins(info plugin.ProgInfo) ([]workspace.
func (p *languageRuntime) Run(info plugin.RunInfo) (string, bool, error) {
// Connect to the resource monitor and create an appropriate client.
conn, err := grpc.Dial(info.MonitorAddress, grpc.WithInsecure())
conn, err := grpc.Dial(
info.MonitorAddress,
grpc.WithInsecure(),
rpcutil.GrpcChannelOptions(),
)
if err != nil {
return "", false, errors.Wrapf(err, "could not connect to resource monitor")
}

View file

@ -33,9 +33,12 @@ type HostClient struct {
// NewHostClient dials the target address, connects over gRPC, and returns a client interface.
func NewHostClient(addr string) (*HostClient, error) {
conn, err := grpc.Dial(addr, grpc.WithInsecure(), grpc.WithUnaryInterceptor(
rpcutil.OpenTracingClientInterceptor(),
))
conn, err := grpc.Dial(
addr,
grpc.WithInsecure(),
grpc.WithUnaryInterceptor(rpcutil.OpenTracingClientInterceptor()),
rpcutil.GrpcChannelOptions(),
)
if err != nil {
return nil, err
}

View file

@ -1,6 +1,7 @@
// Copyright 2016-2020, Pulumi Corporation
using System.Threading.Tasks;
using System.Collections.Generic;
using Grpc.Core;
using Pulumirpc;
@ -12,7 +13,10 @@ namespace Pulumi
public GrpcEngine(string engine)
{
this._engine = new Engine.EngineClient(new Channel(engine, ChannelCredentials.Insecure));
// maxRpcMessageSize raises the gRPC Max Message size from `4194304` (4mb) to `419430400` (400mb)
var maxRpcMessageSize = 400 * 1024 * 1024;
var grpcChannelOptions = new List<ChannelOption> { new ChannelOption(ChannelOptions.MaxReceiveMessageLength, maxRpcMessageSize)};
this._engine = new Engine.EngineClient(new Channel(engine, ChannelCredentials.Insecure, grpcChannelOptions));
}
public async Task LogAsync(LogRequest request)

View file

@ -1,6 +1,7 @@
// Copyright 2016-2020, Pulumi Corporation
using System.Threading.Tasks;
using System.Collections.Generic;
using Grpc.Core;
using Pulumirpc;
@ -12,7 +13,10 @@ namespace Pulumi
public GrpcMonitor(string monitor)
{
this._client = new ResourceMonitor.ResourceMonitorClient(new Channel(monitor, ChannelCredentials.Insecure));
// maxRpcMessageSize raises the gRPC Max Message size from `4194304` (4mb) to `419430400` (400mb)
var maxRpcMessageSize = 400 * 1024 * 1024;
var grpcChannelOptions = new List<ChannelOption> { new ChannelOption(ChannelOptions.MaxReceiveMessageLength, maxRpcMessageSize)};
this._client = new ResourceMonitor.ResourceMonitorClient(new Channel(monitor, ChannelCredentials.Insecure, grpcChannelOptions));
}
public async Task<InvokeResponse> InvokeAsync(InvokeRequest request)

View file

@ -134,7 +134,11 @@ func (host *dotnetLanguageHost) GetRequiredPlugins(
}
// Make a connection to the real engine that we will log messages to.
conn, err := grpc.Dial(host.engineAddress, grpc.WithInsecure())
conn, err := grpc.Dial(
host.engineAddress,
grpc.WithInsecure(),
rpcutil.GrpcChannelOptions(),
)
if err != nil {
return nil, errors.Wrapf(err, "language host could not make connection to engine")
}

View file

@ -59,9 +59,6 @@ type plugin struct {
// pluginRPCConnectionTimeout dictates how long we wait for the plugin's RPC to become available.
var pluginRPCConnectionTimeout = time.Second * 10
// pluginRPCMaxMessageSize raises the gRPC Max Message size from `4194304` to `419430400`
var pluginRPCMaxMessageSize = 1024 * 1024 * 400
// A unique ID provided to the output stream of each plugin. This allows the output of the plugin
// to be streamed to the display, while still allowing that output to be sent a small piece at a
// time.
@ -180,13 +177,13 @@ func newPlugin(ctx *Context, pwd, bin, prefix string, args, env []string) (*plug
plug.stdoutDone = stdoutDone
go runtrace(plug.Stdout, false, stdoutDone)
// We want to increase the default message size as per pulumi/pulumi#2319
messageSizeOpts := grpc.WithDefaultCallOptions(grpc.MaxCallRecvMsgSize(pluginRPCMaxMessageSize))
// Now that we have the port, go ahead and create a gRPC client connection to it.
conn, err := grpc.Dial("127.0.0.1:"+port, grpc.WithInsecure(), grpc.WithUnaryInterceptor(
rpcutil.OpenTracingClientInterceptor(),
), messageSizeOpts)
conn, err := grpc.Dial(
"127.0.0.1:"+port,
grpc.WithInsecure(),
grpc.WithUnaryInterceptor(rpcutil.OpenTracingClientInterceptor()),
rpcutil.GrpcChannelOptions(),
)
if err != nil {
return nil, errors.Wrapf(err, "could not dial plugin [%v] over RPC", bin)
}
@ -244,7 +241,7 @@ func execPlugin(bin string, pluginArgs []string, pwd string, env []string) (*plu
// Flow the logging information if set.
if logging.LogFlow {
if logging.LogToStderr {
args = append(args, "-logtostderr")
args = append(args, "--logtostderr")
}
if logging.Verbose > 0 {
args = append(args, "-v="+strconv.Itoa(logging.Verbose))

View file

@ -25,6 +25,15 @@ import (
"google.golang.org/grpc/reflection"
)
// maxRPCMessageSize raises the gRPC Max Message size from `4194304` (4mb) to `419430400` (400mb)
var maxRPCMessageSize = 1024 * 1024 * 400
// GrpcChannelOptions returns the defaultCallOptions with the max_receive_message_length increased to 400mb
// We want to increase the default message size as per pulumi/pulumi#2319
func GrpcChannelOptions() grpc.DialOption {
return grpc.WithDefaultCallOptions(grpc.MaxCallRecvMsgSize(maxRPCMessageSize))
}
// IsBenignCloseErr returns true if the error is "expected" upon shutdown of the server.
func IsBenignCloseErr(err error) bool {
msg := err.Error()
@ -47,7 +56,10 @@ func Serve(port int, cancel chan bool, registers []func(*grpc.Server) error,
}
// Now new up a gRPC server and register any RPC interfaces the caller wants.
srv := grpc.NewServer(grpc.UnaryInterceptor(OpenTracingServerInterceptor(parentSpan)))
srv := grpc.NewServer(
grpc.UnaryInterceptor(OpenTracingServerInterceptor(parentSpan)),
grpc.MaxRecvMsgSize(maxRPCMessageSize),
)
for _, register := range registers {
if err := register(srv); err != nil {
return port, nil, errors.Errorf("failed to register RPC handler: %v", err)

View file

@ -33,6 +33,7 @@ import (
"github.com/pulumi/pulumi/sdk/v2/go/common/resource/plugin"
"github.com/pulumi/pulumi/sdk/v2/go/common/util/contract"
"github.com/pulumi/pulumi/sdk/v2/go/common/util/logging"
"github.com/pulumi/pulumi/sdk/v2/go/common/util/rpcutil"
pulumirpc "github.com/pulumi/pulumi/sdk/v2/proto/go"
)
@ -60,7 +61,11 @@ func NewContext(ctx context.Context, info RunInfo) (*Context, error) {
var monitorConn *grpc.ClientConn
var monitor pulumirpc.ResourceMonitorClient
if addr := info.MonitorAddr; addr != "" {
conn, err := grpc.Dial(info.MonitorAddr, grpc.WithInsecure())
conn, err := grpc.Dial(
info.MonitorAddr,
grpc.WithInsecure(),
rpcutil.GrpcChannelOptions(),
)
if err != nil {
return nil, fmt.Errorf("connecting to resource monitor over RPC: %w", err)
}
@ -71,7 +76,11 @@ func NewContext(ctx context.Context, info RunInfo) (*Context, error) {
var engineConn *grpc.ClientConn
var engine pulumirpc.EngineClient
if addr := info.EngineAddr; addr != "" {
conn, err := grpc.Dial(info.EngineAddr, grpc.WithInsecure())
conn, err := grpc.Dial(
info.EngineAddr,
grpc.WithInsecure(),
rpcutil.GrpcChannelOptions(),
)
if err != nil {
return nil, fmt.Errorf("connecting to engine over RPC: %w", err)
}

View file

@ -33,6 +33,9 @@ const statusproto = require("../../proto/status_pb.js");
const providerKey: string = "__provider";
// maxRPCMessageSize raises the gRPC Max Message size from `4194304` (4mb) to `419430400` (400mb)
const maxRPCMessageSize: number = 1024 * 1024 * 400;
// We track all uncaught errors here. If we have any, we will make sure we always have a non-0 exit
// code.
const uncaughtErrors = new Set<Error>();
@ -351,7 +354,9 @@ export async function main(args: string[]) {
const engineAddr: string = args[0];
// Finally connect up the gRPC client/server and listen for incoming requests.
const server = new grpc.Server();
const server = new grpc.Server({
"grpc.max_receive_message_length": maxRPCMessageSize,
});
server.addService(provrpc.ResourceProviderService, {
cancel: cancelRPC,
configure: configureRPC,

View file

@ -397,7 +397,11 @@ func (host *nodeLanguageHost) Run(ctx context.Context, req *pulumirpc.RunRequest
tracingSpan := opentracing.SpanFromContext(ctx)
// Make a connection to the real monitor that we will forward messages to.
conn, err := grpc.Dial(req.GetMonitorAddress(), grpc.WithInsecure())
conn, err := grpc.Dial(
req.GetMonitorAddress(),
grpc.WithInsecure(),
rpcutil.GrpcChannelOptions(),
)
if err != nil {
return nil, err
}

View file

@ -25,6 +25,10 @@ const resrpc = require("../proto/resource_grpc_pb.js");
const resproto = require("../proto/resource_pb.js");
const structproto = require("google-protobuf/google/protobuf/struct_pb.js");
// maxRPCMessageSize raises the gRPC Max Message size from `4194304` (4mb) to `419430400` (400mb)
const maxRPCMessageSize: number = 1024 * 1024 * 400;
const grpcChannelOptions = { "grpc.max_receive_message_length": maxRPCMessageSize };
/**
* excessiveDebugOutput enables, well, pretty excessive debug output pertaining to resources and properties.
*/
@ -187,7 +191,11 @@ export function getMonitor(): Object | undefined {
const addr = options.monitorAddr;
if (addr) {
// Lazily initialize the RPC connection to the monitor.
monitor = new resrpc.ResourceMonitorClient(addr, grpc.credentials.createInsecure());
monitor = new resrpc.ResourceMonitorClient(
addr,
grpc.credentials.createInsecure(),
grpcChannelOptions,
);
} else {
// If test mode isn't enabled, we can't run the program without an engine.
requireTestModeEnabled();
@ -228,7 +236,11 @@ export function getEngine(): Object | undefined {
const addr = options.engineAddr;
if (addr) {
// Lazily initialize the RPC connection to the engine.
engine = new engrpc.EngineClient(addr, grpc.credentials.createInsecure());
engine = new engrpc.EngineClient(
addr,
grpc.credentials.createInsecure(),
grpcChannelOptions,
);
}
}
return engine;

View file

@ -0,0 +1,17 @@
// This tests the creation of a resource that contains a very large string.
// In particular we are testing sending large message sizes (>4mb) over an RPC call.
let pulumi = require("../../../../../");
// Read the file contents to create a very large string (5mb)
const longString = 'a'.repeat(1024 * 1024 * 5)
class MyResource extends pulumi.CustomResource {
constructor(name) {
super("test:index:MyLargeStringResource", name, {
"largeStringProp": longString,
});
}
}
new MyResource("testResource1");

View file

@ -1136,6 +1136,24 @@ describe("rpc", () => {
return { urn: makeUrn(t, name), id: undefined, props: undefined };
},
},
// Create a resource with a large string to test grpcMaxMessageSize increase.
"large_resource": {
program: path.join(base, "065.large_resource"),
expectResourceCount: 1,
registerResource: (ctx: any, dryrun: boolean, t: string, name: string, res: any) => {
const longString = "a".repeat(1024 * 1024 * 5);
assert.strictEqual(t, "test:index:MyLargeStringResource");
assert.strictEqual(name, "testResource1");
assert.deepEqual(res, { "largeStringProp": longString });
return {
urn: makeUrn(t, name),
id: name,
props: {
"largeStringProp": "a".repeat(1024 * 1024 * 5),
},
};
},
},
};
for (const casename of Object.keys(cases)) {

View file

@ -28,6 +28,11 @@ from pulumi.dynamic import ResourceProvider
_ONE_DAY_IN_SECONDS = 60 * 60 * 24
PROVIDER_KEY = "__provider"
# _MAX_RPC_MESSAGE_SIZE raises the gRPC Max Message size from `4194304` (4mb) to `419430400` (400mb)
_MAX_RPC_MESSAGE_SIZE = 1024 * 1024 * 400
_GRPC_CHANNEL_OPTIONS = [('grpc.max_receive_message_length', _MAX_RPC_MESSAGE_SIZE)]
def get_provider(props) -> ResourceProvider:
byts = base64.b64decode(props[PROVIDER_KEY])
return dill.loads(byts)
@ -168,7 +173,10 @@ class DynamicResourceProviderServicer(ResourceProviderServicer):
def main():
monitor = DynamicResourceProviderServicer()
server = grpc.server(futures.ThreadPoolExecutor(max_workers=4))
server = grpc.server(
futures.ThreadPoolExecutor(max_workers=4),
options=_GRPC_CHANNEL_OPTIONS
)
provider_pb2_grpc.add_ResourceProviderServicer_to_server(monitor, server)
port = server.add_insecure_port(address="0.0.0.0:0")
server.start()

View file

@ -27,6 +27,10 @@ from ..errors import RunError
if TYPE_CHECKING:
from ..resource import Resource
# _MAX_RPC_MESSAGE_SIZE raises the gRPC Max Message size from `4194304` (4mb) to `419430400` (400mb)
_MAX_RPC_MESSAGE_SIZE = 1024 * 1024 * 400
_GRPC_CHANNEL_OPTIONS = [('grpc.max_receive_message_length', _MAX_RPC_MESSAGE_SIZE)]
class Settings:
monitor: Optional[Union[resource_pb2_grpc.ResourceMonitorStub, Any]]
engine: Optional[Union[engine_pb2_grpc.EngineStub, Any]]
@ -63,17 +67,22 @@ class Settings:
if self.legacy_apply_enabled is None:
self.legacy_apply_enabled = os.getenv("PULUMI_ENABLE_LEGACY_APPLY", "false") == "true"
# Actually connect to the monitor/engine over gRPC.
if monitor is not None:
if isinstance(monitor, str):
self.monitor = resource_pb2_grpc.ResourceMonitorStub(grpc.insecure_channel(monitor))
self.monitor = resource_pb2_grpc.ResourceMonitorStub(
grpc.insecure_channel(monitor, options=_GRPC_CHANNEL_OPTIONS),
)
else:
self.monitor = monitor
else:
self.monitor = None
if engine:
if isinstance(engine, str):
self.engine = engine_pb2_grpc.EngineStub(grpc.insecure_channel(engine))
self.engine = engine_pb2_grpc.EngineStub(
grpc.insecure_channel(engine, options=_GRPC_CHANNEL_OPTIONS),
)
else:
self.engine = engine
else:

View file

@ -0,0 +1,13 @@
# Copyright 2016-2018, 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.

View file

@ -0,0 +1,30 @@
# Copyright 2016-2018, 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.
import functools
from pulumi import CustomResource
long_string = "a" * 1024 * 1024 * 5
class MyResource(CustomResource):
def __init__(self, name):
CustomResource.__init__(self, "test:index:MyLargeStringResource", name, props={
"largeStringProp": long_string,
})
def assert_eq(lhs, rhs):
assert lhs == rhs
res = MyResource("testResource1")
res.largeStringProp.apply(functools.partial(assert_eq, long_string))

View file

@ -0,0 +1,41 @@
# Copyright 2016-2018, 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.
from os import path
import unittest
from ..util import LanghostTest
# This tests the creation of a resource that contains a very large string.
# In particular we are testing sending large message sizes (>4mb) over an RPC call.
long_string = "a" * 1024 * 1024 * 5
class LargeResourceTest(LanghostTest):
def test_large_resource(self):
self.run_test(
program=path.join(self.base_path(), "large_resource"),
expected_resource_count=1)
def register_resource(self, _ctx, _dry_run, ty, name, resource,
_dependencies, _parent, _custom, _protect, _provider, _property_deps, _delete_before_replace,
_ignore_changes, _version):
self.assertEqual(ty, "test:index:MyLargeStringResource")
self.assertEqual(name, "testResource1")
return {
"urn": self.make_urn(ty, name),
"id": name,
"object": {
"largeStringProp": long_string,
}
}

View file

@ -33,6 +33,9 @@ from google.protobuf import empty_pb2, struct_pb2
# test output. Just turn it off.
logging.disable(level=logging.CRITICAL)
# _MAX_RPC_MESSAGE_SIZE raises the gRPC Max Message size from `4194304` (4mb) to `419430400` (400mb)
_MAX_RPC_MESSAGE_SIZE = 1024 * 1024 * 400
_GRPC_CHANNEL_OPTIONS = [('grpc.max_receive_message_length', _MAX_RPC_MESSAGE_SIZE)]
class LanghostMockResourceMonitor(proto.ResourceMonitorServicer):
"""
@ -213,7 +216,7 @@ class LanghostTest(unittest.TestCase):
langhost = self._create_language_host(monitor.port)
# Run the program with the langhost we just launched.
with grpc.insecure_channel("localhost:%d" % langhost.port) as channel:
with grpc.insecure_channel("localhost:%d" % langhost.port, options=_GRPC_CHANNEL_OPTIONS) as channel:
grpc.channel_ready_future(channel).result()
stub = language_pb2_grpc.LanguageRuntimeStub(channel)
result = self._run_program(stub, monitor, project, stack,
@ -297,7 +300,7 @@ class LanghostTest(unittest.TestCase):
def _create_mock_resource_monitor(self, dryrun):
monitor = LanghostMockResourceMonitor(self, dryrun)
engine = MockEngine()
server = grpc.server(futures.ThreadPoolExecutor(max_workers=4))
server = grpc.server(futures.ThreadPoolExecutor(max_workers=4), options=_GRPC_CHANNEL_OPTIONS)
resource_pb2_grpc.add_ResourceMonitorServicer_to_server(monitor, server)
engine_pb2_grpc.add_EngineServicer_to_server(engine, server)

View file

@ -1605,3 +1605,45 @@ func TestPartialValuesPython(t *testing.T) {
AllowEmptyPreviewChanges: true,
})
}
// The following 4 tests are testing to ensure that we can make RPC calls >4mb
// Issue: https://github.com/pulumi/pulumi/issues/4155
//Tests a resource with a large (>4mb) string prop in Node.js
func TestLargeResourceNode(t *testing.T) {
if runtime.GOOS == WindowsOS {
t.Skip("Temporarily skipping test on Windows - pulumi/pulumi#3811")
}
integration.ProgramTest(t, &integration.ProgramTestOptions{
Dir: filepath.Join("large_resource", "nodejs"),
Dependencies: []string{"@pulumi/pulumi"},
})
}
// Tests a resource with a large (>4mb) string prop in Python
func TestLargeResourcePython(t *testing.T) {
integration.ProgramTest(t, &integration.ProgramTestOptions{
Dependencies: []string{
filepath.Join("..", "..", "sdk", "python", "env", "src"),
},
Dir: filepath.Join("large_resource", "python"),
})
}
// Tests a resource with a large (>4mb) string prop in Go
func TestLargeResourceGo(t *testing.T) {
integration.ProgramTest(t, &integration.ProgramTestOptions{
Dependencies: []string{
"github.com/pulumi/pulumi/sdk/v2",
},
Dir: filepath.Join("large_resource", "go"),
})
}
// Tests a resource with a large (>4mb) string prop in .Net
func TestLargeResourceDotNet(t *testing.T) {
integration.ProgramTest(t, &integration.ProgramTestOptions{
Dependencies: []string{"Pulumi"},
Dir: filepath.Join("large_resource", "dotnet"),
})
}

View file

@ -0,0 +1,3 @@
/.pulumi/
[Bb]in/
[Oo]bj/

View file

@ -0,0 +1,21 @@
// Copyright 2016-2019, Pulumi Corporation. All rights reserved.
using System.Collections.Generic;
using System.Threading.Tasks;
using System;
using Pulumi;
class Program
{
static Task<int> Main(string[] args)
{
return Deployment.RunAsync(() =>
{
// Create and export a very long string (>4mb)
return new Dictionary<string, object>
{
{ "LongString", new string('a', 5 * 1024 * 1024) }
};
});
}
}

View file

@ -0,0 +1,3 @@
name: large_resource_dotnet
runtime: dotnet
description: A C# program with >4mb string resource

View file

@ -0,0 +1,8 @@
<Project Sdk="Microsoft.NET.Sdk">
<PropertyGroup>
<OutputType>Exe</OutputType>
<TargetFramework>netcoreapp3.1</TargetFramework>
</PropertyGroup>
</Project>

View file

@ -0,0 +1,3 @@
name: large_resource_go
runtime: go
description: A Go program with >4mb string resource

View file

@ -0,0 +1,15 @@
package main
import (
"strings"
"github.com/pulumi/pulumi/sdk/v2/go/pulumi"
)
func main() {
pulumi.Run(func(ctx *pulumi.Context) error {
// Create and export a very long string (>4mb)
ctx.Export("longString", pulumi.String(strings.Repeat("a", 5*1024*1024)))
return nil
})
}

View file

@ -0,0 +1,3 @@
/.pulumi/
/bin/
/node_modules/

View file

@ -0,0 +1,3 @@
name: large_resource_js
runtime: nodejs
description: A Node.js program with a >4mb string resource

View file

@ -0,0 +1,4 @@
// Copyright 2016-2018, Pulumi Corporation. All rights reserved.
// Create and export a very long string (>4mb)
export const longString = "a".repeat(5 * 1024 * 1024);

View file

@ -0,0 +1,14 @@
{
"name": "config_basic_js",
"version": "0.0.9",
"license": "Apache-2.0",
"devDependencies": {
"typescript": "^2.5.3"
},
"peerDependencies": {
"pulumi": "latest"
},
"dependencies": {
"@types/node": "^8.0.0"
}
}

View file

@ -0,0 +1,21 @@
{
"compilerOptions": {
"strict": true,
"outDir": "bin",
"target": "es2016",
"module": "commonjs",
"moduleResolution": "node",
"declaration": true,
"sourceMap": true,
"stripInternal": true,
"experimentalDecorators": true,
"pretty": true,
"noFallthroughCasesInSwitch": true,
"noImplicitReturns": true,
"forceConsistentCasingInFileNames": true
},
"files": [
"index.ts"
]
}

View file

@ -0,0 +1,2 @@
*.pyc
venv/

View file

@ -0,0 +1,3 @@
name: large_resource_python
runtime: python
description: A python program with a >4mb string resource

View file

@ -0,0 +1,7 @@
import pulumi
# Create a very long string (>4mb)
long_string = "a" * 5 * 1024 * 1025
# Export the name of the bucket
pulumi.export("long_string", long_string)

View file

@ -0,0 +1 @@
pulumi>=2.0.0,<3.0.0