-
Notifications
You must be signed in to change notification settings - Fork 840
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Add cli flag for secrets backends (#2964)
* 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
Showing
5 changed files
with
233 additions
and
1 deletion.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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 | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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) | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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) | ||
} | ||
} |