Skip to content

Commit

Permalink
Add cli flag for secrets backends (#2964)
Browse files Browse the repository at this point in the history
* Pin main of benthos

* Add secrets management

* Update to enterprise headers

* Ensure disable env lookup applies without custom secrets

* Elif the env disable

* Remove disable-env-lookup flag
  • Loading branch information
Jeffail authored Oct 28, 2024
1 parent 5ee8fb6 commit 23f5308
Show file tree
Hide file tree
Showing 5 changed files with 233 additions and 1 deletion.
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -360,7 +360,7 @@ require (
github.com/tilinna/z85 v1.0.0 // indirect
github.com/tklauser/go-sysconf v0.3.13 // indirect
github.com/tklauser/numcpus v0.7.0 // indirect
github.com/urfave/cli/v2 v2.27.4 // indirect
github.com/urfave/cli/v2 v2.27.4
github.com/vmihailenco/tagparser/v2 v2.0.0 // indirect
github.com/xdg-go/pbkdf2 v1.0.0 // indirect
github.com/xdg-go/stringprep v1.0.4 // indirect
Expand Down
26 changes: 26 additions & 0 deletions internal/cli/enterprise.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,8 +16,10 @@ import (

"github.com/redpanda-data/benthos/v4/public/service"
"github.com/rs/xid"
"github.com/urfave/cli/v2"

"github.com/redpanda-data/connect/v4/internal/impl/kafka/enterprise"
"github.com/redpanda-data/connect/v4/internal/secrets"
"github.com/redpanda-data/connect/v4/internal/telemetry"
)

Expand All @@ -39,6 +41,10 @@ func InitEnterpriseCLI(binaryName, version, dateBuilt string, schema *service.Co
os.Exit(1)
}

secretLookupFn := func(ctx context.Context, key string) (string, bool) {
return "", false
}

opts = append(opts,
service.CLIOptSetVersion(version, dateBuilt),
service.CLIOptSetBinaryName(binaryName),
Expand Down Expand Up @@ -82,6 +88,26 @@ func InitEnterpriseCLI(binaryName, version, dateBuilt string, schema *service.Co
rpLogger.SetStreamSummary(s)
return nil
}),

// Secrets management
service.CLIOptCustomRunFlags([]cli.Flag{
&cli.StringSliceFlag{
Name: "secrets",
Usage: "Attempt to load secrets from a provided URN. If more than one entry is specified they will be attempted in order until a value is found. Environment variable lookups are specified with the URN `env:`, which by default is the only entry. In order to disable all secret lookups specify a single entry of `none:`.",
Value: cli.NewStringSlice("env:"),
},
}, func(c *cli.Context) error {
secretsURNs := c.StringSlice("secrets")
if len(secretsURNs) > 0 {
var err error
secretLookupFn, err = secrets.ParseLookupURNs(c.Context, slog.New(rpLogger), secretsURNs...)
return err
}
return nil
}),
service.CLIOptSetEnvVarLookup(func(ctx context.Context, key string) (string, bool) {
return secretLookupFn(ctx, key)
}),
)

exitCode, err := service.RunCLIToCode(context.Background(), opts...)
Expand Down
48 changes: 48 additions & 0 deletions internal/secrets/redis.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,48 @@
// Copyright 2024 Redpanda Data, Inc.
//
// Licensed as a Redpanda Enterprise file under the Redpanda Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md

package secrets

import (
"context"
"errors"
"log/slog"
"net/url"

"github.com/redis/go-redis/v9"
)

type redisSecretsClient struct {
logger *slog.Logger
client *redis.Client
}

func (r *redisSecretsClient) lookup(ctx context.Context, key string) (string, bool) {
res, err := r.client.Get(ctx, key).Result()
if err != nil {
if !errors.Is(err, redis.Nil) {
// An error that isn't due to key-not-found gets logged
r.logger.With("error", err, "key", key).Error("Failed to look up secret")
}
return "", false
}
return res, true
}

func newRedisSecretsLookup(ctx context.Context, logger *slog.Logger, url *url.URL) (LookupFn, error) {
opts, err := redis.ParseURL(url.String())
if err != nil {
return nil, err
}

r := &redisSecretsClient{
logger: logger,
client: redis.NewClient(opts),
}
return r.lookup, nil
}
75 changes: 75 additions & 0 deletions internal/secrets/redis_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,75 @@
// Copyright 2024 Redpanda Data, Inc.
//
// Licensed as a Redpanda Enterprise file under the Redpanda Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md

package secrets

import (
"context"
"fmt"
"log/slog"
"net/url"
"testing"
"time"

"github.com/ory/dockertest/v3"
"github.com/redis/go-redis/v9"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"

_ "github.com/redpanda-data/benthos/v4/public/components/pure"
"github.com/redpanda-data/benthos/v4/public/service/integration"
)

func TestIntegrationRedis(t *testing.T) {
integration.CheckSkip(t)
t.Parallel()

pool, err := dockertest.NewPool("")
require.NoError(t, err)

pool.MaxWait = time.Second * 30
resource, err := pool.Run("redis", "latest", nil)
require.NoError(t, err)
t.Cleanup(func() {
assert.NoError(t, pool.Purge(resource))
})

urlStr := fmt.Sprintf("redis://localhost:%v", resource.GetPort("6379/tcp"))
uri, err := url.Parse(urlStr)
if err != nil {
t.Fatal(err)
}

opts, err := redis.ParseURL(uri.String())
if err != nil {
t.Fatal(err)
}

client := redis.NewClient(opts)

_ = resource.Expire(900)
require.NoError(t, pool.Retry(func() error {
return client.Ping(context.Background()).Err()
}))

ctx, done := context.WithTimeout(context.Background(), time.Minute)
defer done()

require.NoError(t, client.Set(ctx, "bar", "meow", time.Minute).Err())

secretsLookup, err := parseSecretsLookupURN(ctx, slog.Default(), urlStr)
require.NoError(t, err)

v, exists := secretsLookup(ctx, "foo")
assert.False(t, exists)
assert.Equal(t, "", v)

v, exists = secretsLookup(ctx, "bar")
assert.True(t, exists)
assert.Equal(t, "meow", v)
}
83 changes: 83 additions & 0 deletions internal/secrets/secrets.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,83 @@
// Copyright 2024 Redpanda Data, Inc.
//
// Licensed as a Redpanda Enterprise file under the Redpanda Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md

package secrets

import (
"context"
"fmt"
"log/slog"
"net/url"
"os"
)

// LookupFn defines the common closure that a secrets management client provides
// and is then fed into a Redpanda Connect cli constructor.
type LookupFn func(context.Context, string) (string, bool)

type lookupTiers []LookupFn

func (l lookupTiers) Lookup(ctx context.Context, key string) (string, bool) {
for _, fn := range l {
if v, ok := fn(ctx, key); ok {
return v, ok
}
if ctx.Err() != nil {
break
}
}
return "", false
}

// ParseLookupURNs attempts to parse a series of secrets lookup solutions
// defined as URNs and returns a single lookup func for obtaining secrets from
// them in the order provided.
//
// A toggle can be provided that determines whether environment variables should
// be considered the last look up option, in which case if all others fail to
// provide a secret then an environment variable under the key is returned if
// found.
func ParseLookupURNs(ctx context.Context, logger *slog.Logger, secretsMgmtUrns ...string) (LookupFn, error) {
var tiers lookupTiers

for _, urn := range secretsMgmtUrns {
tier, err := parseSecretsLookupURN(ctx, logger, urn)
if err != nil {
return nil, err
}
tiers = append(tiers, tier)
}

return tiers.Lookup, nil
}

func parseSecretsLookupURN(ctx context.Context, logger *slog.Logger, urn string) (LookupFn, error) {
u, err := url.Parse(urn)
if err != nil {
return nil, err
}

switch u.Scheme {
case "test":
return func(ctx context.Context, key string) (string, bool) {
return key + " " + u.Host, true
}, nil
case "redis":
return newRedisSecretsLookup(ctx, logger, u)
case "env":
return func(ctx context.Context, key string) (string, bool) {
return os.LookupEnv(key)
}, nil
case "none":
return func(ctx context.Context, key string) (string, bool) {
return "", false
}, nil
default:
return nil, fmt.Errorf("secrets scheme %v not recognized", u.Scheme)
}
}

0 comments on commit 23f5308

Please sign in to comment.