Merge branch 'master' of github.com:pulumi/pulumi into resource_parenting_lite

This commit is contained in:
joeduffy 2017-11-26 12:09:31 -08:00
commit c788ccfdcf
16 changed files with 846 additions and 651 deletions

View file

@ -4,8 +4,8 @@ package cmd
import (
"github.com/pkg/errors"
"github.com/pulumi/pulumi/pkg/component"
"github.com/pulumi/pulumi/pkg/engine"
"github.com/pulumi/pulumi/pkg/operations"
"github.com/pulumi/pulumi/pkg/tokens"
)
@ -31,5 +31,5 @@ type pulumiBackend interface {
Update(stackName tokens.QName, debug bool, opts engine.DeployOptions) error
Destroy(stackName tokens.QName, debug bool, opts engine.DestroyOptions) error
GetLogs(stackName tokens.QName, query component.LogQuery) ([]component.LogEntry, error)
GetLogs(stackName tokens.QName, query operations.LogQuery) ([]operations.LogEntry, error)
}

View file

@ -14,9 +14,9 @@ import (
"github.com/pkg/errors"
"github.com/pulumi/pulumi/pkg/apitype"
"github.com/pulumi/pulumi/pkg/component"
"github.com/pulumi/pulumi/pkg/diag/colors"
"github.com/pulumi/pulumi/pkg/engine"
"github.com/pulumi/pulumi/pkg/operations"
"github.com/pulumi/pulumi/pkg/pack"
"github.com/pulumi/pulumi/pkg/resource/config"
"github.com/pulumi/pulumi/pkg/util/archive"
@ -225,10 +225,10 @@ func uploadProgram(uploadURL string, printSize bool) error {
return nil
}
func (b *pulumiCloudPulumiBackend) GetLogs(stackName tokens.QName, query component.LogQuery) ([]component.LogEntry, error) {
func (b *pulumiCloudPulumiBackend) GetLogs(stackName tokens.QName, query operations.LogQuery) ([]operations.LogEntry, error) {
// TODO[pulumi/pulumi-service#227]: Relax these conditions once the service can take these arguments.
if query.StartTime != nil || query.EndTime != nil || query.Query != nil {
return nil, errors.New("not implemented")
return nil, errors.New("cloud backend does not (yet) support filtering logs by start time, end time or message contents")
}
projID, err := getCloudProjectIdentifier()
@ -242,9 +242,9 @@ func (b *pulumiCloudPulumiBackend) GetLogs(stackName tokens.QName, query compone
return nil, err
}
logs := make([]component.LogEntry, 0, len(response.Logs))
logs := make([]operations.LogEntry, 0, len(response.Logs))
for _, entry := range response.Logs {
logs = append(logs, component.LogEntry(entry))
logs = append(logs, operations.LogEntry(entry))
}
return logs, nil

View file

@ -9,15 +9,13 @@ import (
"strconv"
"github.com/pkg/errors"
"github.com/pulumi/pulumi/pkg/component"
"github.com/pulumi/pulumi/pkg/encoding"
"github.com/pulumi/pulumi/pkg/pulumiframework"
"github.com/pulumi/pulumi/pkg/util/contract"
"github.com/pulumi/pulumi/pkg/engine"
"github.com/pulumi/pulumi/pkg/operations"
"github.com/pulumi/pulumi/pkg/resource/config"
"github.com/pulumi/pulumi/pkg/tokens"
"github.com/pulumi/pulumi/pkg/util/contract"
)
type localPulumiBackend struct {
@ -139,7 +137,7 @@ func (b *localPulumiBackend) Destroy(stackName tokens.QName, debug bool, opts en
return nil
}
func (b *localPulumiBackend) GetLogs(stackName tokens.QName, query component.LogQuery) ([]component.LogEntry, error) {
func (b *localPulumiBackend) GetLogs(stackName tokens.QName, query operations.LogQuery) ([]operations.LogEntry, error) {
pulumiEngine, err := b.getEngine(stackName)
if err != nil {
return nil, err
@ -158,14 +156,13 @@ func (b *localPulumiBackend) GetLogs(stackName tokens.QName, query component.Log
contract.Assert(snap != nil)
contract.Assert(target != nil)
// TODO[pulumi/pulumi#54]: replace this with a call into a generalized operations provider.
components := pulumiframework.GetComponents(snap.Resources)
ops, err := pulumiframework.OperationsProviderForComponents(target.Config, components)
if err != nil {
components := operations.NewResource(snap.Resources)
ops := components.OperationsProvider(target.Config)
logs, err := ops.GetLogs(query)
if logs == nil {
return nil, err
}
return ops.GetLogs(query)
return *logs, err
}
func (b *localPulumiBackend) getEngine(stackName tokens.QName) (engine.Engine, error) {

View file

@ -4,17 +4,21 @@ package cmd
import (
"fmt"
"regexp"
"strconv"
"time"
"github.com/spf13/cobra"
"github.com/pulumi/pulumi/pkg/component"
"github.com/pulumi/pulumi/pkg/operations"
"github.com/pulumi/pulumi/pkg/util/cmdutil"
)
func newLogsCmd() *cobra.Command {
var stack string
var follow bool
var since string
var resource string
logsCmd := &cobra.Command{
Use: "logs",
@ -25,23 +29,35 @@ func newLogsCmd() *cobra.Command {
return err
}
sinceTime := time.Unix(0, 0)
highestTimeSeen := time.Unix(0, 0)
startTime := parseRelativeDuration(since)
var resourceFilter *operations.ResourceFilter
if resource != "" {
var rf = operations.ResourceFilter(resource)
resourceFilter = &rf
}
// IDEA: This map will grow forever as new log entries are found. We may need to do a more approximate
// approach here to ensure we don't grow memory unboundedly while following logs.
//
// Note: Just tracking latest log date is not sufficient - as stale logs may show up which should have been
// displayed before previously rendered log entries, but weren't available at the time, so still need to be
// rendered now even though they are technically out of order.
shown := map[operations.LogEntry]bool{}
for {
logs, err := backend.GetLogs(stackName, component.LogQuery{})
logs, err := backend.GetLogs(stackName, operations.LogQuery{
StartTime: startTime,
ResourceFilter: resourceFilter,
})
if err != nil {
return err
}
for _, logEntry := range logs {
eventTime := time.Unix(0, logEntry.Timestamp*1000000)
if eventTime.After(sinceTime) {
fmt.Printf("[%v] %v\n", eventTime, logEntry.Message)
}
if eventTime.After(highestTimeSeen) {
highestTimeSeen = eventTime
if _, shownAlready := shown[logEntry]; !shownAlready {
eventTime := time.Unix(0, logEntry.Timestamp*1000000)
fmt.Printf("%30.30s[%30.30s] %v\n", eventTime.Format(time.RFC3339Nano), logEntry.ID, logEntry.Message)
shown[logEntry] = true
}
}
@ -49,7 +65,6 @@ func newLogsCmd() *cobra.Command {
return nil
}
sinceTime = highestTimeSeen
time.Sleep(time.Second)
}
}),
@ -61,6 +76,52 @@ func newLogsCmd() *cobra.Command {
logsCmd.PersistentFlags().BoolVarP(
&follow, "follow", "f", false,
"Follow the log stream in real time (like tail -f)")
logsCmd.PersistentFlags().StringVar(
&since, "since", "",
"Only return logs newer than a relative duration ('5s', '2m', '3h'). Defaults to returning all logs.")
logsCmd.PersistentFlags().StringVarP(
&resource, "resource", "r", "",
"Only return logs for the requested resource ('name', 'type::name' or full URN). Defaults to returning all logs.")
return logsCmd
}
var durationRegexp = regexp.MustCompile(`(\d+)([y|w|d|h|m|s])`)
// parseRelativeDuration extracts a time.Time previous to now by the a relative duration in the format '5s', '2m', '3h'.
func parseRelativeDuration(duration string) *time.Time {
now := time.Now()
if duration == "" {
return nil
}
parts := durationRegexp.FindStringSubmatch(duration)
if parts == nil {
fmt.Printf("Warning: duration could not be parsed: '%v'\n", duration)
return nil
}
num, err := strconv.ParseInt(parts[1], 10, 64)
if err != nil {
fmt.Printf("Warning: duration could not be parsed: '%v'\n", duration)
return nil
}
d := time.Duration(-num)
switch parts[2] {
case "y":
d *= time.Hour * 24 * 365
case "w":
d *= time.Hour * 24 * 7
case "d":
d *= time.Hour * 24
case "h":
d *= time.Hour
case "m":
d *= time.Minute
case "s":
d *= time.Second
default:
fmt.Printf("Warning: duration could not be parsed: '%v'\n", duration)
return nil
}
ret := now.Add(d)
return &ret
}

View file

@ -1,82 +0,0 @@
package component
import (
"time"
"github.com/pulumi/pulumi/pkg/resource"
"github.com/pulumi/pulumi/pkg/tokens"
)
// Components is a map of URN to component
type Components map[resource.URN]*Component
// Component is a serializable virtual node in a resource graph
type Component struct {
Type tokens.Type `json:"type"` // this components's full type token.
Properties resource.PropertyMap `json:"props,omitempty"` // the properties of this component.
Resources map[string]*resource.State `json:"resources,omitempty"` // the resources owned by this component.
}
// LogEntry is a row in the logs for a running compute service
type LogEntry struct {
ID string
Timestamp int64
Message string
}
// LogQuery represents the parameters to a log query operation.
// All fields are optional, leaving them off returns all logs.
type LogQuery struct {
StartTime *time.Time
EndTime *time.Time
Query *string
}
// MetricName is a handle to a metric supported by a Pulumi Framework resources
type MetricName string
type MetricRequest struct {
Name string
}
type MetricDataPoint struct {
Timestamp time.Time
Unit string
Sum float64
SampleCount float64
Average float64
Maximum float64
Minimum float64
}
// OperationsProvider is the interface for making operational requests about the
// state of a Component (or Components)
type OperationsProvider interface {
// GetLogs returns logs matching a query
GetLogs(query LogQuery) ([]LogEntry, error)
// ListMetrics returns the list of supported metrics for the requested component type
ListMetrics() []MetricName
// GetMetricStatistics provides metrics data for a given metric request
GetMetricStatistics(metric MetricRequest) ([]MetricDataPoint, error)
}
// FilterByType returns only components matching the requested type
func (c Components) FilterByType(t tokens.Type) Components {
ret := make(Components)
for k, v := range c {
if v.Type == t {
ret[k] = v
}
}
return ret
}
// GetByTypeAndName returns the component with the requested type and name
func (c Components) GetByTypeAndName(t tokens.Type, name tokens.QName) *Component {
for k, v := range c {
if k.Type() == t && k.Name() == name {
return v
}
}
return nil
}

View file

@ -0,0 +1,61 @@
package operations
import (
"time"
)
// LogEntry is a row in the logs for a running compute service
type LogEntry struct {
ID string
Timestamp int64
Message string
}
// ResourceFilter specifies a specific resource or subset of resources. It can be provided in three formats:
// - Full URN: "<namespace>::<alloc>::<type>::<name>"
// - Type + Name: "<type>::<name>"
// - Name: "<name>"
type ResourceFilter string
// LogQuery represents the parameters to a log query operation.
// All fields are optional, leaving them off returns all logs.
type LogQuery struct {
// StartTime is an optional time indiciating that only logs from after this time should be produced.
StartTime *time.Time
// EndTime is an optional time indiciating that only logs from before this time should be produced.
EndTime *time.Time
// Query is a string indicating a filter to apply to the logs - query syntax TBD
Query *string
// ResourceFilter is a string indicating that logs should be limited to a resource or resources
ResourceFilter *ResourceFilter
}
// MetricName is a handle to a metric supported by a Pulumi Framework resources
type MetricName string
// MetricRequest is a request for a metric name
type MetricRequest struct {
Name string
}
// MetricDataPoint is a data point returned from a metric.
type MetricDataPoint struct {
Timestamp time.Time
Unit string
Sum float64
SampleCount float64
Average float64
Maximum float64
Minimum float64
}
// Provider is the interface for making operational requests about the
// state of a Component (or Components)
type Provider interface {
// GetLogs returns logs matching a query
GetLogs(query LogQuery) (*[]LogEntry, error)
// ListMetrics returns the list of supported metrics for the requested component type
ListMetrics() []MetricName
// GetMetricStatistics provides metrics data for a given metric request
GetMetricStatistics(metric MetricRequest) ([]MetricDataPoint, error)
}

View file

@ -0,0 +1,187 @@
package operations
import (
"fmt"
"sort"
"sync"
"time"
"github.com/aws/aws-sdk-go/aws"
"github.com/aws/aws-sdk-go/aws/session"
"github.com/aws/aws-sdk-go/service/cloudwatch"
"github.com/aws/aws-sdk-go/service/cloudwatchlogs"
"github.com/golang/glog"
"github.com/pkg/errors"
"github.com/pulumi/pulumi/pkg/tokens"
"github.com/pulumi/pulumi/pkg/util/contract"
)
// TODO[pulumi/pulumi#54] This should be factored out behind an OperationsProvider RPC interface and versioned with the
// `pulumi-aws` repo instead of statically linked into the engine.
// AWSOperationsProvider creates an OperationsProvider capable of answering operational queries based on the
// underlying resources of the `@pulumi/aws` implementation.
func AWSOperationsProvider(
config map[tokens.ModuleMember]string,
component *Resource) (Provider, error) {
awsRegion, ok := config[regionKey]
if !ok {
return nil, errors.New("no AWS region found")
}
awsConnection, err := getAWSConnection(awsRegion)
if err != nil {
return nil, err
}
prov := &awsOpsProvider{
awsConnection: awsConnection,
component: component,
}
return prov, nil
}
type awsOpsProvider struct {
awsConnection *awsConnection
component *Resource
}
var _ Provider = (*awsOpsProvider)(nil)
const (
// AWS config keys
regionKey = "aws:config:region"
// AWS resource types
awsFunctionType = tokens.Type("aws:lambda/function:Function")
awsLogGroupType = tokens.Type("aws:cloudwatch/logGroup:LogGroup")
)
func (ops *awsOpsProvider) GetLogs(query LogQuery) (*[]LogEntry, error) {
if query.Query != nil {
contract.Failf("not yet implemented - Query")
}
switch ops.component.state.Type {
case awsFunctionType:
functionName := ops.component.state.Outputs["name"].StringValue()
logResult := ops.awsConnection.getLogsForLogGroupsConcurrently(
[]string{functionName},
[]string{"/aws/lambda/" + functionName},
query.StartTime,
query.EndTime,
)
sort.SliceStable(logResult, func(i, j int) bool { return logResult[i].Timestamp < logResult[j].Timestamp })
return &logResult, nil
case awsLogGroupType:
name := ops.component.state.Outputs["name"].StringValue()
logResult := ops.awsConnection.getLogsForLogGroupsConcurrently(
[]string{name},
[]string{name},
query.StartTime,
query.EndTime,
)
sort.SliceStable(logResult, func(i, j int) bool { return logResult[i].Timestamp < logResult[j].Timestamp })
return &logResult, nil
default:
// Else this resource kind does not produce any logs.
return nil, nil
}
}
func (ops *awsOpsProvider) ListMetrics() []MetricName {
return nil
}
func (ops *awsOpsProvider) GetMetricStatistics(metric MetricRequest) ([]MetricDataPoint, error) {
return nil, fmt.Errorf("Not yet implmeneted: GetMetricStatistics")
}
type awsConnection struct {
sess *session.Session
logSvc *cloudwatchlogs.CloudWatchLogs
metricSvc *cloudwatch.CloudWatch
}
var awsConnectionCache = map[string]*awsConnection{}
var awsConnectionCacheMutex = sync.RWMutex{}
func getAWSConnection(awsRegion string) (*awsConnection, error) {
awsConnectionCacheMutex.RLock()
connection, ok := awsConnectionCache[awsRegion]
awsConnectionCacheMutex.RUnlock()
if !ok {
awsConfig := aws.NewConfig()
awsConfig.Region = aws.String(awsRegion)
sess, err := session.NewSession(awsConfig)
if err != nil {
return nil, errors.Wrap(err, "failed to create AWS session")
}
connection = &awsConnection{
sess: sess,
logSvc: cloudwatchlogs.New(sess),
metricSvc: cloudwatch.New(sess),
}
awsConnectionCacheMutex.Lock()
awsConnectionCache[awsRegion] = connection
awsConnectionCacheMutex.Unlock()
}
return connection, nil
}
func (p *awsConnection) getLogsForLogGroupsConcurrently(
names []string,
logGroups []string,
startTime *time.Time,
endTime *time.Time) []LogEntry {
// Create a channel for collecting log event outputs
ch := make(chan []*cloudwatchlogs.FilteredLogEvent)
var startMilli *int64
if startTime != nil {
startMilli = aws.Int64(aws.TimeUnixMilli(*startTime))
}
var endMilli *int64
if endTime != nil {
endMilli = aws.Int64(aws.TimeUnixMilli(*endTime))
}
// Run FilterLogEvents for each log group in parallel
for _, logGroup := range logGroups {
go func(logGroup string) {
var ret []*cloudwatchlogs.FilteredLogEvent
err := p.logSvc.FilterLogEventsPages(&cloudwatchlogs.FilterLogEventsInput{
LogGroupName: aws.String(logGroup),
StartTime: startMilli,
EndTime: endMilli,
}, func(resp *cloudwatchlogs.FilterLogEventsOutput, lastPage bool) bool {
ret = append(ret, resp.Events...)
if !lastPage {
fmt.Printf("Getting more logs for %v...\n", logGroup)
}
return true
})
if err != nil {
glog.V(5).Infof("[getLogs] Error getting logs: %v %v\n", logGroup, err)
}
ch <- ret
}(logGroup)
}
// Collect responses on the channel and append logs into combined log array
var logs []LogEntry
for i := 0; i < len(logGroups); i++ {
logEvents := <-ch
for _, event := range logEvents {
logs = append(logs, LogEntry{
ID: names[i],
Message: aws.StringValue(event.Message),
Timestamp: aws.Int64Value(event.Timestamp),
})
}
}
return logs
}

View file

@ -0,0 +1,185 @@
package operations
import (
"encoding/json"
"fmt"
"regexp"
"time"
"github.com/pulumi/pulumi/pkg/tokens"
"github.com/pulumi/pulumi/pkg/util/contract"
)
// TODO[pulumi/pulumi#54] This should be factored out behind an OperationsProvider RPC interface and versioned with the
// `pulumi-cloud` repo instead of statically linked into the engine.
// CloudOperationsProvider creates an OperationsProvider capable of answering operational queries based on the
// underlying resources of the `@pulumi/cloud-aws` implementation.
func CloudOperationsProvider(config map[tokens.ModuleMember]string, component *Resource) (Provider, error) {
prov := &cloudOpsProvider{
config: config,
component: component,
}
return prov, nil
}
type cloudOpsProvider struct {
config map[tokens.ModuleMember]string
component *Resource
}
var _ Provider = (*cloudOpsProvider)(nil)
const (
// Pulumi Framework component types
cloudFunctionType = tokens.Type("cloud:function:Function")
cloudLogCollectorType = tokens.Type("cloud:logCollector:LogCollector")
cloudServiceType = tokens.Type("cloud:service:Service")
cloudTaskType = tokens.Type("cloud:task:Task")
// AWS resource types
awsServerlessFunctionTypeName = "aws:serverless:Function"
awsLogGroupTypeName = "aws:cloudwatch/logGroup:LogGroup"
)
func (ops *cloudOpsProvider) GetLogs(query LogQuery) (*[]LogEntry, error) {
switch ops.component.state.Type {
case cloudFunctionType:
// We get the aws:serverless:Function child and request it's logs, parsing out the user-visible content from
// those logs to project into our own log output, but leaving out explicit Lambda metadata.
name := string(ops.component.state.URN.Name())
serverlessFunction := ops.component.GetChild(awsServerlessFunctionTypeName, name)
rawLogs, err := serverlessFunction.OperationsProvider(ops.config).GetLogs(query)
if err != nil {
return nil, err
}
contract.Assertf(rawLogs != nil, "expect aws:serverless:Function to provide logs")
var logs []LogEntry
for _, rawLog := range *rawLogs {
extractedLog := extractLambdaLogMessage(rawLog.Message, name)
if extractedLog != nil {
logs = append(logs, *extractedLog)
}
}
return &logs, nil
case cloudLogCollectorType:
// A LogCollector has an aws:serverless:Function which is wired up to receive logs from all other compute in the
// program. These logs are batched and then console.log'd into the log collector lambdas own logs, so we must
// get those logs and then decode through two layers of Lambda logging to extract the original messages. These
// logs are delayed somewhat more than raw lambda logs, but can survive even after the source lambda is deleted.
// In addition, we set the Lambda logs to automatically delete after 24 hours, which is safe because we have
// centrally archived into the log collector. As a result, we will combine reading these logs with reading the
// live Lambda logs from individual functions, de-duplicating the results, to piece together the full set of
// logs.
name := string(ops.component.state.URN.Name())
serverlessFunction := ops.component.GetChild(awsServerlessFunctionTypeName, name)
rawLogs, err := serverlessFunction.OperationsProvider(ops.config).GetLogs(query)
if err != nil {
return nil, err
}
contract.Assertf(rawLogs != nil, "expect aws:serverless:Function to provide logs")
// Extract out the encoded and batched logs
var logs []LogEntry
for _, rawLog := range *rawLogs {
extractedLog := extractLambdaLogMessage(rawLog.Message, name)
if extractedLog != nil {
// Decode the JSON blog of data from within the log entries, which will itself be a nested log entry.
var logMessage encodedLogMessage
err := json.Unmarshal([]byte(extractedLog.Message), &logMessage)
if err != nil {
return nil, err
}
// Reverse engineer the name of the function that was the source of this message from the LogGroup name.
logName := logMessage.LogGroup
match := functionNameFromLogGroupNameRegExp.FindStringSubmatch(logMessage.LogGroup)
if len(match) == 2 {
logName = match[1]
}
// Extract out each individual log event and add them to our array of logs.
for _, logEvent := range logMessage.LogEvents {
if extracted := extractLambdaLogMessage(logEvent.Message, logName); extracted != nil {
logs = append(logs, *extracted)
}
}
}
}
return &logs, nil
case cloudServiceType, cloudTaskType:
// Both Services and Tasks track a log group, which we can directly query for logs. These logs are only populated by user
// code within containers, so we can safely project these logs back unmodified.
urn := ops.component.state.URN
name := string(urn.Name())
logGroup := ops.component.GetChild(awsLogGroupTypeName, name+"-task-logs")
rawLogs, err := logGroup.OperationsProvider(ops.config).GetLogs(query)
if err != nil {
return nil, err
}
contract.Assertf(rawLogs != nil, "expect aws:cloudwatch/logGroup:LogGroup to provide logs")
var logs []LogEntry
for _, rawLog := range *rawLogs {
logs = append(logs, LogEntry{
ID: name,
Message: rawLog.Message,
Timestamp: rawLog.Timestamp,
})
}
return &logs, nil
default:
// Else this resource kind does not produce any logs.
return nil, nil
}
}
func (ops *cloudOpsProvider) ListMetrics() []MetricName {
return nil
}
func (ops *cloudOpsProvider) GetMetricStatistics(metric MetricRequest) ([]MetricDataPoint, error) {
return nil, fmt.Errorf("Not yet implmeneted: GetMetricStatistics")
}
type encodedLogEvent struct {
ID string `json:"id"`
Timestamp int64 `json:"timestamp"`
Message string `json:"message"`
}
type encodedLogMessage struct {
MessageType string `json:"messageType"`
Owner string `json:"owner"`
LogGroup string `json:"logGroup"`
LogStream string `json:"logStream"`
SubscriptionFilters []string `json:"subscriptionFilters"`
LogEvents []encodedLogEvent `json:"logEvents"`
}
var (
// Extract function name from LogGroup name
functionNameFromLogGroupNameRegExp = regexp.MustCompile(`^/aws/lambda/(.*)\-[0-9A-Fa-f]+$`)
// Extract Lambda log parts from Lambda log format
logRegexp = regexp.MustCompile("(.*Z)\t[a-g0-9\\-]*\t(.*)")
)
// extractLambdaLogMessage extracts out only the log messages associated with user logs, skipping Lambda-specific metadata.
// In particular, only the second line below is extracter, and it is extracted with the recorded timestamp.
//
// ```
// START RequestId: 25e0d1e0-cbd6-11e7-9808-c7085dfe5723 Version: $LATEST
// 2017-11-17T20:30:27.736Z 25e0d1e0-cbd6-11e7-9808-c7085dfe5723 GET /todo
// END RequestId: 25e0d1e0-cbd6-11e7-9808-c7085dfe5723
// REPORT RequestId: 25e0d1e0-cbd6-11e7-9808-c7085dfe5723 Duration: 222.92 ms Billed Duration: 300 ms <snip>
// ```
func extractLambdaLogMessage(message string, id string) *LogEntry {
innerMatches := logRegexp.FindAllStringSubmatch(message, -1)
if len(innerMatches) > 0 {
contract.Assertf(len(innerMatches[0]) >= 3, "expected log regexp to always produce at least two capture groups")
timestamp, err := time.Parse(time.RFC3339Nano, innerMatches[0][1])
contract.Assertf(err == nil, "expected to be able to parse timestamp")
return &LogEntry{
ID: id,
Message: innerMatches[0][2],
Timestamp: timestamp.UnixNano() / 1000000, // milliseconds
}
}
return nil
}

View file

@ -0,0 +1,23 @@
package operations
import (
"testing"
"github.com/stretchr/testify/assert"
)
func Test_extractLambdaLogMessage(t *testing.T) {
res := extractLambdaLogMessage("START RequestId: 25e0d1e0-cbd6-11e7-9808-c7085dfe5723 Version: $LATEST", "foo")
assert.Nil(t, res)
res = extractLambdaLogMessage("2017-11-17T20:30:27.736Z 25e0d1e0-cbd6-11e7-9808-c7085dfe5723 GET /todo", "foo")
assert.NotNil(t, res)
assert.Equal(t, "GET /todo", res.Message)
res = extractLambdaLogMessage("END RequestId: 25e0d1e0-cbd6-11e7-9808-c7085dfe5723", "foo")
assert.Nil(t, res)
}
func Test_functionNameFromLogGroupNameRegExp(t *testing.T) {
match := functionNameFromLogGroupNameRegExp.FindStringSubmatch("/aws/lambda/examples-todoc57917fa-023a27bc")
assert.Len(t, match, 2)
assert.Equal(t, "examples-todoc57917fa", match[1])
}

220
pkg/operations/resources.go Normal file
View file

@ -0,0 +1,220 @@
package operations
import (
"fmt"
"sort"
"github.com/pulumi/pulumi/pkg/resource"
"github.com/pulumi/pulumi/pkg/tokens"
"github.com/pulumi/pulumi/pkg/util/contract"
)
// Resource is a tree representation of a resource/component hierarchy
type Resource struct {
ns tokens.QName
alloc tokens.PackageName
state *resource.State
parent *Resource
children map[resource.URN]*Resource
}
// NewResource constructs a tree representation of a resource/component hierarchy
func NewResource(source []*resource.State) *Resource {
treeNodes := map[resource.URN]*Resource{}
var ns tokens.QName
var alloc tokens.PackageName
// Walk the ordered resource list and build tree nodes based on child relationships
for _, state := range source {
ns = state.URN.Namespace()
alloc = state.URN.Alloc()
newTree := &Resource{
ns: ns,
alloc: alloc,
state: state,
parent: nil,
children: map[resource.URN]*Resource{},
}
for _, childURN := range state.Children {
childTree, ok := treeNodes[childURN]
contract.Assertf(ok, "Expected children to be before parents in resource checkpoint")
childTree.parent = newTree
newTree.children[childTree.state.URN] = childTree
}
treeNodes[state.URN] = newTree
}
// Create a single root node which is the parent of all unparented nodes
root := &Resource{
ns: ns,
alloc: alloc,
state: nil,
parent: nil,
children: map[resource.URN]*Resource{},
}
for _, node := range treeNodes {
if node.parent == nil {
root.children[node.state.URN] = node
node.parent = root
}
}
// Return the root node
return root
}
// GetChild find a child with the given type and name or returns `nil`.
func (r *Resource) GetChild(typ string, name string) *Resource {
childURN := resource.NewURN(r.ns, r.alloc, tokens.Type(typ), tokens.QName(name))
return r.children[childURN]
}
// OperationsProvider gets an OperationsProvider for this resource.
func (r *Resource) OperationsProvider(config map[tokens.ModuleMember]string) Provider {
return &resourceOperations{
resource: r,
config: config,
}
}
// ResourceOperations is an OperationsProvider for Resources
type resourceOperations struct {
resource *Resource
config map[tokens.ModuleMember]string
}
var _ Provider = (*resourceOperations)(nil)
// GetLogs gets logs for a Resource
func (ops *resourceOperations) GetLogs(query LogQuery) (*[]LogEntry, error) {
// Only get logs for this resource if it matches the resource filter query
if ops.matchesResourceFilter(query.ResourceFilter) {
// Set query to be a new query with `ResourceFilter` nil so that we don't filter out logs from any children of
// this resource since this resource did match the resource filter.
query = LogQuery{
StartTime: query.StartTime,
EndTime: query.EndTime,
Query: query.Query,
ResourceFilter: nil,
}
// Try to get an operations provider for this resource, it may be `nil`
opsProvider, err := ops.getOperationsProvider()
if err != nil {
return nil, err
}
if opsProvider != nil {
// If this resource has an operations provider - use it and don't recur into children. It is the
// responsibility of it's GetLogs implementation to aggregate all logs from children, either by passing them
// through or by filtering specific content out.
logsResult, err := opsProvider.GetLogs(query)
if err != nil {
return logsResult, err
}
if logsResult != nil {
return logsResult, nil
}
}
}
// If this resource did not choose to provide it's own logs, recur into children and collect + aggregate their logs.
var logs []LogEntry
// Kick off GetLogs on all children in parallel, writing results to shared channels
ch := make(chan *[]LogEntry)
errch := make(chan error)
for _, child := range ops.resource.children {
childOps := &resourceOperations{
resource: child,
config: ops.config,
}
go func() {
childLogs, err := childOps.GetLogs(query)
ch <- childLogs
errch <- err
}()
}
// Handle results from GetLogs calls as they complete
for range ops.resource.children {
childLogs := <-ch
err := <-errch
if err != nil {
return &logs, err
}
if childLogs != nil {
logs = append(logs, *childLogs...)
}
}
// Sort
sort.SliceStable(logs, func(i, j int) bool { return logs[i].Timestamp < logs[j].Timestamp })
// Remove duplicates
var retLogs []LogEntry
var lastLogTimestamp int64
var lastLogs []LogEntry
for _, log := range logs {
shouldContinue := false
if log.Timestamp == lastLogTimestamp {
for _, lastLog := range lastLogs {
if log.Message == lastLog.Message {
shouldContinue = true
break
}
}
} else {
lastLogs = nil
}
if shouldContinue {
continue
}
lastLogs = append(lastLogs, log)
lastLogTimestamp = log.Timestamp
retLogs = append(retLogs, log)
}
return &retLogs, nil
}
// matchesResourceFilter determines whether this resource matches the provided resource filter.
func (ops *resourceOperations) matchesResourceFilter(filter *ResourceFilter) bool {
if filter == nil {
// No filter, all resources match it.
return true
}
if ops.resource == nil || ops.resource.state == nil {
return false
}
urn := ops.resource.state.URN
if resource.URN(*filter) == urn {
// The filter matched the full URN
return true
}
if string(*filter) == string(urn.Type())+"::"+string(urn.Name()) {
// The filter matched the '<type>::<name>' part of the URN
return true
}
if tokens.QName(*filter) == urn.Name() {
// The filter matched the '<name>' part of the URN
return true
}
return false
}
// ListMetrics lists metrics for a Resource
func (ops *resourceOperations) ListMetrics() []MetricName {
return []MetricName{}
}
// GetMetricStatistics gets metric statistics for a Resource
func (ops *resourceOperations) GetMetricStatistics(metric MetricRequest) ([]MetricDataPoint, error) {
return nil, fmt.Errorf("not yet implemented")
}
func (ops *resourceOperations) getOperationsProvider() (Provider, error) {
if ops.resource == nil || ops.resource.state == nil {
return nil, nil
}
switch ops.resource.state.Type.Package() {
case "cloud":
return CloudOperationsProvider(ops.config, ops.resource)
case "aws":
return AWSOperationsProvider(ops.config, ops.resource)
default:
return nil, nil
}
}

View file

@ -0,0 +1,81 @@
package operations
import (
"encoding/json"
"io/ioutil"
"testing"
"github.com/davecgh/go-spew/spew"
"github.com/stretchr/testify/assert"
"github.com/pulumi/pulumi/pkg/resource/stack"
)
func getPulumiResources(t *testing.T, path string) *Resource {
var checkpoint stack.Checkpoint
byts, err := ioutil.ReadFile(path)
assert.NoError(t, err)
err = json.Unmarshal(byts, &checkpoint)
assert.NoError(t, err)
_, _, snapshot, err := stack.DeserializeCheckpoint(&checkpoint)
assert.NoError(t, err)
resources := NewResource(snapshot.Resources)
spew.Dump(resources)
return resources
}
func TestTodo(t *testing.T) {
components := getPulumiResources(t, "testdata/todo.json")
assert.Equal(t, 4, len(components.children))
// Table child
table := components.GetChild("cloud:table:Table", "todo")
if !assert.NotNil(t, table) {
return
}
assert.Equal(t, 2, len(table.state.Inputs))
assert.Equal(t, "id", table.state.Inputs["primaryKey"].StringValue())
assert.Equal(t, 1, len(table.children))
assert.NotNil(t, table.GetChild("aws:dynamodb/table:Table", "todo"))
// Endpoint child
endpoint := components.GetChild("cloud:http:HttpEndpoint", "todo")
if !assert.NotNil(t, endpoint) {
return
}
assert.Equal(t, 5, len(endpoint.state.Inputs))
assert.Equal(t, "https://eupwl7wu4i.execute-api.us-east-2.amazonaws.com/", endpoint.state.Inputs["url"].StringValue())
assert.Equal(t, 14, len(endpoint.children))
assert.NotNil(t, endpoint.GetChild("aws:apigateway/restApi:RestApi", "todo"))
}
func TestCrawler(t *testing.T) {
components := getPulumiResources(t, "testdata/crawler.json")
assert.Equal(t, 7, len(components.children))
// Topic child
topic := components.GetChild("cloud:topic:Topic", "countDown")
if !assert.NotNil(t, topic) {
return
}
assert.Equal(t, 0, len(topic.state.Inputs))
assert.Equal(t, 1, len(topic.children))
assert.NotNil(t, topic.GetChild("aws:sns/topic:Topic", "countDown"))
// Timer child
heartbeat := components.GetChild("cloud:timer:Timer", "heartbeat")
if !assert.NotNil(t, heartbeat) {
return
}
assert.Equal(t, 1, len(heartbeat.state.Inputs))
assert.Equal(t, "rate(5 minutes)", heartbeat.state.Inputs["scheduleExpression"].StringValue())
assert.Equal(t, 4, len(heartbeat.children))
// Function child of timer
function := heartbeat.GetChild("cloud:function:Function", "heartbeat")
if !assert.NotNil(t, function) {
return
}
assert.Equal(t, 1, len(function.state.Inputs))
assert.Equal(t, 3, len(function.children))
}

View file

@ -1,98 +0,0 @@
package pulumiframework
import (
"regexp"
"github.com/aws/aws-sdk-go/service/cloudwatch"
"github.com/aws/aws-sdk-go/aws"
"github.com/aws/aws-sdk-go/aws/session"
"github.com/aws/aws-sdk-go/service/cloudwatchlogs"
"github.com/golang/glog"
"github.com/pulumi/pulumi/pkg/component"
)
type awsConnection struct {
sess *session.Session
logSvc *cloudwatchlogs.CloudWatchLogs
metricSvc *cloudwatch.CloudWatch
}
func newAWSConnection(sess *session.Session) *awsConnection {
return &awsConnection{
sess: sess,
logSvc: cloudwatchlogs.New(sess),
metricSvc: cloudwatch.New(sess),
}
}
var logRegexp = regexp.MustCompile(".*Z\t[a-g0-9\\-]*\t(.*)")
func (p *awsConnection) getLogsForFunctionsConcurrently(functions []string) []component.LogEntry {
var logs []component.LogEntry
ch := make(chan []component.LogEntry)
for _, functionName := range functions {
go func(functionName string) {
ch <- p.getLogsForFunction(functionName)
}(functionName)
}
for i := 0; i < len(functions); i++ {
logs = append(logs, <-ch...)
}
return logs
}
func (p *awsConnection) getLogsForFunction(functionName string) []component.LogEntry {
logGroupName := "/aws/lambda/" + functionName
resp, err := p.logSvc.DescribeLogStreams(&cloudwatchlogs.DescribeLogStreamsInput{
LogGroupName: aws.String(logGroupName),
})
if err != nil {
glog.V(5).Infof("[getLogs] Error getting logs: %v %v\n", logGroupName, err)
return nil
}
glog.V(5).Infof("[getLogs] Log streams: %v\n", resp)
logResult := p.getLogsForFunctionNameStreamsConcurrently(functionName, resp.LogStreams)
return logResult
}
func (p *awsConnection) getLogsForFunctionNameStreamsConcurrently(functionName string,
logStreams []*cloudwatchlogs.LogStream) []component.LogEntry {
var logs []component.LogEntry
ch := make(chan []component.LogEntry)
for _, logStream := range logStreams {
go func(logStreamName *string) {
ch <- p.getLogsForFunctionNameStream(functionName, logStreamName)
}(logStream.LogStreamName)
}
for i := 0; i < len(logStreams); i++ {
logs = append(logs, <-ch...)
}
return logs
}
func (p *awsConnection) getLogsForFunctionNameStream(functionName string, logStreamName *string) []component.LogEntry {
var logResult []component.LogEntry
logGroupName := "/aws/lambda/" + functionName
logsResp, err := p.logSvc.GetLogEvents(&cloudwatchlogs.GetLogEventsInput{
LogGroupName: aws.String(logGroupName),
LogStreamName: logStreamName,
StartFromHead: aws.Bool(true),
})
if err != nil {
glog.V(5).Infof("[getLogs] Error getting logs: %v %v\n", logStreamName, err)
}
glog.V(5).Infof("[getLogs] Log events: %v\n", logsResp)
for _, event := range logsResp.Events {
innerMatches := logRegexp.FindAllStringSubmatch(aws.StringValue(event.Message), -1)
glog.V(5).Infof("[getLogs] Inner matches: %v\n", innerMatches)
if len(innerMatches) > 0 {
logResult = append(logResult, component.LogEntry{
ID: functionName,
Message: innerMatches[0][1],
Timestamp: aws.Int64Value(event.Timestamp),
})
}
}
return logResult
}

View file

@ -1,344 +0,0 @@
package pulumiframework
import (
"fmt"
"sort"
"strings"
"github.com/aws/aws-sdk-go/aws"
"github.com/aws/aws-sdk-go/aws/session"
"github.com/aws/aws-sdk-go/service/cloudwatch"
"github.com/pkg/errors"
"github.com/pulumi/pulumi/pkg/component"
"github.com/pulumi/pulumi/pkg/resource"
"github.com/pulumi/pulumi/pkg/tokens"
"github.com/pulumi/pulumi/pkg/util/contract"
)
// This file contains the implementation of the component.Components interface for the
// AWS implementation of the Pulumi Framework defined in this repo.
// GetComponents extracts the Pulumi Framework components from a checkpoint
// file, based on the raw resources created by the implementation of the Pulumi Framework
// in this repo.
func GetComponents(source []*resource.State) component.Components {
sourceMap := makeIDLookup(source)
components := make(component.Components)
for _, res := range source {
name := string(res.URN.Name())
if res.Type == stageType {
stage := res
deployment := lookup(sourceMap, deploymentType, stage.Inputs["deployment"].StringValue())
restAPI := lookup(sourceMap, restAPIType, stage.Inputs["restApi"].StringValue())
baseURL := deployment.Outputs["invokeUrl"].StringValue() + stage.Inputs["stageName"].StringValue() + "/"
restAPIName := restAPI.URN.Name()
urn := newPulumiFrameworkURN(res.URN, pulumiEndpointType, restAPIName)
components[urn] = &component.Component{
Type: pulumiEndpointType,
Properties: resource.NewPropertyMapFromMap(map[string]interface{}{
"url": baseURL,
}),
Resources: map[string]*resource.State{
"restapi": restAPI,
"deployment": deployment,
"stage": stage,
},
}
} else if res.Type == eventRuleType {
urn := newPulumiFrameworkURN(res.URN, pulumiTimerType, tokens.QName(name))
components[urn] = &component.Component{
Type: pulumiTimerType,
Properties: resource.NewPropertyMapFromMap(map[string]interface{}{
"schedule": res.Inputs["scheduleExpression"].StringValue(),
}),
Resources: map[string]*resource.State{
"rule": res,
"target": nil,
"permission": nil,
},
}
} else if res.Type == tableType {
urn := newPulumiFrameworkURN(res.URN, pulumiTableType, tokens.QName(name))
components[urn] = &component.Component{
Type: pulumiTableType,
Properties: resource.NewPropertyMapFromMap(map[string]interface{}{
"primaryKey": res.Inputs["hashKey"].StringValue(),
}),
Resources: map[string]*resource.State{
"table": res,
},
}
} else if res.Type == topicType {
if !strings.HasSuffix(name, "unhandled-error-topic") {
urn := newPulumiFrameworkURN(res.URN, pulumiTopicType, tokens.QName(name))
components[urn] = &component.Component{
Type: pulumiTopicType,
Properties: resource.NewPropertyMapFromMap(map[string]interface{}{}),
Resources: map[string]*resource.State{
"topic": res,
},
}
}
} else if res.Type == functionType {
if !strings.HasSuffix(name, "-log-collector") {
urn := newPulumiFrameworkURN(res.URN, pulumiFunctionType, tokens.QName(name))
components[urn] = &component.Component{
Type: pulumiFunctionType,
Properties: resource.NewPropertyMapFromMap(map[string]interface{}{}),
Resources: map[string]*resource.State{
"function": res,
"role": nil,
"roleAttachment": nil,
"logGroup": nil,
"logSubscriptionFilter": nil,
"permission": nil,
},
}
}
}
}
return components
}
// This function grovels through the given configuration bag, extracts the bits necessary to create an AWS session
// (currently just the AWS region to target), and creates and returns the session. If the bag does not contain the
// necessary properties or if session creation fails, this function returns `nil, error`.
func createSessionFromConfig(config map[tokens.ModuleMember]string) (*session.Session, error) {
awsRegion, ok := config[regionKey]
if !ok {
return nil, errors.New("no AWS region found")
}
awsConfig := aws.NewConfig()
awsConfig.Region = aws.String(awsRegion)
return session.NewSession(awsConfig)
}
// OperationsProviderForComponent creates an OperationsProvider capable of answering
// operational queries based on the underlying resources of the AWS Pulumi Framework implementation.
func OperationsProviderForComponent(
config map[tokens.ModuleMember]string,
component *component.Component) (component.OperationsProvider, error) {
sess, err := createSessionFromConfig(config)
if err != nil {
return nil, errors.Wrap(err, "failed to create AWS session")
}
prov := &componentOpsProvider{
awsConnection: newAWSConnection(sess),
component: component,
}
return prov, nil
}
type componentOpsProvider struct {
awsConnection *awsConnection
component *component.Component
}
var _ component.OperationsProvider = (*componentOpsProvider)(nil)
const (
// AWS config keys
regionKey = "aws:config:region"
// AWS Resource Types
stageType = "aws:apigateway/stage:Stage"
deploymentType = "aws:apigateway/deployment:Deployment"
restAPIType = "aws:apigateway/restApi:RestApi"
eventRuleType = "aws:cloudwatch/eventRule:EventRule"
tableType = "aws:dynamodb/table:Table"
topicType = "aws:sns/topic:Topic"
functionType = "aws:lambda/function:Function"
// Pulumi Framework "virtual" types
pulumiEndpointType = tokens.Type("pulumi:framework:Endpoint")
pulumiTopicType = tokens.Type("pulumi:framework:Topic")
pulumiTimerType = tokens.Type("pulumi:framework:Timer")
pulumiTableType = tokens.Type("pulumi:framework:Table")
pulumiFunctionType = tokens.Type("pulumi:framework:Function")
// Operational metric names for Pulumi Framework components
functionInvocations component.MetricName = "Invocation"
functionDuration component.MetricName = "Duration"
functionErrors component.MetricName = "Errors"
functionThrottles component.MetricName = "Throttles"
endpoint4xxError component.MetricName = "4xxerror"
endpoint5xxError component.MetricName = "5xxerror"
endpointCount component.MetricName = "count"
endpointLatency component.MetricName = "latency"
topicPulished component.MetricName = "published"
topicPublishSize component.MetricName = "publishsize"
topicDelivered component.MetricName = "delivered"
topicFailed component.MetricName = "failed"
timerInvocations component.MetricName = "invocations"
timerFailedInvocations component.MetricName = "failedinvocations"
tableConsumedReadCapacity component.MetricName = "consumedreadcapacity"
tableConsumedWriteCapacity component.MetricName = "consumerwritecapacity"
tableThrottles component.MetricName = "throttles"
)
func (ops *componentOpsProvider) GetLogs(query component.LogQuery) ([]component.LogEntry, error) {
if query.StartTime != nil || query.EndTime != nil || query.Query != nil {
contract.Failf("not yet implemented - StartTime, Endtime, Query")
}
switch ops.component.Type {
case pulumiFunctionType:
functionName := ops.component.Resources["function"].Outputs["name"].StringValue()
logResult := ops.awsConnection.getLogsForFunction(functionName)
sort.SliceStable(logResult, func(i, j int) bool { return logResult[i].Timestamp < logResult[j].Timestamp })
return logResult, nil
default:
return nil, fmt.Errorf("Logs not supported for component type: %s", ops.component.Type)
}
}
func (ops *componentOpsProvider) ListMetrics() []component.MetricName {
switch ops.component.Type {
case pulumiFunctionType:
// Don't include these which are internal implementation metrics: DLQ delivery errors
return []component.MetricName{functionInvocations, functionDuration, functionErrors, functionThrottles}
case pulumiEndpointType:
return []component.MetricName{endpoint4xxError, endpoint5xxError, endpointCount, endpointLatency}
case pulumiTopicType:
return []component.MetricName{topicPulished, topicPublishSize, topicDelivered, topicFailed}
case pulumiTimerType:
return []component.MetricName{timerInvocations, timerFailedInvocations}
case pulumiTableType:
// Internal only: "provisionedreadcapacity", "provisionedwritecapacity", "usererrors", "timetolivedeleted",
// "systemerrors", "succesfulrequestlatency", "returnedrecordscount", "returenditemcount", "returnedbytes",
// "onlineindex*", "conditionalcheckfailed"
return []component.MetricName{tableConsumedReadCapacity, tableConsumedWriteCapacity, tableThrottles}
default:
contract.Failf("invalid component type")
return nil
}
}
func (ops *componentOpsProvider) GetMetricStatistics(metric component.MetricRequest) ([]component.MetricDataPoint, error) {
var dimensions []*cloudwatch.Dimension
var namespace string
switch ops.component.Type {
case pulumiFunctionType:
dimensions = append(dimensions, &cloudwatch.Dimension{
Name: aws.String("FunctionName"),
Value: aws.String(string(ops.component.Resources["function"].ID)),
})
namespace = "AWS/Lambda"
case pulumiEndpointType:
contract.Failf("not yet implemented")
case pulumiTopicType:
contract.Failf("not yet implemented")
case pulumiTimerType:
contract.Failf("not yet implemented")
case pulumiTableType:
contract.Failf("not yet implemented")
default:
contract.Failf("invalid component type")
}
resp, err := ops.awsConnection.metricSvc.GetMetricStatistics(&cloudwatch.GetMetricStatisticsInput{
Namespace: aws.String(namespace),
MetricName: aws.String(metric.Name),
Dimensions: dimensions,
Statistics: []*string{
aws.String("Sum"), aws.String("SampleCount"), aws.String("Average"),
aws.String("Maximum"), aws.String("Minimum"),
},
})
if err != nil {
return nil, err
}
var metrics []component.MetricDataPoint
for _, datapoint := range resp.Datapoints {
metrics = append(metrics, component.MetricDataPoint{
Timestamp: aws.TimeValue(datapoint.Timestamp),
Unit: aws.StringValue(datapoint.Unit),
Sum: aws.Float64Value(datapoint.Sum),
SampleCount: aws.Float64Value(datapoint.SampleCount),
Average: aws.Float64Value(datapoint.Average),
Maximum: aws.Float64Value(datapoint.Maximum),
Minimum: aws.Float64Value(datapoint.Minimum),
})
}
return metrics, nil
}
// OperationsProviderForComponents creates an OperationsProvider capable of answering
// operational queries about a collection of Pulumi Framework Components based on the
// underlying resources of the AWS Pulumi Framework implementation.
func OperationsProviderForComponents(
config map[tokens.ModuleMember]string,
components component.Components) (component.OperationsProvider, error) {
sess, err := createSessionFromConfig(config)
if err != nil {
return nil, errors.Wrap(err, "failed to create AWS session")
}
prov := &componentsOpsProvider{
awsConnection: newAWSConnection(sess),
components: components,
}
return prov, nil
}
type componentsOpsProvider struct {
awsConnection *awsConnection
components component.Components
}
var _ component.OperationsProvider = (*componentsOpsProvider)(nil)
// GetLogs for a collection of Components returns combined logs from all Pulumi Function
// components in the collection.
func (ops *componentsOpsProvider) GetLogs(query component.LogQuery) ([]component.LogEntry, error) {
if query.StartTime != nil || query.EndTime != nil || query.Query != nil {
contract.Failf("not yet implemented - StartTime, Endtime, Query")
}
var functionNames []string
functionComponents := ops.components.FilterByType(pulumiFunctionType)
for _, v := range functionComponents {
functionName := v.Resources["function"].Outputs["name"].StringValue()
functionNames = append(functionNames, functionName)
}
logResults := ops.awsConnection.getLogsForFunctionsConcurrently(functionNames)
sort.SliceStable(logResults, func(i, j int) bool { return logResults[i].Timestamp < logResults[j].Timestamp })
return logResults, nil
}
func (ops *componentsOpsProvider) ListMetrics() []component.MetricName {
return []component.MetricName{}
}
func (ops *componentsOpsProvider) GetMetricStatistics(metric component.MetricRequest) ([]component.MetricDataPoint, error) {
return nil, fmt.Errorf("not yet implemented")
}
type typeid struct {
Type tokens.Type
ID resource.ID
}
func makeIDLookup(source []*resource.State) map[typeid]*resource.State {
ret := make(map[typeid]*resource.State)
for _, state := range source {
tid := typeid{Type: state.Type, ID: state.ID}
ret[tid] = state
}
return ret
}
func lookup(m map[typeid]*resource.State, t string, id string) *resource.State {
return m[typeid{Type: tokens.Type(t), ID: resource.ID(id)}]
}
func newPulumiFrameworkURN(resourceURN resource.URN, t tokens.Type, name tokens.QName) resource.URN {
namespace := resourceURN.Namespace()
return resource.NewURN(namespace, resourceURN.Alloc(), t, name)
}

View file

@ -1,96 +0,0 @@
package pulumiframework
import (
"encoding/json"
"io/ioutil"
"testing"
"github.com/aws/aws-sdk-go/aws"
"github.com/aws/aws-sdk-go/aws/session"
"github.com/davecgh/go-spew/spew"
"github.com/stretchr/testify/assert"
"github.com/pulumi/pulumi/pkg/component"
"github.com/pulumi/pulumi/pkg/resource"
"github.com/pulumi/pulumi/pkg/resource/stack"
"github.com/pulumi/pulumi/pkg/tokens"
)
var sess *session.Session
func init() {
var err error
config := aws.NewConfig()
config.Region = aws.String("eu-west-1")
sess, err = session.NewSession(config)
if err != nil {
panic("Could not create AWS session")
}
}
func getPulumiResources(t *testing.T, path string) (component.Components, tokens.QName) {
var checkpoint stack.Checkpoint
byts, err := ioutil.ReadFile(path)
assert.NoError(t, err)
err = json.Unmarshal(byts, &checkpoint)
assert.NoError(t, err)
name, _, snapshot, err := stack.DeserializeCheckpoint(&checkpoint)
assert.NoError(t, err)
resources := GetComponents(snapshot.Resources)
spew.Dump(resources)
return resources, name
}
func TestTodo(t *testing.T) {
components, targetName := getPulumiResources(t, "testdata/todo.json")
assert.Equal(t, 5, len(components))
rawURN := resource.NewURN(targetName, "todo", "aws:dynamodb/table:Table:", "todo")
tableArn := newPulumiFrameworkURN(rawURN, tokens.Type(pulumiTableType), tokens.QName("todo"))
table, ok := components[tableArn]
if !assert.True(t, ok) {
return
}
assert.Equal(t, 1, len(table.Properties))
assert.Equal(t, "id", table.Properties[resource.PropertyKey("primaryKey")].StringValue())
assert.Equal(t, 1, len(table.Resources))
assert.Equal(t, pulumiTableType, table.Type)
endpointArn := newPulumiFrameworkURN(rawURN, tokens.Type(pulumiEndpointType), tokens.QName("todo"))
endpoint, ok := components[endpointArn]
if !assert.True(t, ok) {
return
}
assert.Equal(t, 1, len(endpoint.Properties))
assert.Equal(t, "https://eupwl7wu4i.execute-api.us-east-2.amazonaws.com/stage/",
endpoint.Properties[resource.PropertyKey("url")].StringValue())
assert.Equal(t, 3, len(endpoint.Resources))
assert.Equal(t, pulumiEndpointType, endpoint.Type)
}
func TestCrawler(t *testing.T) {
components, targetName := getPulumiResources(t, "testdata/crawler.json")
assert.Equal(t, 4, len(components))
rawURN := resource.NewURN(targetName, "countdown", "aws:sns/topic:Topic", "countDown")
countDownArn := newPulumiFrameworkURN(rawURN, tokens.Type(pulumiTopicType), tokens.QName("countDown"))
countDown, ok := components[countDownArn]
if !assert.True(t, ok) {
return
}
assert.Equal(t, 0, len(countDown.Properties))
assert.Equal(t, 1, len(countDown.Resources))
assert.Equal(t, pulumiTopicType, countDown.Type)
heartbeatArn := newPulumiFrameworkURN(rawURN, tokens.Type(pulumiTimerType), tokens.QName("heartbeat"))
heartbeat, ok := components[heartbeatArn]
if !assert.True(t, ok) {
return
}
assert.Equal(t, 1, len(heartbeat.Properties))
assert.Equal(t, "rate(5 minutes)", heartbeat.Properties[resource.PropertyKey("schedule")].StringValue())
assert.Equal(t, 3, len(heartbeat.Resources))
assert.Equal(t, pulumiTimerType, heartbeat.Type)
}