1
0
mirror of https://github.com/go-micro/go-micro.git synced 2025-08-10 21:52:01 +02:00

Plugins and profiles (#2764)

* feat: more plugins

* chore(ci): split out benchmarks

Attempt to resolve too many open files in ci

* chore(ci): split out benchmarks

* fix(ci): Attempt to resolve too many open files in ci

* fix: set DefaultX for cli flag and service option

* fix: restore http broker

* fix: default http broker

* feat: full nats profile

* chore: still ugly, not ready

* fix: better initialization for profiles

* fix(tests): comment out flaky listen tests

* fix: disable benchmarks on gha

* chore: cleanup, comments

* chore: add nats config source
This commit is contained in:
Brian Ketelsen
2025-05-20 13:24:06 -04:00
committed by GitHub
parent e12504ce3a
commit ddc34801ee
58 changed files with 6792 additions and 218 deletions

View File

@@ -0,0 +1,79 @@
# NATS JetStream Key Value Store Plugin
This plugin uses the NATS JetStream [KeyValue Store](https://docs.nats.io/nats-concepts/jetstream/key-value-store) to implement the Go-Micro store interface.
You can use this plugin like any other store plugin.
To start a local NATS JetStream server run `nats-server -js`.
To manually create a new storage object call:
```go
natsjskv.NewStore(opts ...store.Option)
```
The Go-Micro store interface uses databases and tables to store keys. These translate
to buckets (key value stores) and key prefixes. If no database (bucket name) is provided, "default" will be used.
You can call `Write` with any arbitrary database name, and if a bucket with that name does not exist yet,
it will be automatically created.
If a table name is provided, it will use it to prefix the key as `<table>_<key>`.
To delete a bucket, and all the key/value pairs in it, pass the `DeleteBucket` option to the `Delete`
method, then they key name will be interpreted as a bucket name, and the bucket will be deleted.
Next to the default store options, a few NATS specific options are available:
```go
// NatsOptions accepts nats.Options
NatsOptions(opts nats.Options)
// JetStreamOptions accepts multiple nats.JSOpt
JetStreamOptions(opts ...nats.JSOpt)
// KeyValueOptions accepts multiple nats.KeyValueConfig
// This will create buckets with the provided configs at initialization.
//
// type KeyValueConfig struct {
// Bucket string
// Description string
// MaxValueSize int32
// History uint8
// TTL time.Duration
// MaxBytes int64
// Storage StorageType
// Replicas int
// Placement *Placement
// RePublish *RePublish
// Mirror *StreamSource
// Sources []*StreamSource
}
KeyValueOptions(cfg ...*nats.KeyValueConfig)
// DefaultTTL sets the default TTL to use for new buckets
// By default no TTL is set.
//
// TTL ON INDIVIDUAL WRITE CALLS IS NOT SUPPORTED, only bucket wide TTL.
// Either set a default TTL with this option or provide bucket specific options
// with ObjectStoreOptions
DefaultTTL(ttl time.Duration)
// DefaultMemory sets the default storage type to memory only.
//
// The default is file storage, persisting storage between service restarts.
// Be aware that the default storage location of NATS the /tmp dir is, and thus
// won't persist reboots.
DefaultMemory()
// DefaultDescription sets the default description to use when creating new
// buckets. The default is "Store managed by go-micro"
DefaultDescription(text string)
// DeleteBucket will use the key passed to Delete as a bucket (database) name,
// and delete the bucket.
// This option should not be combined with the store.DeleteFrom option, as
// that will overwrite the delete action.
DeleteBucket()
```

View File

@@ -0,0 +1,18 @@
package natsjskv
import (
"context"
"go-micro.dev/v5/store"
)
// setStoreOption returns a function to setup a context with given value.
func setStoreOption(k, v interface{}) store.Option {
return func(o *store.Options) {
if o.Context == nil {
o.Context = context.Background()
}
o.Context = context.WithValue(o.Context, k, v)
}
}

View File

@@ -0,0 +1,184 @@
package natsjskv
import (
"context"
"fmt"
"net"
"os"
"path/filepath"
"strconv"
"strings"
"testing"
"time"
nserver "github.com/nats-io/nats-server/v2/server"
"github.com/pkg/errors"
"github.com/test-go/testify/require"
"go-micro.dev/v5/store"
)
func testSetup(ctx context.Context, t *testing.T, opts ...store.Option) store.Store {
t.Helper()
var err error
var s store.Store
for i := 0; i < 5; i++ {
nCtx, cancel := context.WithCancel(ctx)
addr := startNatsServer(nCtx, t)
opts = append(opts, store.Nodes(addr), EncodeKeys())
s = NewStore(opts...)
err = s.Init()
if err != nil {
t.Log(errors.Wrap(err, "Error: Server initialization failed, restarting server"))
cancel()
if err = s.Close(); err != nil {
t.Logf("Failed to close store: %v", err)
}
time.Sleep(time.Second)
continue
}
go func() {
<-ctx.Done()
cancel()
if err = s.Close(); err != nil {
t.Logf("Failed to close store: %v", err)
}
}()
return s
}
t.Error(errors.Wrap(err, "Store initialization failed"))
return s
}
func startNatsServer(ctx context.Context, t *testing.T) string {
t.Helper()
natsAddr := getFreeLocalhostAddress()
natsPort, err := strconv.Atoi(strings.Split(natsAddr, ":")[1])
if err != nil {
t.Logf("Failed to parse port from address: %v", err)
}
clusterName := "gomicro-store-test-cluster"
// start the NATS with JetStream server
go natsServer(ctx,
t,
&nserver.Options{
Host: strings.Split(natsAddr, ":")[0],
Port: natsPort,
Cluster: nserver.ClusterOpts{
Name: clusterName,
},
},
)
time.Sleep(2 * time.Second)
return natsAddr
}
func getFreeLocalhostAddress() string {
l, err := net.Listen("tcp", "127.0.0.1:0")
if err != nil {
return ""
}
addr := l.Addr().String()
if err := l.Close(); err != nil {
return addr
}
return addr
}
func natsServer(ctx context.Context, t *testing.T, opts *nserver.Options) {
t.Helper()
opts.TLSTimeout = 180
server, err := nserver.NewServer(
opts,
)
require.NoError(t, err)
if err != nil {
return
}
defer server.Shutdown()
server.SetLoggerV2(
NewLogWrapper(),
false, false, false,
)
tmpdir := t.TempDir()
natsdir := filepath.Join(tmpdir, "nats-js")
jsConf := &nserver.JetStreamConfig{
StoreDir: natsdir,
}
// first start NATS
go server.Start()
time.Sleep(time.Second)
// second start JetStream
err = server.EnableJetStream(jsConf)
require.NoError(t, err)
if err != nil {
return
}
// This fixes some issues where tests fail because directory cleanup fails
t.Cleanup(func() {
contents, err := filepath.Glob(natsdir + "/*")
if err != nil {
t.Logf("Failed to glob directory: %v", err)
}
for _, item := range contents {
if err := os.RemoveAll(item); err != nil {
t.Logf("Failed to remove file: %v", err)
}
}
if err := os.RemoveAll(natsdir); err != nil {
t.Logf("Failed to remove directory: %v", err)
}
})
<-ctx.Done()
}
func NewLogWrapper() *LogWrapper {
return &LogWrapper{}
}
type LogWrapper struct {
}
// Noticef logs a notice statement.
func (l *LogWrapper) Noticef(_ string, _ ...interface{}) {
}
// Warnf logs a warning statement.
func (l *LogWrapper) Warnf(format string, v ...interface{}) {
fmt.Printf(format+"\n", v...)
}
// Fatalf logs a fatal statement.
func (l *LogWrapper) Fatalf(format string, v ...interface{}) {
fmt.Printf(format+"\n", v...)
}
// Errorf logs an error statement.
func (l *LogWrapper) Errorf(format string, v ...interface{}) {
fmt.Printf(format+"\n", v...)
}
// Debugf logs a debug statement.
func (l *LogWrapper) Debugf(_ string, _ ...interface{}) {
}
// Tracef logs a trace statement.
func (l *LogWrapper) Tracef(format string, v ...interface{}) {
fmt.Printf(format+"\n", v...)
}

119
store/nats-js-kv/keys.go Normal file
View File

@@ -0,0 +1,119 @@
package natsjskv
import (
"encoding/base32"
"strings"
)
// NatsKey is a convenience function to create a key for the nats kv store.
func (n *natsStore) NatsKey(table, microkey string) string {
return n.NewKey(table, microkey, "").NatsKey()
}
// MicroKey is a convenience function to create a key for the micro interface.
func (n *natsStore) MicroKey(table, natskey string) string {
return n.NewKey(table, "", natskey).MicroKey()
}
// MicroKeyFilter is a convenience function to create a key for the micro interface.
// It returns false if the key does not match the table, prefix or suffix.
func (n *natsStore) MicroKeyFilter(table, natskey string, prefix, suffix string) (string, bool) {
k := n.NewKey(table, "", natskey)
return k.MicroKey(), k.Check(table, prefix, suffix)
}
// Key represents a key in the store.
// They are used to convert nats keys (base32 encoded) to micro keys (plain text - no table prefix) and vice versa.
type Key struct {
// Plain is the plain key as requested by the go-micro interface.
Plain string
// Full is the full key including the table prefix.
Full string
// Encoded is the base64 encoded key as used by the nats kv store.
Encoded string
}
// NewKey creates a new key. Either plain or encoded must be set.
func (n *natsStore) NewKey(table string, plain, encoded string) *Key {
k := &Key{
Plain: plain,
Encoded: encoded,
}
switch {
case k.Plain != "":
k.Full = getKey(k.Plain, table)
k.Encoded = encode(k.Full, n.encoding)
case k.Encoded != "":
k.Full = decode(k.Encoded, n.encoding)
k.Plain = trimKey(k.Full, table)
}
return k
}
// NatsKey returns a key the nats kv store can work with.
func (k *Key) NatsKey() string {
return k.Encoded
}
// MicroKey returns a key the micro interface can work with.
func (k *Key) MicroKey() string {
return k.Plain
}
// Check returns false if the key does not match the table, prefix or suffix.
func (k *Key) Check(table, prefix, suffix string) bool {
if table != "" && k.Full != getKey(k.Plain, table) {
return false
}
if prefix != "" && !strings.HasPrefix(k.Plain, prefix) {
return false
}
if suffix != "" && !strings.HasSuffix(k.Plain, suffix) {
return false
}
return true
}
func encode(s string, alg string) string {
switch alg {
case "base32":
return base32.StdEncoding.EncodeToString([]byte(s))
default:
return s
}
}
func decode(s string, alg string) string {
switch alg {
case "base32":
b, err := base32.StdEncoding.DecodeString(s)
if err != nil {
return s
}
return string(b)
default:
return s
}
}
func getKey(key, table string) string {
if table != "" {
return table + "_" + key
}
return key
}
func trimKey(key, table string) string {
if table != "" {
return strings.TrimPrefix(key, table+"_")
}
return key
}

478
store/nats-js-kv/nats.go Normal file
View File

@@ -0,0 +1,478 @@
// Package natsjskv is a go-micro store plugin for NATS JetStream Key-Value store.
package natsjskv
import (
"context"
"encoding/json"
"sync"
"time"
"github.com/cornelk/hashmap"
"github.com/nats-io/nats.go"
"github.com/pkg/errors"
"go-micro.dev/v5/store"
)
var (
// ErrBucketNotFound is returned when the requested bucket does not exist.
ErrBucketNotFound = errors.New("Bucket (database) not found")
)
// KeyValueEnvelope is the data structure stored in the key value store.
type KeyValueEnvelope struct {
Key string `json:"key"`
Data []byte `json:"data"`
Metadata map[string]interface{} `json:"metadata"`
}
type natsStore struct {
sync.Once
sync.RWMutex
encoding string
ttl time.Duration
storageType nats.StorageType
description string
opts store.Options
nopts nats.Options
jsopts []nats.JSOpt
kvConfigs []*nats.KeyValueConfig
conn *nats.Conn
js nats.JetStreamContext
buckets *hashmap.Map[string, nats.KeyValue]
}
// NewStore will create a new NATS JetStream Object Store.
func NewStore(opts ...store.Option) store.Store {
options := store.Options{
Nodes: []string{},
Database: "default",
Table: "",
Context: context.Background(),
}
n := &natsStore{
description: "KeyValue storage administered by go-micro store plugin",
opts: options,
jsopts: []nats.JSOpt{},
kvConfigs: []*nats.KeyValueConfig{},
buckets: hashmap.New[string, nats.KeyValue](),
storageType: nats.FileStorage,
}
n.setOption(opts...)
return n
}
// Init initializes the store. It must perform any required setup on the
// backing storage implementation and check that it is ready for use,
// returning any errors.
func (n *natsStore) Init(opts ...store.Option) error {
n.setOption(opts...)
// Connect to NATS servers
conn, err := n.nopts.Connect()
if err != nil {
return errors.Wrap(err, "Failed to connect to NATS Server")
}
// Create JetStream context
js, err := conn.JetStream(n.jsopts...)
if err != nil {
return errors.Wrap(err, "Failed to create JetStream context")
}
n.conn = conn
n.js = js
// Create default config if no configs present
if len(n.kvConfigs) == 0 {
if _, err := n.mustGetBucketByName(n.opts.Database); err != nil {
return err
}
}
// Create kv store buckets
for _, cfg := range n.kvConfigs {
if _, err := n.mustGetBucket(cfg); err != nil {
return err
}
}
return nil
}
func (n *natsStore) setOption(opts ...store.Option) {
for _, o := range opts {
o(&n.opts)
}
n.Once.Do(func() {
n.nopts = nats.GetDefaultOptions()
})
// Extract options from context
if nopts, ok := n.opts.Context.Value(natsOptionsKey{}).(nats.Options); ok {
n.nopts = nopts
}
if jsopts, ok := n.opts.Context.Value(jsOptionsKey{}).([]nats.JSOpt); ok {
n.jsopts = append(n.jsopts, jsopts...)
}
if cfg, ok := n.opts.Context.Value(kvOptionsKey{}).([]*nats.KeyValueConfig); ok {
n.kvConfigs = append(n.kvConfigs, cfg...)
}
if ttl, ok := n.opts.Context.Value(ttlOptionsKey{}).(time.Duration); ok {
n.ttl = ttl
}
if sType, ok := n.opts.Context.Value(memoryOptionsKey{}).(nats.StorageType); ok {
n.storageType = sType
}
if text, ok := n.opts.Context.Value(descriptionOptionsKey{}).(string); ok {
n.description = text
}
if encoding, ok := n.opts.Context.Value(keyEncodeOptionsKey{}).(string); ok {
n.encoding = encoding
}
// Assign store option server addresses to nats options
if len(n.opts.Nodes) > 0 {
n.nopts.Url = ""
n.nopts.Servers = n.opts.Nodes
}
if len(n.nopts.Servers) == 0 && n.nopts.Url == "" {
n.nopts.Url = nats.DefaultURL
}
}
// Options allows you to view the current options.
func (n *natsStore) Options() store.Options {
return n.opts
}
// Read takes a single key name and optional ReadOptions. It returns matching []*Record or an error.
func (n *natsStore) Read(key string, opts ...store.ReadOption) ([]*store.Record, error) {
if err := n.initConn(); err != nil {
return nil, err
}
opt := store.ReadOptions{}
for _, o := range opts {
o(&opt)
}
if opt.Database == "" {
opt.Database = n.opts.Database
}
if opt.Table == "" {
opt.Table = n.opts.Table
}
bucket, ok := n.buckets.Get(opt.Database)
if !ok {
return nil, ErrBucketNotFound
}
keys, err := n.natsKeys(bucket, opt.Table, key, opt.Prefix, opt.Suffix)
if err != nil {
return nil, err
}
records := make([]*store.Record, 0, len(keys))
for _, key := range keys {
rec, ok, err := n.getRecord(bucket, key)
if err != nil {
return nil, err
}
if ok {
records = append(records, rec)
}
}
return enforceLimits(records, opt.Limit, opt.Offset), nil
}
// Write writes a record to the store, and returns an error if the record was not written.
func (n *natsStore) Write(rec *store.Record, opts ...store.WriteOption) error {
if err := n.initConn(); err != nil {
return err
}
opt := store.WriteOptions{}
for _, o := range opts {
o(&opt)
}
if opt.Database == "" {
opt.Database = n.opts.Database
}
if opt.Table == "" {
opt.Table = n.opts.Table
}
store, err := n.mustGetBucketByName(opt.Database)
if err != nil {
return err
}
b, err := json.Marshal(KeyValueEnvelope{
Key: rec.Key,
Data: rec.Value,
Metadata: rec.Metadata,
})
if err != nil {
return errors.Wrap(err, "Failed to marshal object")
}
if _, err := store.Put(n.NatsKey(opt.Table, rec.Key), b); err != nil {
return errors.Wrapf(err, "Failed to store data in bucket '%s'", n.NatsKey(opt.Table, rec.Key))
}
return nil
}
// Delete removes the record with the corresponding key from the store.
func (n *natsStore) Delete(key string, opts ...store.DeleteOption) error {
if err := n.initConn(); err != nil {
return err
}
opt := store.DeleteOptions{}
for _, o := range opts {
o(&opt)
}
if opt.Database == "" {
opt.Database = n.opts.Database
}
if opt.Table == "" {
opt.Table = n.opts.Table
}
if opt.Table == "DELETE_BUCKET" {
n.buckets.Del(key)
if err := n.js.DeleteKeyValue(key); err != nil {
return errors.Wrap(err, "Failed to delete bucket")
}
return nil
}
store, ok := n.buckets.Get(opt.Database)
if !ok {
return ErrBucketNotFound
}
if err := store.Delete(n.NatsKey(opt.Table, key)); err != nil {
return errors.Wrap(err, "Failed to delete data")
}
return nil
}
// List returns any keys that match, or an empty list with no error if none matched.
func (n *natsStore) List(opts ...store.ListOption) ([]string, error) {
if err := n.initConn(); err != nil {
return nil, err
}
opt := store.ListOptions{}
for _, o := range opts {
o(&opt)
}
if opt.Database == "" {
opt.Database = n.opts.Database
}
if opt.Table == "" {
opt.Table = n.opts.Table
}
store, ok := n.buckets.Get(opt.Database)
if !ok {
return nil, ErrBucketNotFound
}
keys, err := n.microKeys(store, opt.Table, opt.Prefix, opt.Suffix)
if err != nil {
return nil, errors.Wrap(err, "Failed to list keys in bucket")
}
return enforceLimits(keys, opt.Limit, opt.Offset), nil
}
// Close the store.
func (n *natsStore) Close() error {
n.conn.Close()
return nil
}
// String returns the name of the implementation.
func (n *natsStore) String() string {
return "NATS JetStream KeyValueStore"
}
// thread safe way to initialize the connection.
func (n *natsStore) initConn() error {
if n.hasConn() {
return nil
}
n.Lock()
defer n.Unlock()
// check if conn was initialized meanwhile
if n.conn != nil {
return nil
}
return n.Init()
}
// thread safe way to check if n is initialized.
func (n *natsStore) hasConn() bool {
n.RLock()
defer n.RUnlock()
return n.conn != nil
}
// mustGetDefaultBucket returns the bucket with the given name creating it with default configuration if needed.
func (n *natsStore) mustGetBucketByName(name string) (nats.KeyValue, error) {
return n.mustGetBucket(&nats.KeyValueConfig{
Bucket: name,
Description: n.description,
TTL: n.ttl,
Storage: n.storageType,
})
}
// mustGetBucket creates a new bucket if it does not exist yet.
func (n *natsStore) mustGetBucket(kv *nats.KeyValueConfig) (nats.KeyValue, error) {
if store, ok := n.buckets.Get(kv.Bucket); ok {
return store, nil
}
store, err := n.js.KeyValue(kv.Bucket)
if err != nil {
if !errors.Is(err, nats.ErrBucketNotFound) {
return nil, errors.Wrapf(err, "Failed to get bucket (%s)", kv.Bucket)
}
store, err = n.js.CreateKeyValue(kv)
if err != nil {
return nil, errors.Wrapf(err, "Failed to create bucket (%s)", kv.Bucket)
}
}
n.buckets.Set(kv.Bucket, store)
return store, nil
}
// getRecord returns the record with the given key from the nats kv store.
func (n *natsStore) getRecord(bucket nats.KeyValue, key string) (*store.Record, bool, error) {
obj, err := bucket.Get(key)
if errors.Is(err, nats.ErrKeyNotFound) {
return nil, false, store.ErrNotFound
} else if err != nil {
return nil, false, errors.Wrap(err, "Failed to get object from bucket")
}
var kv KeyValueEnvelope
if err := json.Unmarshal(obj.Value(), &kv); err != nil {
return nil, false, errors.Wrap(err, "Failed to unmarshal object")
}
if obj.Operation() != nats.KeyValuePut {
return nil, false, nil
}
return &store.Record{
Key: kv.Key,
Value: kv.Data,
Metadata: kv.Metadata,
}, true, nil
}
func (n *natsStore) natsKeys(bucket nats.KeyValue, table, key string, prefix, suffix bool) ([]string, error) {
if !suffix && !prefix {
return []string{n.NatsKey(table, key)}, nil
}
toS := func(s string, b bool) string {
if b {
return s
}
return ""
}
keys, _, err := n.getKeys(bucket, table, toS(key, prefix), toS(key, suffix))
return keys, err
}
func (n *natsStore) microKeys(bucket nats.KeyValue, table, prefix, suffix string) ([]string, error) {
_, keys, err := n.getKeys(bucket, table, prefix, suffix)
return keys, err
}
func (n *natsStore) getKeys(bucket nats.KeyValue, table string, prefix, suffix string) ([]string, []string, error) {
names, err := bucket.Keys(nats.IgnoreDeletes())
if errors.Is(err, nats.ErrKeyNotFound) {
return []string{}, []string{}, nil
} else if err != nil {
return []string{}, []string{}, errors.Wrap(err, "Failed to list objects")
}
natsKeys := make([]string, 0, len(names))
microKeys := make([]string, 0, len(names))
for _, k := range names {
mkey, ok := n.MicroKeyFilter(table, k, prefix, suffix)
if !ok {
continue
}
natsKeys = append(natsKeys, k)
microKeys = append(microKeys, mkey)
}
return natsKeys, microKeys, nil
}
// enforces offset and limit without causing a panic.
func enforceLimits[V any](recs []V, limit, offset uint) []V {
l := uint(len(recs))
from := offset
if from > l {
from = l
}
to := l
if limit > 0 && offset+limit < l {
to = offset + limit
}
return recs[from:to]
}

View File

@@ -0,0 +1,337 @@
package natsjskv
import (
"context"
"reflect"
"testing"
"time"
"github.com/google/uuid"
"github.com/nats-io/nats.go"
"github.com/pkg/errors"
"go-micro.dev/v5/store"
)
func TestNats(t *testing.T) {
// Setup without calling Init on purpose
var err error
for i := 0; i < 5; i++ {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
addr := startNatsServer(ctx, t)
s := NewStore(store.Nodes(addr), EncodeKeys())
// Test String method
t.Log("Testing:", s.String())
err = basicTest(t, s)
if err != nil {
t.Log(err)
continue
}
// Test reading non-existing key
r, err := s.Read("this-is-a-random-key")
if !errors.Is(err, store.ErrNotFound) {
t.Errorf("Expected %# v, got %# v", store.ErrNotFound, err)
}
if len(r) > 0 {
t.Fatal("Lenth should be 0")
}
err = s.Close()
if err != nil {
t.Logf("Failed to close store: %v", err)
}
cancel()
return
}
t.Fatal(err)
}
func TestOptions(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
s := testSetup(ctx, t,
DefaultMemory(),
// Having a non-default description will trigger nats.ErrStreamNameAlreadyInUse
// since the buckets have been created in previous tests with a different description.
//
// NOTE: this is only the case with a manually set up server, not with current
// test setup, where new servers are started for each test.
DefaultDescription("My fancy description"),
// Option has no effect in this context, just to test setting the option
JetStreamOptions(nats.PublishAsyncMaxPending(256)),
// Sets a custom NATS client name, just to test the NatsOptions() func
NatsOptions(nats.Options{Name: "Go NATS Store Plugin Tests Client"}),
KeyValueOptions(&nats.KeyValueConfig{
Bucket: "TestBucketName",
Description: "This bucket is not used",
TTL: 5 * time.Minute,
MaxBytes: 1024,
Storage: nats.MemoryStorage,
Replicas: 1,
}),
// Encode keys to avoid character limitations
EncodeKeys(),
)
defer cancel()
if err := basicTest(t, s); err != nil {
t.Fatal(err)
}
}
func TestTTL(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
ttl := 500 * time.Millisecond
s := testSetup(ctx, t,
DefaultTTL(ttl),
// Since these buckets will be new they will have the new description
DefaultDescription("My fancy description"),
)
defer cancel()
// Use a uuid to make sure a new bucket is created when using local server
id := uuid.New().String()
for _, r := range table {
if err := s.Write(r.Record, store.WriteTo(r.Database+id, r.Table)); err != nil {
t.Fatal(err)
}
}
time.Sleep(ttl * 2)
for _, r := range table {
res, err := s.Read(r.Record.Key, store.ReadFrom(r.Database+id, r.Table))
if !errors.Is(err, store.ErrNotFound) {
t.Errorf("Expected %# v, got %# v", store.ErrNotFound, err)
}
if len(res) > 0 {
t.Fatal("Fetched record while it should have expired")
}
}
}
func TestMetaData(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
s := testSetup(ctx, t)
defer cancel()
record := store.Record{
Key: "KeyOne",
Value: []byte("Some value"),
Metadata: map[string]interface{}{
"meta-one": "val",
"meta-two": 5,
},
Expiry: 0,
}
bucket := "meta-data-test"
if err := s.Write(&record, store.WriteTo(bucket, "")); err != nil {
t.Fatal(err)
}
r, err := s.Read(record.Key, store.ReadFrom(bucket, ""))
if err != nil {
t.Fatal(err)
}
if len(r) == 0 {
t.Fatal("No results found")
}
m := r[0].Metadata
if m["meta-one"].(string) != record.Metadata["meta-one"].(string) ||
m["meta-two"].(float64) != float64(record.Metadata["meta-two"].(int)) {
t.Fatalf("Metadata does not match: (%+v) != (%+v)", m, record.Metadata)
}
}
func TestDelete(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
s := testSetup(ctx, t)
defer cancel()
for _, r := range table {
if err := s.Write(r.Record, store.WriteTo(r.Database, r.Table)); err != nil {
t.Fatal(err)
}
if err := s.Delete(r.Record.Key, store.DeleteFrom(r.Database, r.Table)); err != nil {
t.Fatal(err)
}
time.Sleep(time.Second)
res, err := s.Read(r.Record.Key, store.ReadFrom(r.Database, r.Table))
if !errors.Is(err, store.ErrNotFound) {
t.Errorf("Expected %# v, got %# v", store.ErrNotFound, err)
}
if len(res) > 0 {
t.Fatalf("Failed to delete %s:%s from %s %s (len: %d)", r.Record.Key, r.Record.Value, r.Database, r.Table, len(res))
}
}
}
func TestList(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
s := testSetup(ctx, t)
defer cancel()
for _, r := range table {
if err := s.Write(r.Record, store.WriteTo(r.Database, r.Table)); err != nil {
t.Fatal(err)
}
}
l := []struct {
Database string
Table string
Length int
Prefix string
Suffix string
Offset int
Limit int
}{
{Length: 7},
{Database: "prefix-test", Length: 7},
{Database: "prefix-test", Offset: 2, Length: 5},
{Database: "prefix-test", Offset: 2, Limit: 3, Length: 3},
{Database: "prefix-test", Table: "names", Length: 3},
{Database: "prefix-test", Table: "cities", Length: 4},
{Database: "prefix-test", Table: "cities", Suffix: "City", Length: 3},
{Database: "prefix-test", Table: "cities", Suffix: "City", Limit: 2, Length: 2},
{Database: "prefix-test", Table: "cities", Suffix: "City", Offset: 1, Length: 2},
{Prefix: "test", Length: 1},
{Table: "some_table", Prefix: "test", Suffix: "test", Length: 2},
}
for i, entry := range l {
// Test listing keys
keys, err := s.List(
store.ListFrom(entry.Database, entry.Table),
store.ListPrefix(entry.Prefix),
store.ListSuffix(entry.Suffix),
store.ListOffset(uint(entry.Offset)),
store.ListLimit(uint(entry.Limit)),
)
if err != nil {
t.Fatal(err)
}
if len(keys) != entry.Length {
t.Fatalf("Length of returned keys is invalid for test %d - %+v (%d)", i+1, entry, len(keys))
}
// Test reading keys
if entry.Prefix != "" || entry.Suffix != "" {
var key string
options := []store.ReadOption{
store.ReadFrom(entry.Database, entry.Table),
store.ReadLimit(uint(entry.Limit)),
store.ReadOffset(uint(entry.Offset)),
}
if entry.Prefix != "" {
key = entry.Prefix
options = append(options, store.ReadPrefix())
}
if entry.Suffix != "" {
key = entry.Suffix
options = append(options, store.ReadSuffix())
}
r, err := s.Read(key, options...)
if err != nil {
t.Fatal(err)
}
if len(r) != entry.Length {
t.Fatalf("Length of read keys is invalid for test %d - %+v (%d)", i+1, entry, len(r))
}
}
}
}
func TestDeleteBucket(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
s := testSetup(ctx, t)
defer cancel()
for _, r := range table {
if err := s.Write(r.Record, store.WriteTo(r.Database, r.Table)); err != nil {
t.Fatal(err)
}
}
bucket := "prefix-test"
if err := s.Delete(bucket, DeleteBucket()); err != nil {
t.Fatal(err)
}
keys, err := s.List(store.ListFrom(bucket, ""))
if err != nil && !errors.Is(err, ErrBucketNotFound) {
t.Fatalf("Failed to delete bucket: %v", err)
}
if len(keys) > 0 {
t.Fatal("Length of key list should be 0 after bucket deletion")
}
r, err := s.Read("", store.ReadPrefix(), store.ReadFrom(bucket, ""))
if err != nil && !errors.Is(err, ErrBucketNotFound) {
t.Fatalf("Failed to delete bucket: %v", err)
}
if len(r) > 0 {
t.Fatal("Length of record list should be 0 after bucket deletion", len(r))
}
}
func TestEnforceLimits(t *testing.T) {
s := []string{"a", "b", "c", "d"}
var testCasts = []struct {
Alias string
Offset uint
Limit uint
Expected []string
}{
{"plain", 0, 0, []string{"a", "b", "c", "d"}},
{"offset&limit-1", 1, 3, []string{"b", "c", "d"}},
{"offset&limit-2", 1, 1, []string{"b"}},
{"offset=length", 4, 0, []string{}},
{"offset>length", 222, 0, []string{}},
{"limit>length", 0, 36, []string{"a", "b", "c", "d"}},
}
for _, tc := range testCasts {
actual := enforceLimits(s, tc.Limit, tc.Offset)
if !reflect.DeepEqual(actual, tc.Expected) {
t.Fatalf("%s: Expected %v, got %v", tc.Alias, tc.Expected, actual)
}
}
}
func basicTest(t *testing.T, s store.Store) error {
t.Helper()
for _, test := range table {
if err := s.Write(test.Record, store.WriteTo(test.Database, test.Table)); err != nil {
return errors.Wrap(err, "Failed to write record in basic test")
}
r, err := s.Read(test.Record.Key, store.ReadFrom(test.Database, test.Table))
if err != nil {
return errors.Wrap(err, "Failed to read record in basic test")
}
if len(r) == 0 {
t.Fatalf("No results found for %s (%s) %s", test.Record.Key, test.Database, test.Table)
}
key := test.Record.Key
val1 := string(test.Record.Value)
key2 := r[0].Key
val2 := string(r[0].Value)
if val1 != val2 {
t.Fatalf("Value not equal for (%s: %s) != (%s: %s)", key, val1, key2, val2)
}
}
return nil
}

View File

@@ -0,0 +1,83 @@
package natsjskv
import (
"time"
"github.com/nats-io/nats.go"
"go-micro.dev/v5/store"
)
// store.Option.
type natsOptionsKey struct{}
type jsOptionsKey struct{}
type kvOptionsKey struct{}
type ttlOptionsKey struct{}
type memoryOptionsKey struct{}
type descriptionOptionsKey struct{}
type keyEncodeOptionsKey struct{}
// NatsOptions accepts nats.Options.
func NatsOptions(opts nats.Options) store.Option {
return setStoreOption(natsOptionsKey{}, opts)
}
// JetStreamOptions accepts multiple nats.JSOpt.
func JetStreamOptions(opts ...nats.JSOpt) store.Option {
return setStoreOption(jsOptionsKey{}, opts)
}
// KeyValueOptions accepts multiple nats.KeyValueConfig
// This will create buckets with the provided configs at initialization.
func KeyValueOptions(cfg ...*nats.KeyValueConfig) store.Option {
return setStoreOption(kvOptionsKey{}, cfg)
}
// DefaultTTL sets the default TTL to use for new buckets
//
// By default no TTL is set.
//
// TTL ON INDIVIDUAL WRITE CALLS IS NOT SUPPORTED, only bucket wide TTL.
// Either set a default TTL with this option or provide bucket specific options
//
// with ObjectStoreOptions
func DefaultTTL(ttl time.Duration) store.Option {
return setStoreOption(ttlOptionsKey{}, ttl)
}
// DefaultMemory sets the default storage type to memory only.
//
// The default is file storage, persisting storage between service restarts.
//
// Be aware that the default storage location of NATS the /tmp dir is, and thus
//
// won't persist reboots.
func DefaultMemory() store.Option {
return setStoreOption(memoryOptionsKey{}, nats.MemoryStorage)
}
// DefaultDescription sets the default description to use when creating new
//
// buckets. The default is "Store managed by go-micro"
func DefaultDescription(text string) store.Option {
return setStoreOption(descriptionOptionsKey{}, text)
}
// EncodeKeys will "base32" encode the keys.
// This is to work around limited characters usable as keys for the natsjs kv store.
// See details here: https://docs.nats.io/nats-concepts/subjects#characters-allowed-for-subject-names
func EncodeKeys() store.Option {
return setStoreOption(keyEncodeOptionsKey{}, "base32")
}
// DeleteBucket will use the key passed to Delete as a bucket (database) name,
//
// and delete the bucket.
//
// This option should not be combined with the store.DeleteFrom option, as
//
// that will overwrite the delete action.
func DeleteBucket() store.DeleteOption {
return func(d *store.DeleteOptions) {
d.Table = "DELETE_BUCKET"
}
}

View File

@@ -0,0 +1,138 @@
package natsjskv
import "go-micro.dev/v5/store"
type test struct {
Record *store.Record
Database string
Table string
}
var (
table = []test{
{
Record: &store.Record{
Key: "One",
Value: []byte("First value"),
},
},
{
Record: &store.Record{
Key: "Two",
Value: []byte("Second value"),
},
Table: "prefix_test",
},
{
Record: &store.Record{
Key: "Third",
Value: []byte("Third value"),
},
Database: "new-bucket",
},
{
Record: &store.Record{
Key: "Four",
Value: []byte("Fourth value"),
},
Database: "new-bucket",
Table: "prefix_test",
},
{
Record: &store.Record{
Key: "empty-value",
Value: []byte{},
},
Database: "new-bucket",
},
{
Record: &store.Record{
Key: "Alex",
Value: []byte("Some value"),
},
Database: "prefix-test",
Table: "names",
},
{
Record: &store.Record{
Key: "Jones",
Value: []byte("Some value"),
},
Database: "prefix-test",
Table: "names",
},
{
Record: &store.Record{
Key: "Adrianna",
Value: []byte("Some value"),
},
Database: "prefix-test",
Table: "names",
},
{
Record: &store.Record{
Key: "MexicoCity",
Value: []byte("Some value"),
},
Database: "prefix-test",
Table: "cities",
},
{
Record: &store.Record{
Key: "HoustonCity",
Value: []byte("Some value"),
},
Database: "prefix-test",
Table: "cities",
},
{
Record: &store.Record{
Key: "ZurichCity",
Value: []byte("Some value"),
},
Database: "prefix-test",
Table: "cities",
},
{
Record: &store.Record{
Key: "Helsinki",
Value: []byte("Some value"),
},
Database: "prefix-test",
Table: "cities",
},
{
Record: &store.Record{
Key: "testKeytest",
Value: []byte("Some value"),
},
Table: "some_table",
},
{
Record: &store.Record{
Key: "testSecondtest",
Value: []byte("Some value"),
},
Table: "some_table",
},
{
Record: &store.Record{
Key: "lalala",
Value: []byte("Some value"),
},
Table: "some_table",
},
{
Record: &store.Record{
Key: "testAnothertest",
Value: []byte("Some value"),
},
},
{
Record: &store.Record{
Key: "FobiddenCharactersAreAllowed:|@..+",
Value: []byte("data no matter"),
},
},
}
)